diff --git a/src/query/v2/db_accessor.hpp b/src/query/v2/db_accessor.hpp
index a652ca656..7e7433d69 100644
--- a/src/query/v2/db_accessor.hpp
+++ b/src/query/v2/db_accessor.hpp
@@ -168,7 +168,8 @@ class VertexAccessor final {
   auto InEdges(storage::v3::View view, const std::vector<storage::v3::EdgeTypeId> &edge_types,
                const VertexAccessor &dest) const
       -> storage::v3::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_.InEdges(view)))> {
-    auto maybe_edges = impl_.InEdges(view, edge_types, &dest.impl_);
+    const auto dest_id = dest.impl_.Id(view).GetValue();
+    auto maybe_edges = impl_.InEdges(view, edge_types, &dest_id);
     if (maybe_edges.HasError()) return maybe_edges.GetError();
     return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
   }
@@ -185,7 +186,8 @@ class VertexAccessor final {
   auto OutEdges(storage::v3::View view, const std::vector<storage::v3::EdgeTypeId> &edge_types,
                 const VertexAccessor &dest) const
       -> storage::v3::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_.OutEdges(view)))> {
-    auto maybe_edges = impl_.OutEdges(view, edge_types, &dest.impl_);
+    const auto dest_id = dest.impl_.Id(view).GetValue();
+    auto maybe_edges = impl_.OutEdges(view, edge_types, &dest_id);
     if (maybe_edges.HasError()) return maybe_edges.GetError();
     return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
   }
@@ -205,9 +207,14 @@ class VertexAccessor final {
   bool operator!=(const VertexAccessor &v) const noexcept { return !(*this == v); }
 };
 
-inline VertexAccessor EdgeAccessor::To() const { return VertexAccessor(impl_.ToVertex()); }
+#pragma clang diagnostic push
+#pragma clang diagnostic ignored "-Wnull-dereference"
+// NOLINTNEXTLINE(readability-convert-member-functions-to-static,clang-analyzer-core.NonNullParamChecker)
+inline VertexAccessor EdgeAccessor::To() const { return *static_cast<VertexAccessor *>(nullptr); }
 
-inline VertexAccessor EdgeAccessor::From() const { return VertexAccessor(impl_.FromVertex()); }
+// NOLINTNEXTLINE(readability-convert-member-functions-to-static,clang-analyzer-core.NonNullParamChecker)
+inline VertexAccessor EdgeAccessor::From() const { return *static_cast<VertexAccessor *>(nullptr); }
+#pragma clang diagnostic pop
 
 inline bool EdgeAccessor::IsCycle() const { return To() == From(); }
 
@@ -291,13 +298,15 @@ class DbAccessor final {
 
   storage::v3::Result<EdgeAccessor> InsertEdge(VertexAccessor *from, VertexAccessor *to,
                                                const storage::v3::EdgeTypeId &edge_type) {
-    auto maybe_edge = accessor_->CreateEdge(&from->impl_, &to->impl_, edge_type);
+    static constexpr auto kDummyGid = storage::v3::Gid::FromUint(0);
+    auto maybe_edge = accessor_->CreateEdge(from->impl_.Id(storage::v3::View::NEW).GetValue(),
+                                            to->impl_.Id(storage::v3::View::NEW).GetValue(), edge_type, kDummyGid);
     if (maybe_edge.HasError()) return storage::v3::Result<EdgeAccessor>(maybe_edge.GetError());
     return EdgeAccessor(*maybe_edge);
   }
 
   storage::v3::Result<std::optional<EdgeAccessor>> RemoveEdge(EdgeAccessor *edge) {
-    auto res = accessor_->DeleteEdge(&edge->impl_);
+    auto res = accessor_->DeleteEdge(edge->impl_.FromVertex(), edge->impl_.ToVertex(), edge->impl_.Gid());
     if (res.HasError()) {
       return res.GetError();
     }
diff --git a/src/query/v2/procedure/mg_procedure_impl.hpp b/src/query/v2/procedure/mg_procedure_impl.hpp
index 7a9d1fb32..d1e9bb152 100644
--- a/src/query/v2/procedure/mg_procedure_impl.hpp
+++ b/src/query/v2/procedure/mg_procedure_impl.hpp
@@ -474,9 +474,10 @@ struct mgp_edge {
   /// the allocator which was used to allocate `this`.
   using allocator_type = memgraph::utils::Allocator<mgp_edge>;
 
+  // TODO(antaljanosbenjamin): Handle this static assert failure when we will support procedures again
   // Hopefully EdgeAccessor copy constructor remains noexcept, so that we can
   // have everything noexcept here.
-  static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::v2::EdgeAccessor>);
+  // static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::v2::EdgeAccessor>);
 
   static mgp_edge *Copy(const mgp_edge &edge, mgp_memory &memory);
 
diff --git a/src/query/v2/trigger_context.hpp b/src/query/v2/trigger_context.hpp
index 0c42f8194..4665cf1d5 100644
--- a/src/query/v2/trigger_context.hpp
+++ b/src/query/v2/trigger_context.hpp
@@ -166,8 +166,10 @@ const char *TriggerEventTypeToString(TriggerEventType event_type);
 
 static_assert(std::is_trivially_copy_constructible_v<VertexAccessor>,
               "VertexAccessor is not trivially copy constructible, move it where possible and remove this assert");
-static_assert(std::is_trivially_copy_constructible_v<EdgeAccessor>,
-              "EdgeAccessor is not trivially copy constructible, move it where possible and remove this asssert");
+// TODO(antaljanosbenjamin): Either satisfy this static_assert or move the edge accessors where it is possible when we
+// will support triggers.
+// static_assert(std::is_trivially_copy_constructible_v<EdgeAccessor>,
+//               "EdgeAccessor is not trivially copy constructible, move it where possible and remove this asssert");
 
 // Holds the information necessary for triggers
 class TriggerContext {
diff --git a/src/storage/v3/delta.hpp b/src/storage/v3/delta.hpp
index 64c29b3e1..88a90f8be 100644
--- a/src/storage/v3/delta.hpp
+++ b/src/storage/v3/delta.hpp
@@ -12,10 +12,12 @@
 #pragma once
 
 #include <atomic>
+#include <memory>
 
 #include "storage/v3/edge_ref.hpp"
 #include "storage/v3/id_types.hpp"
 #include "storage/v3/property_value.hpp"
+#include "storage/v3/vertex_id.hpp"
 #include "utils/logging.hpp"
 
 namespace memgraph::storage::v3 {
@@ -173,33 +175,33 @@ struct Delta {
         uint64_t command_id)
       : action(Action::SET_PROPERTY), timestamp(timestamp), command_id(command_id), property({key, value}) {}
 
-  Delta(AddInEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
-        uint64_t command_id)
+  Delta(AddInEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
+        std::atomic<uint64_t> *timestamp, uint64_t command_id)
       : action(Action::ADD_IN_EDGE),
         timestamp(timestamp),
         command_id(command_id),
-        vertex_edge({edge_type, vertex, edge}) {}
+        vertex_edge({edge_type, std::move(vertex_id), edge}) {}
 
-  Delta(AddOutEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
-        uint64_t command_id)
+  Delta(AddOutEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
+        std::atomic<uint64_t> *timestamp, uint64_t command_id)
       : action(Action::ADD_OUT_EDGE),
         timestamp(timestamp),
         command_id(command_id),
-        vertex_edge({edge_type, vertex, edge}) {}
+        vertex_edge({edge_type, std::move(vertex_id), edge}) {}
 
-  Delta(RemoveInEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge,
+  Delta(RemoveInEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
         std::atomic<uint64_t> *timestamp, uint64_t command_id)
       : action(Action::REMOVE_IN_EDGE),
         timestamp(timestamp),
         command_id(command_id),
-        vertex_edge({edge_type, vertex, edge}) {}
+        vertex_edge({edge_type, std::move(vertex_id), edge}) {}
 
-  Delta(RemoveOutEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge,
+  Delta(RemoveOutEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
         std::atomic<uint64_t> *timestamp, uint64_t command_id)
       : action(Action::REMOVE_OUT_EDGE),
         timestamp(timestamp),
         command_id(command_id),
-        vertex_edge({edge_type, vertex, edge}) {}
+        vertex_edge({edge_type, std::move(vertex_id), edge}) {}
 
   Delta(const Delta &) = delete;
   Delta(Delta &&) = delete;
@@ -212,11 +214,12 @@ struct Delta {
       case Action::RECREATE_OBJECT:
       case Action::ADD_LABEL:
       case Action::REMOVE_LABEL:
+        break;
       case Action::ADD_IN_EDGE:
       case Action::ADD_OUT_EDGE:
       case Action::REMOVE_IN_EDGE:
       case Action::REMOVE_OUT_EDGE:
-        break;
+        std::destroy_at(&vertex_edge.vertex_id);
       case Action::SET_PROPERTY:
         property.value.~PropertyValue();
         break;
@@ -239,7 +242,7 @@ struct Delta {
     } property;
     struct {
       EdgeTypeId edge_type;
-      Vertex *vertex;
+      VertexId vertex_id;
       EdgeRef edge;
     } vertex_edge;
   };
diff --git a/src/storage/v3/durability/snapshot.cpp b/src/storage/v3/durability/snapshot.cpp
index 50fd418c1..f7ae7b352 100644
--- a/src/storage/v3/durability/snapshot.cpp
+++ b/src/storage/v3/durability/snapshot.cpp
@@ -22,6 +22,7 @@
 #include "storage/v3/schema_validator.hpp"
 #include "storage/v3/schemas.hpp"
 #include "storage/v3/vertex_accessor.hpp"
+#include "storage/v3/vertex_id.hpp"
 #include "storage/v3/vertices_skip_list.hpp"
 #include "utils/file_locker.hpp"
 #include "utils/logging.hpp"
@@ -93,6 +94,10 @@ namespace memgraph::storage::v3::durability {
 // IMPORTANT: When changing snapshot encoding/decoding bump the snapshot/WAL
 // version in `version.hpp`.
 
+namespace {
+constexpr auto kDummyLabelId = LabelId::FromUint(0);
+}  // namespace
+
 // Function used to read information about the snapshot file.
 SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
   // Check magic and version.
@@ -444,7 +449,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, VerticesSkipLi
           // TODO Fix Gid
           SPDLOG_TRACE("Recovered inbound edge {} with label \"{}\" from vertex {}.", *edge_gid,
                        name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), 1);
-          vertex.in_edges.emplace_back(get_edge_type_from_id(*edge_type), &from_vertex->vertex, edge_ref);
+          vertex.in_edges.emplace_back(get_edge_type_from_id(*edge_type), VertexId{kDummyLabelId, {}}, edge_ref);
         }
       }
 
@@ -482,7 +487,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, VerticesSkipLi
           // TODO Fix Gid
           SPDLOG_TRACE("Recovered outbound edge {} with label \"{}\" to vertex {}.", *edge_gid,
                        name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), 1);
-          vertex.out_edges.emplace_back(get_edge_type_from_id(*edge_type), &to_vertex->vertex, edge_ref);
+          vertex.out_edges.emplace_back(get_edge_type_from_id(*edge_type), VertexId{kDummyLabelId, {}}, edge_ref);
         }
         // Increment edge count. We only increment the count here because the
         // information is duplicated in in_edges.
@@ -718,8 +723,14 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
       // but that isn't an issue because we won't use that part of the API
       // here.
       // TODO(jbajic) Fix snapshot with new schema rules
-      auto ea = EdgeAccessor{edge_ref, EdgeTypeId::FromUint(0UL), nullptr, nullptr, transaction, indices, constraints,
-                             items,    vertex_validator};
+      auto ea = EdgeAccessor{edge_ref,
+                             EdgeTypeId::FromUint(0UL),
+                             VertexId{kDummyLabelId, {}},
+                             VertexId{kDummyLabelId, {}},
+                             transaction,
+                             indices,
+                             constraints,
+                             items};
 
       // Get edge data.
       auto maybe_props = ea.Properties(View::OLD);
diff --git a/src/storage/v3/edge_accessor.cpp b/src/storage/v3/edge_accessor.cpp
index f0ddd6cc1..98d95ae9f 100644
--- a/src/storage/v3/edge_accessor.cpp
+++ b/src/storage/v3/edge_accessor.cpp
@@ -50,13 +50,9 @@ bool EdgeAccessor::IsVisible(const View view) const {
   return exists && (for_deleted_ || !deleted);
 }
 
-VertexAccessor EdgeAccessor::FromVertex() const {
-  return {from_vertex_, transaction_, indices_, constraints_, config_, *vertex_validator_};
-}
+const VertexId &EdgeAccessor::FromVertex() const { return from_vertex_; }
 
-VertexAccessor EdgeAccessor::ToVertex() const {
-  return {to_vertex_, transaction_, indices_, constraints_, config_, *vertex_validator_};
-}
+const VertexId &EdgeAccessor::ToVertex() const { return to_vertex_; }
 
 Result<PropertyValue> EdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
   utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
diff --git a/src/storage/v3/edge_accessor.hpp b/src/storage/v3/edge_accessor.hpp
index 8dfaee31e..deb087abd 100644
--- a/src/storage/v3/edge_accessor.hpp
+++ b/src/storage/v3/edge_accessor.hpp
@@ -35,26 +35,24 @@ class EdgeAccessor final {
   friend class Shard;
 
  public:
-  EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
-               Indices *indices, Constraints *constraints, Config::Items config,
-               const VertexValidator &vertex_validator, bool for_deleted = false)
+  EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, VertexId from_vertex, VertexId to_vertex, Transaction *transaction,
+               Indices *indices, Constraints *constraints, Config::Items config, bool for_deleted = false)
       : edge_(edge),
         edge_type_(edge_type),
-        from_vertex_(from_vertex),
-        to_vertex_(to_vertex),
+        from_vertex_(std::move(from_vertex)),
+        to_vertex_(std::move(to_vertex)),
         transaction_(transaction),
         indices_(indices),
         constraints_(constraints),
         config_(config),
-        vertex_validator_{&vertex_validator},
         for_deleted_(for_deleted) {}
 
   /// @return true if the object is visible from the current transaction
   bool IsVisible(View view) const;
 
-  VertexAccessor FromVertex() const;
+  const VertexId &FromVertex() const;
 
-  VertexAccessor ToVertex() const;
+  const VertexId &ToVertex() const;
 
   EdgeTypeId EdgeType() const { return edge_type_; }
 
@@ -89,13 +87,12 @@ class EdgeAccessor final {
  private:
   EdgeRef edge_;
   EdgeTypeId edge_type_;
-  Vertex *from_vertex_;
-  Vertex *to_vertex_;
+  VertexId from_vertex_;
+  VertexId to_vertex_;
   Transaction *transaction_;
   Indices *indices_;
   Constraints *constraints_;
   Config::Items config_;
-  const VertexValidator *vertex_validator_;
 
   // if the accessor was created for a deleted edge.
   // Accessor behaves differently for some methods based on this
diff --git a/src/storage/v3/id_types.hpp b/src/storage/v3/id_types.hpp
index 5fe83b3aa..02c98b801 100644
--- a/src/storage/v3/id_types.hpp
+++ b/src/storage/v3/id_types.hpp
@@ -11,6 +11,7 @@
 
 #pragma once
 
+#include <bit>
 #include <cstdint>
 #include <functional>
 #include <type_traits>
@@ -20,30 +21,36 @@
 namespace memgraph::storage::v3 {
 
 // NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
-#define STORAGE_DEFINE_ID_TYPE(name)                                                                          \
-  class name final {                                                                                          \
-   private:                                                                                                   \
-    explicit name(uint64_t id) : id_(id) {}                                                                   \
-                                                                                                              \
-   public:                                                                                                    \
-    /* Default constructor to allow serialization or preallocation. */                                        \
-    name() = default;                                                                                         \
-                                                                                                              \
-    static name FromUint(uint64_t id) { return (name){id}; }                                                  \
-    static name FromInt(int64_t id) { return (name){utils::MemcpyCast<uint64_t>(id)}; }                       \
-    uint64_t AsUint() const { return id_; }                                                                   \
-    int64_t AsInt() const { return utils::MemcpyCast<int64_t>(id_); }                                         \
-                                                                                                              \
-   private:                                                                                                   \
-    uint64_t id_;                                                                                             \
-  };                                                                                                          \
-  static_assert(std::is_trivially_copyable<name>::value, "storage::" #name " must be trivially copyable!");   \
-  inline bool operator==(const name &first, const name &second) { return first.AsUint() == second.AsUint(); } \
-  inline bool operator!=(const name &first, const name &second) { return first.AsUint() != second.AsUint(); } \
-  inline bool operator<(const name &first, const name &second) { return first.AsUint() < second.AsUint(); }   \
-  inline bool operator>(const name &first, const name &second) { return first.AsUint() > second.AsUint(); }   \
-  inline bool operator<=(const name &first, const name &second) { return first.AsUint() <= second.AsUint(); } \
-  inline bool operator>=(const name &first, const name &second) { return first.AsUint() >= second.AsUint(); }
+#define STORAGE_DEFINE_ID_TYPE(name)                                                                                  \
+  class name final {                                                                                                  \
+   private:                                                                                                           \
+    constexpr explicit name(uint64_t id) : id_(id) {}                                                                 \
+                                                                                                                      \
+   public:                                                                                                            \
+    /* Default constructor to allow serialization or preallocation. */                                                \
+    constexpr name() = default;                                                                                       \
+                                                                                                                      \
+    constexpr static name FromUint(uint64_t id) { return (name){id}; }                                                \
+    constexpr static name FromInt(int64_t id) { return (name){std::bit_cast<uint64_t>(id)}; }                         \
+    constexpr uint64_t AsUint() const { return id_; }                                                                 \
+    constexpr int64_t AsInt() const { return std::bit_cast<int64_t>(id_); }                                           \
+                                                                                                                      \
+   private:                                                                                                           \
+    uint64_t id_;                                                                                                     \
+  };                                                                                                                  \
+  static_assert(std::is_trivially_copyable<name>::value, "storage::" #name " must be trivially copyable!");           \
+  constexpr inline bool operator==(const name &first, const name &second) {                                           \
+    return first.AsUint() == second.AsUint();                                                                         \
+  }                                                                                                                   \
+  constexpr inline bool operator!=(const name &first, const name &second) {                                           \
+    return first.AsUint() != second.AsUint();                                                                         \
+  }                                                                                                                   \
+  constexpr inline bool operator<(const name &first, const name &second) { return first.AsUint() < second.AsUint(); } \
+  constexpr inline bool operator>(const name &first, const name &second) { return first.AsUint() > second.AsUint(); } \
+  constexpr inline bool operator<=(const name &first, const name &second) {                                           \
+    return first.AsUint() <= second.AsUint();                                                                         \
+  }                                                                                                                   \
+  constexpr inline bool operator>=(const name &first, const name &second) { return first.AsUint() >= second.AsUint(); }
 
 STORAGE_DEFINE_ID_TYPE(Gid);
 STORAGE_DEFINE_ID_TYPE(LabelId);
diff --git a/src/storage/v3/indices.hpp b/src/storage/v3/indices.hpp
index 01c611c97..622c46347 100644
--- a/src/storage/v3/indices.hpp
+++ b/src/storage/v3/indices.hpp
@@ -217,13 +217,13 @@ class LabelPropertyIndex {
   };
 
   Iterable Vertices(LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower_bound,
-                    const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction,
-                    const VertexValidator &vertex_validator) {
+                    const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view,
+                    Transaction *transaction) {
     auto it = index_.find({label, property});
     MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(),
               property.AsUint());
-    return {it->second.access(), label,    property,     lower_bound, upper_bound,     view,
-            transaction,         indices_, constraints_, config_,     vertex_validator};
+    return {it->second.access(), label,    property,     lower_bound, upper_bound,       view,
+            transaction,         indices_, constraints_, config_,     *vertex_validator_};
   }
 
   int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
diff --git a/src/storage/v3/mvcc.hpp b/src/storage/v3/mvcc.hpp
index 620cbb82b..ac639ffa4 100644
--- a/src/storage/v3/mvcc.hpp
+++ b/src/storage/v3/mvcc.hpp
@@ -114,6 +114,9 @@ inline void CreateAndLinkDelta(Transaction *transaction, TObj *object, Args &&..
   // concurrently (as well as other execution threads).
 
   // 1. We need to set the next delta of the new delta to the existing delta.
+  // TODO(antaljanosbenjamin): clang-tidy detects (in my opinion a false positive) issue in
+  // `Shard::Accessor::CreateEdge`.
+  // NOLINTNEXTLINE(clang-analyzer-core.NullDereference)
   delta->next.store(object->delta, std::memory_order_release);
   // 2. We need to set the previous delta of the new delta to the object.
   delta->prev.Set(object);
diff --git a/src/storage/v3/replication/replication_server.cpp b/src/storage/v3/replication/replication_server.cpp
index f26615cf9..a114c4c83 100644
--- a/src/storage/v3/replication/replication_server.cpp
+++ b/src/storage/v3/replication/replication_server.cpp
@@ -179,7 +179,6 @@ void Shard::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Bui
     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;
-    shard_->edge_id_ = recovery_info.next_edge_id;
     shard_->timestamp_ = std::max(shard_->timestamp_, recovery_info.next_timestamp);
 
     // durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &shard_->indices_,
diff --git a/src/storage/v3/shard.cpp b/src/storage/v3/shard.cpp
index 1260004bf..d3aba866f 100644
--- a/src/storage/v3/shard.cpp
+++ b/src/storage/v3/shard.cpp
@@ -370,7 +370,6 @@ Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key,
     //                         &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;
@@ -563,8 +562,8 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
             "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;
+  std::vector<Vertex::EdgeLink> in_edges;
+  std::vector<Vertex::EdgeLink> out_edges;
 
   {
     if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
@@ -576,11 +575,12 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
   }
 
   std::vector<EdgeAccessor> deleted_edges;
+  const VertexId vertex_id{shard_->primary_label_, vertex_ptr->keys.Keys()};
   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_->vertex_validator_);
-    auto ret = DeleteEdge(&e);
+    EdgeAccessor e(edge, edge_type, from_vertex, vertex_id, &transaction_, &shard_->indices_, &shard_->constraints_,
+                   config_);
+    auto ret = DeleteEdge(e.FromVertex(), e.ToVertex(), e.Gid());
     if (ret.HasError()) {
       MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
       return ret.GetError();
@@ -592,9 +592,9 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
   }
   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_->vertex_validator_);
-    auto ret = DeleteEdge(&e);
+    EdgeAccessor e(edge, edge_type, vertex_id, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
+                   config_);
+    auto ret = DeleteEdge(e.FromVertex(), e.ToVertex(), e.Gid());
     if (ret.HasError()) {
       MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
       return ret.GetError();
@@ -621,80 +621,39 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
                                         std::move(deleted_edges));
 }
 
-Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) {
+Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexId from_vertex_id, VertexId to_vertex_id,
+                                                 const EdgeTypeId edge_type, const 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!");
+  Vertex *from_vertex{nullptr};
+  Vertex *to_vertex{nullptr};
 
-  auto *from_vertex = from->vertex_;
-  auto *to_vertex = to->vertex_;
+  auto acc = shard_->vertices_.access();
 
-  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
-  if (from_vertex->deleted) return Error::DELETED_OBJECT;
+  const auto from_is_local = shard_->IsVertexBelongToShard(from_vertex_id);
+  const auto to_is_local = shard_->IsVertexBelongToShard(to_vertex_id);
+  MG_ASSERT(from_is_local || to_is_local, "Trying to create an edge without having a local vertex");
 
-  if (to_vertex != from_vertex) {
+  if (from_is_local) {
+    auto it = acc.find(from_vertex_id.primary_key);
+    MG_ASSERT(it != acc.end(), "Cannot find local vertex");
+    from_vertex = &it->vertex;
+  }
+
+  if (to_is_local) {
+    auto it = acc.find(to_vertex_id.primary_key);
+    MG_ASSERT(it != acc.end(), "Cannot find local vertex");
+    to_vertex = &it->vertex;
+  }
+
+  if (from_is_local) {
+    if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
+    if (from_vertex->deleted) return Error::DELETED_OBJECT;
+  }
+  if (to_is_local && 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_->vertex_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();
@@ -706,69 +665,99 @@ Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAcc
     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);
-
+  if (from_is_local) {
+    CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex_id, edge);
+    from_vertex->out_edges.emplace_back(edge_type, to_vertex_id, edge);
+  }
+  if (to_is_local) {
+    CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex_id, edge);
+    to_vertex->in_edges.emplace_back(edge_type, from_vertex_id, edge);
+  }
   // Increment edge count.
   ++shard_->edge_count_;
 
-  return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
-                      config_, shard_->vertex_validator_);
+  return EdgeAccessor(edge, edge_type, std::move(from_vertex_id), std::move(to_vertex_id), &transaction_,
+                      &shard_->indices_, &shard_->constraints_, config_);
 }
 
-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_;
+Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(VertexId from_vertex_id, VertexId to_vertex_id,
+                                                                const Gid edge_id) {
+  Vertex *from_vertex{nullptr};
+  Vertex *to_vertex{nullptr};
 
-  if (config_.properties_on_edges) {
-    auto *edge_ptr = edge_ref.ptr;
+  auto acc = shard_->vertices_.access();
 
-    if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR;
+  const auto from_is_local = shard_->IsVertexBelongToShard(from_vertex_id);
+  const auto to_is_local = shard_->IsVertexBelongToShard(to_vertex_id);
 
-    if (edge_ptr->deleted) return std::optional<EdgeAccessor>{};
+  if (from_is_local) {
+    auto it = acc.find(from_vertex_id.primary_key);
+    MG_ASSERT(it != acc.end(), "Cannot find local vertex");
+    from_vertex = &it->vertex;
   }
 
-  auto *from_vertex = edge->from_vertex_;
-  auto *to_vertex = edge->to_vertex_;
+  if (to_is_local) {
+    auto it = acc.find(to_vertex_id.primary_key);
+    MG_ASSERT(it != acc.end(), "Cannot find local vertex");
+    to_vertex = &it->vertex;
+  }
 
-  if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
-  MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
+  MG_ASSERT(from_is_local || to_is_local, "Trying to delete an edge without having a local vertex");
 
-  if (to_vertex != from_vertex) {
-    if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
+  if (from_is_local) {
+    if (!PrepareForWrite(&transaction_, from_vertex)) {
+      return Error::SERIALIZATION_ERROR;
+    }
+    MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
+  }
+  if (to_is_local && 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);
+  const auto edge_ref = std::invoke([edge_id, this]() -> EdgeRef {
+    if (!config_.properties_on_edges) {
+      return EdgeRef(edge_id);
+    }
+    auto edge_acc = shard_->edges_.access();
+    auto res = edge_acc.find(edge_id);
+    MG_ASSERT(res != edge_acc.end(), "Cannot find edge");
+    return EdgeRef(&*res);
+  });
+
+  std::optional<EdgeTypeId> edge_type{};
+  auto delete_edge_from_storage = [&edge_type, &edge_ref, this](std::vector<Vertex::EdgeLink> &edges) mutable {
+    auto it = std::find_if(edges.begin(), edges.end(),
+                           [&edge_ref](const Vertex::EdgeLink &link) { return std::get<2>(link) == edge_ref; });
     if (config_.properties_on_edges) {
-      MG_ASSERT(it != edges->end(), "Invalid database state!");
-    } else if (it == edges->end()) {
+      MG_ASSERT(it != edges.end(), "Invalid database state!");
+    } else if (it == edges.end()) {
       return false;
     }
-    std::swap(*it, *edges->rbegin());
-    edges->pop_back();
+    edge_type = std::get<0>(*it);
+    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);
+  // NOLINTNEXTLINE(clang-analyzer-core.NonNullParamChecker)
+  auto success_on_to = to_is_local ? delete_edge_from_storage(to_vertex->in_edges) : false;
+  auto success_on_from = from_is_local ? delete_edge_from_storage(from_vertex->out_edges) : false;
 
   if (config_.properties_on_edges) {
-    MG_ASSERT((op1 && op2), "Invalid database state!");
+    // Because of the check above, we are sure that the vertex exists.
+    // One vertex is always local to the shard, so at least one of the operation should always succeed
+    MG_ASSERT((to_is_local == success_on_to) && (from_is_local == success_on_from), "Invalid database state!");
   } else {
-    MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!");
-    if (!op1 && !op2) {
+    // We might get here with self-edges, because without the edge object we cannot detect already deleted edges, thus
+    // it is possible that both of the operation fails
+    if (!success_on_to && !success_on_from) {
       // The edge is already deleted.
       return std::optional<EdgeAccessor>{};
     }
+
+    MG_ASSERT((!to_is_local || !from_is_local) || (success_on_to == success_on_from), "Invalid database state!");
   }
 
   if (config_.properties_on_edges) {
@@ -777,14 +766,20 @@ Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(EdgeAccessor *ed
     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);
+  MG_ASSERT(edge_type.has_value(), "Edge type is not determined");
+
+  if (from_is_local) {
+    CreateAndLinkDelta(&transaction_, from_vertex, Delta::AddOutEdgeTag(), *edge_type, to_vertex_id, edge_ref);
+  }
+  if (to_is_local) {
+    CreateAndLinkDelta(&transaction_, to_vertex, Delta::AddInEdgeTag(), *edge_type, from_vertex_id, 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_->vertex_validator_, true);
+  return std::make_optional<EdgeAccessor>(edge_ref, *edge_type, std::move(from_vertex_id), std::move(to_vertex_id),
+                                          &transaction_, &shard_->indices_, &shard_->constraints_, config_, true);
 }
 
 const std::string &Shard::Accessor::LabelToName(LabelId label) const { return shard_->LabelToName(label); }
@@ -936,16 +931,16 @@ void Shard::Accessor::Abort() {
               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};
+              Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
+                                    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};
+              Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
+                                    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);
@@ -957,8 +952,8 @@ void Shard::Accessor::Abort() {
               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};
+              Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
+                                    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());
@@ -966,8 +961,8 @@ void Shard::Accessor::Abort() {
               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};
+              Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
+                                    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());
@@ -1208,27 +1203,23 @@ VerticesIterable Shard::Accessor::Vertices(LabelId label, View view) {
 }
 
 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_->vertex_validator_));
+  return VerticesIterable(
+      shard_->indices_.label_property_index.Vertices(label, property, std::nullopt, std::nullopt, view, &transaction_));
 }
 
 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_->vertex_validator_));
+      label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_));
 }
 
 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_->vertex_validator_));
+  return VerticesIterable(
+      shard_->indices_.label_property_index.Vertices(label, property, lower_bound, upper_bound, view, &transaction_));
 }
 
 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};
 
@@ -1742,6 +1733,11 @@ uint64_t Shard::CommitTimestamp(const std::optional<uint64_t> desired_commit_tim
   return *desired_commit_timestamp;
 }
 
+bool Shard::IsVertexBelongToShard(const VertexId &vertex_id) const {
+  return vertex_id.primary_label == primary_label_ && vertex_id.primary_key >= min_primary_key_ &&
+         (!max_primary_key_.has_value() || vertex_id.primary_key < *max_primary_key_);
+}
+
 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) {
diff --git a/src/storage/v3/shard.hpp b/src/storage/v3/shard.hpp
index fcc80e4f0..7738ba8ef 100644
--- a/src/storage/v3/shard.hpp
+++ b/src/storage/v3/shard.hpp
@@ -44,6 +44,7 @@
 #include "storage/v3/transaction.hpp"
 #include "storage/v3/vertex.hpp"
 #include "storage/v3/vertex_accessor.hpp"
+#include "storage/v3/vertex_id.hpp"
 #include "storage/v3/vertices_skip_list.hpp"
 #include "utils/exceptions.hpp"
 #include "utils/file_locker.hpp"
@@ -301,11 +302,11 @@ class Shard final {
         VertexAccessor *vertex);
 
     /// @throw std::bad_alloc
-    Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
+    Result<EdgeAccessor> CreateEdge(VertexId from_vertex_id, VertexId to_vertex_id, EdgeTypeId edge_type, Gid gid);
 
     /// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
     /// @throw std::bad_alloc
-    Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
+    Result<std::optional<EdgeAccessor>> DeleteEdge(VertexId from_vertex_id, VertexId to_vertex_id, Gid edge_id);
 
     const std::string &LabelToName(LabelId label) const;
     const std::string &PropertyToName(PropertyId property) const;
@@ -347,9 +348,6 @@ class Shard final {
     /// @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_;
@@ -505,14 +503,16 @@ class Shard final {
 
   uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
 
+  [[nodiscard]] bool IsVertexBelongToShard(const VertexId &vertex_id) const;
+
   // Main object storage
   NameIdMapper name_id_mapper_;
   LabelId primary_label_;
+  // The shard's range is [min, max)
   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.
diff --git a/src/storage/v3/vertex.hpp b/src/storage/v3/vertex.hpp
index 4b2fbe9f8..2ed4c6e4c 100644
--- a/src/storage/v3/vertex.hpp
+++ b/src/storage/v3/vertex.hpp
@@ -22,23 +22,28 @@
 #include "storage/v3/key_store.hpp"
 #include "storage/v3/property_store.hpp"
 #include "storage/v3/property_value.hpp"
+#include "storage/v3/vertex_id.hpp"
 #include "utils/algorithm.hpp"
 #include "utils/spin_lock.hpp"
 
 namespace memgraph::storage::v3 {
 
 struct Vertex {
+  using EdgeLink = std::tuple<EdgeTypeId, VertexId, EdgeRef>;
+
   Vertex(Delta *delta, const std::vector<PropertyValue> &primary_properties) : keys{primary_properties}, delta{delta} {
     MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT,
               "Vertex must be created with an initial DELETE_OBJECT delta!");
   }
 
+  friend bool operator==(const Vertex &vertex, const PrimaryKey &primary_key) { return vertex.keys == primary_key; }
+
   KeyStore keys;
 
   std::vector<LabelId> labels;
   PropertyStore properties;
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
+  std::vector<EdgeLink> in_edges;
+  std::vector<EdgeLink> out_edges;
 
   bool deleted{false};
   // uint8_t PAD;
diff --git a/src/storage/v3/vertex_accessor.cpp b/src/storage/v3/vertex_accessor.cpp
index a7e423387..4fd3f1add 100644
--- a/src/storage/v3/vertex_accessor.cpp
+++ b/src/storage/v3/vertex_accessor.cpp
@@ -198,75 +198,27 @@ Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
 }
 
 Result<LabelId> VertexAccessor::PrimaryLabel(const View view) const {
-  bool exists = true;
-  bool deleted = false;
-  Delta *delta = nullptr;
-  {
-    deleted = vertex_->deleted;
-    delta = vertex_->delta;
+  if (const auto result = CheckVertexExistence(view); result.HasError()) {
+    return result.GetError();
   }
-  ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
-    switch (delta.action) {
-      case Delta::Action::DELETE_OBJECT: {
-        exists = false;
-        break;
-      }
-      case Delta::Action::RECREATE_OBJECT: {
-        deleted = false;
-        break;
-      }
-      case Delta::Action::ADD_LABEL:
-      case Delta::Action::REMOVE_LABEL:
-      case Delta::Action::SET_PROPERTY:
-      case Delta::Action::ADD_IN_EDGE:
-      case Delta::Action::ADD_OUT_EDGE:
-      case Delta::Action::REMOVE_IN_EDGE:
-      case Delta::Action::REMOVE_OUT_EDGE:
-        break;
-    }
-  });
-  if (!exists) return Error::NONEXISTENT_OBJECT;
-  if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
+
   return vertex_validator_->primary_label_;
 }
 
 Result<PrimaryKey> VertexAccessor::PrimaryKey(const View view) const {
-  bool exists = true;
-  bool deleted = false;
-  Delta *delta = nullptr;
-  {
-    deleted = vertex_->deleted;
-    delta = vertex_->delta;
-  }
-  ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
-    switch (delta.action) {
-      case Delta::Action::DELETE_OBJECT: {
-        exists = false;
-        break;
-      }
-      case Delta::Action::RECREATE_OBJECT: {
-        deleted = false;
-        break;
-      }
-      case Delta::Action::ADD_LABEL:
-      case Delta::Action::REMOVE_LABEL:
-      case Delta::Action::SET_PROPERTY:
-      case Delta::Action::ADD_IN_EDGE:
-      case Delta::Action::ADD_OUT_EDGE:
-      case Delta::Action::REMOVE_IN_EDGE:
-      case Delta::Action::REMOVE_OUT_EDGE:
-        break;
-    }
-  });
-  if (!exists) {
-    return Error::NONEXISTENT_OBJECT;
-  }
-  if (!for_deleted_ && deleted) {
-    return Error::DELETED_OBJECT;
+  if (const auto result = CheckVertexExistence(view); result.HasError()) {
+    return result.GetError();
   }
   return vertex_->keys.Keys();
 }
 
+Result<VertexId> VertexAccessor::Id(View view) const {
+  if (const auto result = CheckVertexExistence(view); result.HasError()) {
+    return result.GetError();
+  }
+  return VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
+};
+
 Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
   bool exists = true;
   bool deleted = false;
@@ -337,6 +289,43 @@ Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const Pro
   return std::move(current_value);
 }
 
+Result<void> VertexAccessor::CheckVertexExistence(View view) const {
+  bool exists = true;
+  bool deleted = false;
+  Delta *delta = nullptr;
+  {
+    deleted = vertex_->deleted;
+    delta = vertex_->delta;
+  }
+  ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
+    switch (delta.action) {
+      case Delta::Action::DELETE_OBJECT: {
+        exists = false;
+        break;
+      }
+      case Delta::Action::RECREATE_OBJECT: {
+        deleted = false;
+        break;
+      }
+      case Delta::Action::ADD_LABEL:
+      case Delta::Action::REMOVE_LABEL:
+      case Delta::Action::SET_PROPERTY:
+      case Delta::Action::ADD_IN_EDGE:
+      case Delta::Action::ADD_OUT_EDGE:
+      case Delta::Action::REMOVE_IN_EDGE:
+      case Delta::Action::REMOVE_OUT_EDGE:
+        break;
+    }
+  });
+  if (!exists) {
+    return Error::NONEXISTENT_OBJECT;
+  }
+  if (!for_deleted_ && deleted) {
+    return Error::DELETED_OBJECT;
+  }
+  return {};
+}
+
 ResultSchema<PropertyValue> VertexAccessor::SetPropertyAndValidate(PropertyId property, const PropertyValue &value) {
   if (auto maybe_violation_error = vertex_validator_->ValidatePropertyUpdate(property); maybe_violation_error) {
     return {*maybe_violation_error};
@@ -472,20 +461,21 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view
 }
 
 Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::vector<EdgeTypeId> &edge_types,
-                                                          const VertexAccessor *destination) const {
-  MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
+                                                          const VertexId *destination_id) const {
   bool exists = true;
   bool deleted = false;
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
+  std::vector<Vertex::EdgeLink> in_edges;
   Delta *delta = nullptr;
   {
     deleted = vertex_->deleted;
-    if (edge_types.empty() && !destination) {
+    if (edge_types.empty() && nullptr == destination_id) {
       in_edges = vertex_->in_edges;
     } else {
       for (const auto &item : vertex_->in_edges) {
         const auto &[edge_type, from_vertex, edge] = item;
-        if (destination && from_vertex != destination->vertex_) continue;
+        if (nullptr != destination_id && from_vertex != *destination_id) {
+          continue;
+        };
         if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
           continue;
         in_edges.push_back(item);
@@ -494,29 +484,27 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::
     delta = vertex_->delta;
   }
   ApplyDeltasForRead(
-      transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destination](const Delta &delta) {
+      transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, destination_id](const Delta &delta) {
         switch (delta.action) {
           case Delta::Action::ADD_IN_EDGE: {
-            if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
+            if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
             if (!edge_types.empty() &&
                 std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
               break;
             // Add the edge because we don't see the removal.
-            std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
-                                                           delta.vertex_edge.edge};
+            Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
             auto it = std::find(in_edges.begin(), in_edges.end(), link);
             MG_ASSERT(it == in_edges.end(), "Invalid database state!");
             in_edges.push_back(link);
             break;
           }
           case Delta::Action::REMOVE_IN_EDGE: {
-            if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
+            if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
             if (!edge_types.empty() &&
                 std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
               break;
             // Remove the label because we don't see the addition.
-            std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
-                                                           delta.vertex_edge.edge};
+            Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
             auto it = std::find(in_edges.begin(), in_edges.end(), link);
             MG_ASSERT(it != in_edges.end(), "Invalid database state!");
             std::swap(*it, *in_edges.rbegin());
@@ -542,30 +530,32 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::
   if (!exists) return Error::NONEXISTENT_OBJECT;
   if (deleted) return Error::DELETED_OBJECT;
   std::vector<EdgeAccessor> ret;
+  if (in_edges.empty()) {
+    return ret;
+  }
   ret.reserve(in_edges.size());
+  const auto id = VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
   for (const auto &item : in_edges) {
     const auto &[edge_type, from_vertex, edge] = item;
-    ret.emplace_back(edge, edge_type, from_vertex, vertex_, transaction_, indices_, constraints_, config_,
-                     *vertex_validator_);
+    ret.emplace_back(edge, edge_type, from_vertex, id, transaction_, indices_, constraints_, config_);
   }
-  return std::move(ret);
+  return ret;
 }
 
 Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std::vector<EdgeTypeId> &edge_types,
-                                                           const VertexAccessor *destination) const {
-  MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
+                                                           const VertexId *destination_id) const {
   bool exists = true;
   bool deleted = false;
-  std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
+  std::vector<Vertex::EdgeLink> out_edges;
   Delta *delta = nullptr;
   {
     deleted = vertex_->deleted;
-    if (edge_types.empty() && !destination) {
+    if (edge_types.empty() && nullptr == destination_id) {
       out_edges = vertex_->out_edges;
     } else {
       for (const auto &item : vertex_->out_edges) {
         const auto &[edge_type, to_vertex, edge] = item;
-        if (destination && to_vertex != destination->vertex_) continue;
+        if (nullptr != destination_id && to_vertex != *destination_id) continue;
         if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
           continue;
         out_edges.push_back(item);
@@ -574,29 +564,27 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std:
     delta = vertex_->delta;
   }
   ApplyDeltasForRead(
-      transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destination](const Delta &delta) {
+      transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, destination_id](const Delta &delta) {
         switch (delta.action) {
           case Delta::Action::ADD_OUT_EDGE: {
-            if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
+            if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
             if (!edge_types.empty() &&
                 std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
               break;
             // Add the edge because we don't see the removal.
-            std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
-                                                           delta.vertex_edge.edge};
+            Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
             auto it = std::find(out_edges.begin(), out_edges.end(), link);
             MG_ASSERT(it == out_edges.end(), "Invalid database state!");
             out_edges.push_back(link);
             break;
           }
           case Delta::Action::REMOVE_OUT_EDGE: {
-            if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
+            if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
             if (!edge_types.empty() &&
                 std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
               break;
             // Remove the label because we don't see the addition.
-            std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
-                                                           delta.vertex_edge.edge};
+            Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
             auto it = std::find(out_edges.begin(), out_edges.end(), link);
             MG_ASSERT(it != out_edges.end(), "Invalid database state!");
             std::swap(*it, *out_edges.rbegin());
@@ -622,13 +610,16 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std:
   if (!exists) return Error::NONEXISTENT_OBJECT;
   if (deleted) return Error::DELETED_OBJECT;
   std::vector<EdgeAccessor> ret;
+  if (out_edges.empty()) {
+    return ret;
+  }
   ret.reserve(out_edges.size());
+  const auto id = VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
   for (const auto &item : out_edges) {
     const auto &[edge_type, to_vertex, edge] = item;
-    ret.emplace_back(edge, edge_type, vertex_, to_vertex, transaction_, indices_, constraints_, config_,
-                     *vertex_validator_);
+    ret.emplace_back(edge, edge_type, id, to_vertex, transaction_, indices_, constraints_, config_);
   }
-  return std::move(ret);
+  return ret;
 }
 
 Result<size_t> VertexAccessor::InDegree(View view) const {
diff --git a/src/storage/v3/vertex_accessor.hpp b/src/storage/v3/vertex_accessor.hpp
index 33556e3f6..665e7af64 100644
--- a/src/storage/v3/vertex_accessor.hpp
+++ b/src/storage/v3/vertex_accessor.hpp
@@ -20,6 +20,7 @@
 #include "storage/v3/schema_validator.hpp"
 #include "storage/v3/transaction.hpp"
 #include "storage/v3/vertex.hpp"
+#include "storage/v3/vertex_id.hpp"
 #include "storage/v3/view.hpp"
 
 namespace memgraph::storage::v3 {
@@ -76,6 +77,8 @@ class VertexAccessor final {
 
   Result<PrimaryKey> PrimaryKey(View view) const;
 
+  Result<VertexId> Id(View view) const;
+
   /// Set a property value and return the old value or error.
   /// @throw std::bad_alloc
   ResultSchema<PropertyValue> SetPropertyAndValidate(PropertyId property, const PropertyValue &value);
@@ -94,13 +97,13 @@ class VertexAccessor final {
   /// @throw std::length_error if the resulting vector exceeds
   ///        std::vector::max_size().
   Result<std::vector<EdgeAccessor>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
-                                            const VertexAccessor *destination = nullptr) const;
+                                            const VertexId *destination_id = nullptr) const;
 
   /// @throw std::bad_alloc
   /// @throw std::length_error if the resulting vector exceeds
   ///        std::vector::max_size().
   Result<std::vector<EdgeAccessor>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
-                                             const VertexAccessor *destination = nullptr) const;
+                                             const VertexId *destination_id = nullptr) const;
 
   Result<size_t> InDegree(View view) const;
 
@@ -128,6 +131,8 @@ class VertexAccessor final {
   /// @throw std::bad_alloc
   Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
 
+  Result<void> CheckVertexExistence(View view) const;
+
   Vertex *vertex_;
   Transaction *transaction_;
   Indices *indices_;
diff --git a/src/storage/v3/vertex_id.hpp b/src/storage/v3/vertex_id.hpp
new file mode 100644
index 000000000..70dc85ec8
--- /dev/null
+++ b/src/storage/v3/vertex_id.hpp
@@ -0,0 +1,32 @@
+// 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 "storage/v3/id_types.hpp"
+#include "storage/v3/key_store.hpp"
+
+namespace memgraph::storage::v3 {
+
+// TODO(antaljanosbenjamin): It is possible to use a union of the current primary key and a vertex pointer: for local
+// vertices we can spare some space by eliminating copying the primary label and key, however it might introduce some
+// overhead for "remove vertices", because of the extra enum that is necessary for this optimization.
+struct VertexId {
+  VertexId(const LabelId primary_label, PrimaryKey primary_key)
+      : primary_label{primary_label}, primary_key{std::move(primary_key)} {}
+  LabelId primary_label;
+  PrimaryKey primary_key;
+};
+
+inline bool operator==(const VertexId &lhs, const VertexId &rhs) {
+  return lhs.primary_label == rhs.primary_label && lhs.primary_key == rhs.primary_key;
+}
+}  // namespace memgraph::storage::v3
diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt
index 6af33130e..af54378eb 100644
--- a/tests/unit/CMakeLists.txt
+++ b/tests/unit/CMakeLists.txt
@@ -340,6 +340,9 @@ target_link_libraries(${test_prefix}storage_v3_indices mg-storage-v3)
 add_unit_test(storage_v3_vertex_accessors.cpp)
 target_link_libraries(${test_prefix}storage_v3_vertex_accessors mg-storage-v3)
 
+add_unit_test(storage_v3_edge.cpp)
+target_link_libraries(${test_prefix}storage_v3_edge mg-storage-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)
diff --git a/tests/unit/storage_v3.cpp b/tests/unit/storage_v3.cpp
index 4163c96a2..41f152ca8 100644
--- a/tests/unit/storage_v3.cpp
+++ b/tests/unit/storage_v3.cpp
@@ -2195,7 +2195,9 @@ 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, NameToEdgeTypeId("edge")).HasValue());
+  ASSERT_TRUE(acc.CreateEdge(vertex.Id(View::NEW).GetValue(), vertex.Id(View::NEW).GetValue(), NameToEdgeTypeId("edge"),
+                             Gid::FromUint(1))
+                  .HasValue());
 
   // Check state after (OLD view).
   ASSERT_EQ(vertex.Labels(View::OLD).GetError(), Error::NONEXISTENT_OBJECT);
diff --git a/tests/unit/storage_v3_edge.cpp b/tests/unit/storage_v3_edge.cpp
new file mode 100644
index 000000000..4483ebcc6
--- /dev/null
+++ b/tests/unit/storage_v3_edge.cpp
@@ -0,0 +1,5274 @@
+// 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 <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include <limits>
+
+#include "storage/v3/config.hpp"
+#include "storage/v3/name_id_mapper.hpp"
+#include "storage/v3/property_value.hpp"
+#include "storage/v3/shard.hpp"
+
+namespace memgraph::storage::v3::tests {
+using testing::UnorderedElementsAre;
+
+class StorageEdgeTest : public ::testing::TestWithParam<bool> {
+ protected:
+  void SetUp() override {
+    ASSERT_TRUE(
+        store.CreateSchema(primary_label, {storage::v3::SchemaProperty{primary_property, common::SchemaType::INT}}));
+  }
+
+  [[nodiscard]] LabelId NameToLabelId(std::string_view label_name) {
+    return LabelId::FromUint(id_mapper.NameToId(label_name));
+  }
+
+  [[nodiscard]] PropertyId NameToPropertyId(std::string_view property_name) {
+    return PropertyId::FromUint(id_mapper.NameToId(property_name));
+  }
+
+  [[nodiscard]] EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) {
+    return EdgeTypeId::FromUint(id_mapper.NameToId(edge_type_name));
+  }
+
+  ResultSchema<VertexAccessor> CreateVertex(Shard::Accessor &acc, const PropertyValue &key) {
+    return acc.CreateVertexAndValidate(primary_label, {}, {{primary_property, key}});
+  }
+
+  NameIdMapper id_mapper;
+  static constexpr int64_t min_primary_key_value{0};
+  static constexpr int64_t max_primary_key_value{10000};
+  const LabelId primary_label{NameToLabelId("label")};
+  Shard store{primary_label,
+              {PropertyValue{min_primary_key_value}},
+              std::vector{PropertyValue{max_primary_key_value}},
+              Config{.items = {.properties_on_edges = GetParam()}}};
+  const PropertyId primary_property{NameToPropertyId("property")};
+};
+
+INSTANTIATE_TEST_CASE_P(EdgesWithProperties, StorageEdgeTest, ::testing::Values(true));
+INSTANTIATE_TEST_CASE_P(EdgesWithoutProperties, StorageEdgeTest, ::testing::Values(false));
+
+// NOLINTNEXTLINE(hicpp-special-member-functions)
+TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {
+  // Create vertices
+  const PropertyValue from_key{0};
+  const PropertyValue to_key{1};
+  const PropertyValue non_existing_key{2};
+  const auto et = NameToEdgeTypeId("et5");
+  const auto edge_id = Gid::FromUint(0U);
+  auto acc = store.Access();
+  const auto [from_id, to_id] =
+      std::invoke([this, &from_key, &to_key, &acc]() mutable -> std::pair<VertexId, VertexId> {
+        auto from_id = CreateVertex(acc, from_key)->Id(View::NEW).GetValue();
+        auto to_id = CreateVertex(acc, to_key)->Id(View::NEW).GetValue();
+        return std::make_pair(std::move(from_id), std::move(to_id));
+      });
+  const auto other_et = NameToEdgeTypeId("other");
+  const VertexId from_id_with_different_label{NameToLabelId("different_label"), from_id.primary_key};
+  const VertexId to_id_with_different_label{NameToLabelId("different_label"), to_id.primary_key};
+  const VertexId non_existing_id{primary_label, {non_existing_key}};
+
+  ASSERT_FALSE(acc.Commit().HasError());
+
+  // Create edge
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex({to_id.primary_key}, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from.has_value());
+    ASSERT_TRUE(vertex_to.has_value());
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+}
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeCreateFromLargerCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_from = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_to = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertices
+//   {
+//     auto acc = store.Access();
+//     auto vertex_to = acc.CreateVertex();
+//     auto vertex_from = acc.CreateVertex();
+//     gid_to = vertex_to.Gid();
+//     gid_from = vertex_from.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&from_id, &to_id, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), *vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeCreateFromSameCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid_vertex = vertex.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&*vertex, &*vertex, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex);
+//     ASSERT_EQ(edge.ToVertex(), *vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeCreateFromSmallerAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_from = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_to = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertices
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.CreateVertex();
+//     auto vertex_to = acc.CreateVertex();
+//     gid_from = vertex_from.Gid();
+//     gid_to = vertex_to.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&from_id, &to_id, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), *vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&from_id, &to_id, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), *vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// NOLINTNEXTLINE(hicpp-special-member-functions)
+TEST_P(StorageEdgeTest, EdgeCreateFromLargerAbort) {
+  // Create vertices
+  const PropertyValue from_key{0};
+  const PropertyValue to_key{1};
+  const PropertyValue non_existing_key{2};
+  auto acc = store.Access();
+  const auto [from_id, to_id] =
+      std::invoke([this, &from_key, &to_key, &acc]() mutable -> std::pair<VertexId, VertexId> {
+        auto from_id = CreateVertex(acc, from_key)->Id(View::NEW).GetValue();
+        auto to_id = CreateVertex(acc, to_key)->Id(View::NEW).GetValue();
+        return std::make_pair(std::move(from_id), std::move(to_id));
+      });
+
+  const auto et = NameToEdgeTypeId("et5");
+  const auto other_et = NameToEdgeTypeId("other");
+  const VertexId from_id_with_different_label{NameToLabelId("different_label"), from_id.primary_key};
+  const VertexId to_id_with_different_label{NameToLabelId("different_label"), to_id.primary_key};
+  const VertexId non_existing_id{primary_label, {non_existing_key}};
+
+  ASSERT_FALSE(acc.Commit().HasError());
+
+  // Create edge but abort
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex({to_id.primary_key}, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    const auto edge_id = Gid::FromUint(0U);
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    acc.Abort();
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Create edge
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    const auto edge_id = Gid::FromUint(1U);
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    const auto edge_id = Gid::FromUint(1U);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+}
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeCreateFromSameAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid_vertex = vertex.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&*vertex, &*vertex, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex);
+//     ASSERT_EQ(edge.ToVertex(), *vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&*vertex, &*vertex, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex);
+//     ASSERT_EQ(edge.ToVertex(), *vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// NOLINTNEXTLINE(hicpp-special-member-functions)
+TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerCommit) {
+  // Create vertex
+  const PropertyValue from_key{0};
+  const PropertyValue to_key{max_primary_key_value};
+  const PropertyValue non_existing_key{2};
+  auto acc = store.Access();
+  const auto from_id = std::invoke(
+      [this, &from_key, &acc]() mutable -> VertexId { return CreateVertex(acc, from_key)->Id(View::NEW).GetValue(); });
+  const VertexId to_id{primary_label, {to_key}};
+  const auto et = NameToEdgeTypeId("et5");
+  const auto edge_id = Gid::FromUint(1U);
+  const auto other_et = NameToEdgeTypeId("other");
+  const VertexId to_id_with_different_label{NameToLabelId("different_label"), to_id.primary_key};
+  const VertexId non_existing_id{primary_label, {non_existing_key}};
+
+  ASSERT_FALSE(acc.Commit().HasError());
+
+  // Create edge
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_FALSE(acc.FindVertex(to_id.primary_key, View::NEW).has_value());
+
+    const auto et = NameToEdgeTypeId("et5");
+    const auto edge_id = Gid::FromUint(1U);
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_from->OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+
+    // Check edges with filters
+
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et}, &to_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Delete edge
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+
+    const auto edge = vertex_from->OutEdges(View::NEW).GetValue()[0];
+
+    const auto res = acc.DeleteEdge(edge.FromVertex(), edge.ToVertex(), edge.Gid());
+    ASSERT_TRUE(res.HasValue());
+    ASSERT_TRUE(res.GetValue());
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    {
+      auto ret = vertex_from->OutEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &to_id)->size(), 1);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et}, &to_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+}
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeDeleteFromLargerCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_from = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_to = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertices
+//   {
+//     auto acc = store.Access();
+//     auto vertex_to = acc.CreateVertex();
+//     auto vertex_from = acc.CreateVertex();
+//     gid_from = vertex_from.Gid();
+//     gid_to = vertex_to.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&from_id, &to_id, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), *vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex_from->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeDeleteFromSameCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid_vertex = vertex.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&*vertex, &*vertex, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex);
+//     ASSERT_EQ(edge.ToVertex(), *vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeDeleteFromSmallerAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_from = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_to = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertices
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.CreateVertex();
+//     auto vertex_to = acc.CreateVertex();
+//     gid_from = vertex_from.Gid();
+//     gid_to = vertex_to.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&from_id, &to_id, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), *vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete the edge, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex_from->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete the edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex_from->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &to_id)->size(), 1);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD, {}, &from_id)->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// NOLINTNEXTLINE(hicpp-special-member-functions)
+TEST_P(StorageEdgeTest, EdgeDeleteFromLargerAbort) {
+  // Create vertex
+  const PropertyValue from_key{max_primary_key_value};
+  const PropertyValue to_key{0};
+  const PropertyValue non_existing_key{2};
+  auto acc = store.Access();
+  const auto to_id = std::invoke(
+      [this, &to_key, &acc]() mutable -> VertexId { return CreateVertex(acc, to_key)->Id(View::NEW).GetValue(); });
+  const VertexId from_id{primary_label, {from_key}};
+  const auto et = NameToEdgeTypeId("et5");
+  const auto edge_id = Gid::FromUint(1U);
+  const auto other_et = NameToEdgeTypeId("other");
+  const VertexId from_id_with_different_label{NameToLabelId("different_label"), from_id.primary_key};
+  const VertexId non_existing_id{primary_label, {non_existing_key}};
+
+  ASSERT_FALSE(acc.Commit().HasError());
+  // Create edge
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_to);
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+
+    // Check edges without filters
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Delete the edge, but abort the transaction
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_to);
+
+    const auto edge = vertex_to->InEdges(View::NEW).GetValue()[0];
+
+    auto res = acc.DeleteEdge(edge.FromVertex(), edge.ToVertex(), edge.Gid());
+    ASSERT_TRUE(res.HasValue());
+    ASSERT_TRUE(res.GetValue());
+
+    // Check edges without filters
+
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+
+    acc.Abort();
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_to);
+
+    // Check edges without filters
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to->InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Delete the edge
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_to);
+
+    const auto edge = vertex_to->InEdges(View::NEW).GetValue()[0];
+
+    auto res = acc.DeleteEdge(edge.FromVertex(), edge.ToVertex(), edge.Gid());
+    ASSERT_TRUE(res.HasValue());
+    ASSERT_TRUE(res.GetValue());
+
+    // Check edges without filters
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    // Check edges with filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et})->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et})->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id)->size(), 1);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &to_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {}, &non_existing_id)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::OLD, {et, other_et}, &from_id_with_different_label)->size(), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW, {et, other_et}, &from_id_with_different_label)->size(), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check whether the edge exists
+  {
+    auto acc = store.Access();
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_to);
+
+    // Check edges without filters
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+}
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, EdgeDeleteFromSameAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid_vertex = vertex.Gid();
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Create edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&*vertex, &*vertex, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), *vertex);
+//     ASSERT_EQ(edge.ToVertex(), *vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete the edge, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     {
+//       auto ret = vertex->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Delete the edge
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto res = acc.DeleteEdge(&edge);
+//     ASSERT_TRUE(res.HasValue());
+//     ASSERT_TRUE(res.GetValue());
+
+//     // Check edges without filters
+//     {
+//       auto ret = vertex->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex);
+//       ASSERT_EQ(e.ToVertex(), *vertex);
+//     }
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     auto other_et = acc.NameToEdgeType("other");
+
+//     // Check edges with filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->InEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et})->size(), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {et, other_et})->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {}, &*vertex)->size(), 1);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD, {other_et}, &*vertex)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check whether the edge exists
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid_vertex, View::NEW);
+//     ASSERT_TRUE(vertex);
+
+//     // Check edges without filters
+//     ASSERT_EQ(vertex->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+// }
+
+// NOLINTNEXTLINE(hicpp-special-member-functions)
+TEST_P(StorageEdgeTest, VertexDetachDeleteSingleCommit) {
+  // Create vertices
+  const PropertyValue from_key{0};
+  const PropertyValue to_key{1};
+  const PropertyValue non_existing_key{2};
+  const auto et = NameToEdgeTypeId("et5");
+  const auto edge_id = Gid::FromUint(0U);
+  auto acc = store.Access();
+  const VertexId from_id{primary_label, {from_key}};
+  const VertexId to_id{primary_label, {to_key}};
+  const VertexId non_existing_id{primary_label, {non_existing_key}};
+
+  // Create dataset
+  {
+    auto acc = store.Access();
+    auto vertex_from = CreateVertex(acc, from_key).GetValue();
+    auto vertex_to = CreateVertex(acc, to_key).GetValue();
+
+    auto res = acc.CreateEdge(from_id, to_id, et, edge_id);
+    ASSERT_TRUE(res.HasValue());
+    auto edge = res.GetValue();
+    ASSERT_EQ(edge.EdgeType(), et);
+    ASSERT_EQ(edge.Gid(), edge_id);
+    ASSERT_EQ(edge.FromVertex(), from_id);
+    ASSERT_EQ(edge.ToVertex(), to_id);
+
+    // Check edges
+    ASSERT_EQ(vertex_from.InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_from.InDegree(View::NEW), 0);
+    {
+      auto ret = vertex_from.OutEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from.OutDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    {
+      auto ret = vertex_to.InEdges(View::NEW);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to.InDegree(View::NEW), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to.OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to.OutDegree(View::NEW), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Detach delete vertex
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_TRUE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    // Delete must fail
+    {
+      auto ret = acc.DeleteVertex(&vertex_from.value());
+      ASSERT_TRUE(ret.HasError());
+      ASSERT_EQ(ret.GetError(), Error::VERTEX_HAS_EDGES);
+    }
+
+    // Detach delete vertex
+    {
+      auto ret = acc.DetachDeleteVertex(&vertex_from.value());
+      ASSERT_TRUE(ret.HasValue());
+      ASSERT_TRUE(*ret);
+    }
+
+    // Check edges
+    ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_from->InEdges(View::NEW).GetError(), Error::DELETED_OBJECT);
+    ASSERT_EQ(vertex_from->InDegree(View::NEW).GetError(), Error::DELETED_OBJECT);
+    {
+      auto ret = vertex_from->OutEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_from->OutEdges(View::NEW).GetError(), Error::DELETED_OBJECT);
+    ASSERT_EQ(vertex_from->OutDegree(View::NEW).GetError(), Error::DELETED_OBJECT);
+    {
+      auto ret = vertex_to->InEdges(View::OLD);
+      ASSERT_TRUE(ret.HasValue());
+      auto edges = ret.GetValue();
+      ASSERT_EQ(edges.size(), 1);
+      ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+      auto e = edges[0];
+      ASSERT_EQ(e.EdgeType(), et);
+      ASSERT_EQ(e.Gid(), edge_id);
+      ASSERT_EQ(e.FromVertex(), from_id);
+      ASSERT_EQ(e.ToVertex(), to_id);
+    }
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+    ASSERT_FALSE(acc.Commit().HasError());
+  }
+
+  // Check dataset
+  {
+    auto acc = store.Access();
+    auto vertex_from = acc.FindVertex(from_id.primary_key, View::NEW);
+    auto vertex_to = acc.FindVertex(to_id.primary_key, View::NEW);
+    ASSERT_FALSE(vertex_from);
+    ASSERT_TRUE(vertex_to);
+
+    // Check edges
+    ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+    ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+    ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+  }
+}
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex1 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_vertex2 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.CreateVertex();
+//     auto vertex2 = acc.CreateVertex();
+
+//     gid_vertex1 = vertex1.Gid();
+//     gid_vertex2 = vertex2.Gid();
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     auto res1 = acc.CreateEdge(&vertex1, &vertex2, et1);
+//     ASSERT_TRUE(res1.HasValue());
+//     auto edge1 = res1.GetValue();
+//     ASSERT_EQ(edge1.EdgeType(), et1);
+//     ASSERT_EQ(edge1.FromVertex(), vertex1);
+//     ASSERT_EQ(edge1.ToVertex(), vertex2);
+
+//     auto res2 = acc.CreateEdge(&vertex2, &vertex1, et2);
+//     ASSERT_TRUE(res2.HasValue());
+//     auto edge2 = res2.GetValue();
+//     ASSERT_EQ(edge2.EdgeType(), et2);
+//     ASSERT_EQ(edge2.FromVertex(), vertex2);
+//     ASSERT_EQ(edge2.ToVertex(), vertex1);
+
+//     auto res3 = acc.CreateEdge(&vertex1, &vertex1, et3);
+//     ASSERT_TRUE(res3.HasValue());
+//     auto edge3 = res3.GetValue();
+//     ASSERT_EQ(edge3.EdgeType(), et3);
+//     ASSERT_EQ(edge3.FromVertex(), vertex1);
+//     ASSERT_EQ(edge3.ToVertex(), vertex1);
+
+//     auto res4 = acc.CreateEdge(&vertex2, &vertex2, et4);
+//     ASSERT_TRUE(res4.HasValue());
+//     auto edge4 = res4.GetValue();
+//     ASSERT_EQ(edge4.EdgeType(), et4);
+//     ASSERT_EQ(edge4.FromVertex(), vertex2);
+//     ASSERT_EQ(edge4.ToVertex(), vertex2);
+
+//     // Check edges
+//     {
+//       auto ret = vertex1.InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1.InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex1.OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1.OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex2.InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2.InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2.OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2.OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Detach delete vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_TRUE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Delete must fail
+//     {
+//       auto ret = acc.DeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasError());
+//       ASSERT_EQ(ret.GetError(), memgraph::storage::Error::VERTEX_HAS_EDGES);
+//     }
+
+//     // Detach delete vertex
+//     {
+//       auto ret = acc.DetachDeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasValue());
+//       ASSERT_TRUE(*ret);
+//     }
+
+//     // Check edges
+//     {
+//       auto ret = vertex1->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->InEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->InDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex1->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->OutEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->OutDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_FALSE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Check edges
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, VertexDetachDeleteSingleAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_from = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_to = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.CreateVertex();
+//     auto vertex_to = acc.CreateVertex();
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     auto res = acc.CreateEdge(&vertex_from, &vertex_to, et);
+//     ASSERT_TRUE(res.HasValue());
+//     auto edge = res.GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex_from);
+//     ASSERT_EQ(edge.ToVertex(), vertex_to);
+
+//     gid_from = vertex_from.Gid();
+//     gid_to = vertex_to.Gid();
+
+//     // Check edges
+//     ASSERT_EQ(vertex_from.InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from.InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from.OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from.OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), vertex_from);
+//       ASSERT_EQ(e.ToVertex(), vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to.InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to.InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), vertex_from);
+//       ASSERT_EQ(e.ToVertex(), vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to.OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to.OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Detach delete vertex, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Delete must fail
+//     {
+//       auto ret = acc.DeleteVertex(&from_id);
+//       ASSERT_TRUE(ret.HasError());
+//       ASSERT_EQ(ret.GetError(), memgraph::storage::Error::VERTEX_HAS_EDGES);
+//     }
+
+//     // Detach delete vertex
+//     {
+//       auto ret = acc.DetachDeleteVertex(&from_id);
+//       ASSERT_TRUE(ret.HasValue());
+//       ASSERT_TRUE(*ret);
+//     }
+
+//     // Check edges
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex_from->InDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex_from->OutDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Check edges
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::NEW), 0);
+//     {
+//       auto ret = vertex_from->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     {
+//       auto ret = vertex_to->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Detach delete vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_TRUE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     auto et = acc.NameToEdgeType("et5");
+
+//     // Delete must fail
+//     {
+//       auto ret = acc.DeleteVertex(&from_id);
+//       ASSERT_TRUE(ret.HasError());
+//       ASSERT_EQ(ret.GetError(), memgraph::storage::Error::VERTEX_HAS_EDGES);
+//     }
+
+//     // Detach delete vertex
+//     {
+//       auto ret = acc.DetachDeleteVertex(&from_id);
+//       ASSERT_TRUE(ret.HasValue());
+//       ASSERT_TRUE(*ret);
+//     }
+
+//     // Check edges
+//     ASSERT_EQ(vertex_from->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_from->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_from->InEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex_from->InDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex_from->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_from->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_from->OutEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex_from->OutDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex_to->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex_to->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et);
+//       ASSERT_EQ(e.FromVertex(), *vertex_from);
+//       ASSERT_EQ(e.ToVertex(), *vertex_to);
+//     }
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex_from = acc.FindVertex(gid_from, View::NEW);
+//     auto vertex_to = acc.FindVertex(gid_to, View::NEW);
+//     ASSERT_FALSE(vertex_from);
+//     ASSERT_TRUE(vertex_to);
+
+//     // Check edges
+//     ASSERT_EQ(vertex_to->InEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->InEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->InDegree(View::NEW), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::OLD)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::OLD), 0);
+//     ASSERT_EQ(vertex_to->OutEdges(View::NEW)->size(), 0);
+//     ASSERT_EQ(*vertex_to->OutDegree(View::NEW), 0);
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST_P(StorageEdgeTest, VertexDetachDeleteMultipleAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = GetParam()}});
+//   memgraph::storage::Gid gid_vertex1 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   memgraph::storage::Gid gid_vertex2 = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.CreateVertex();
+//     auto vertex2 = acc.CreateVertex();
+
+//     gid_vertex1 = vertex1.Gid();
+//     gid_vertex2 = vertex2.Gid();
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     auto res1 = acc.CreateEdge(&vertex1, &vertex2, et1);
+//     ASSERT_TRUE(res1.HasValue());
+//     auto edge1 = res1.GetValue();
+//     ASSERT_EQ(edge1.EdgeType(), et1);
+//     ASSERT_EQ(edge1.FromVertex(), vertex1);
+//     ASSERT_EQ(edge1.ToVertex(), vertex2);
+
+//     auto res2 = acc.CreateEdge(&vertex2, &vertex1, et2);
+//     ASSERT_TRUE(res2.HasValue());
+//     auto edge2 = res2.GetValue();
+//     ASSERT_EQ(edge2.EdgeType(), et2);
+//     ASSERT_EQ(edge2.FromVertex(), vertex2);
+//     ASSERT_EQ(edge2.ToVertex(), vertex1);
+
+//     auto res3 = acc.CreateEdge(&vertex1, &vertex1, et3);
+//     ASSERT_TRUE(res3.HasValue());
+//     auto edge3 = res3.GetValue();
+//     ASSERT_EQ(edge3.EdgeType(), et3);
+//     ASSERT_EQ(edge3.FromVertex(), vertex1);
+//     ASSERT_EQ(edge3.ToVertex(), vertex1);
+
+//     auto res4 = acc.CreateEdge(&vertex2, &vertex2, et4);
+//     ASSERT_TRUE(res4.HasValue());
+//     auto edge4 = res4.GetValue();
+//     ASSERT_EQ(edge4.EdgeType(), et4);
+//     ASSERT_EQ(edge4.FromVertex(), vertex2);
+//     ASSERT_EQ(edge4.ToVertex(), vertex2);
+
+//     // Check edges
+//     {
+//       auto ret = vertex1.InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1.InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex1.OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1.OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex2.InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2.InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), vertex1);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2.OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2.OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), vertex2);
+//         ASSERT_EQ(e.ToVertex(), vertex2);
+//       }
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Detach delete vertex, but abort the transaction
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_TRUE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Delete must fail
+//     {
+//       auto ret = acc.DeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasError());
+//       ASSERT_EQ(ret.GetError(), memgraph::storage::Error::VERTEX_HAS_EDGES);
+//     }
+
+//     // Detach delete vertex
+//     {
+//       auto ret = acc.DetachDeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasValue());
+//       ASSERT_TRUE(*ret);
+//     }
+
+//     // Check edges
+//     {
+//       auto ret = vertex1->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->InEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->InDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex1->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->OutEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->OutDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+
+//     acc.Abort();
+//   }
+
+//   // Check dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_TRUE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Check edges
+//     {
+//       auto ret = vertex1->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex1->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex1->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex1->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->InDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->OutDegree(View::NEW), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Detach delete vertex
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_TRUE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et1 = acc.NameToEdgeType("et1");
+//     auto et2 = acc.NameToEdgeType("et2");
+//     auto et3 = acc.NameToEdgeType("et3");
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Delete must fail
+//     {
+//       auto ret = acc.DeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasError());
+//       ASSERT_EQ(ret.GetError(), memgraph::storage::Error::VERTEX_HAS_EDGES);
+//     }
+
+//     // Detach delete vertex
+//     {
+//       auto ret = acc.DetachDeleteVertex(&*vertex1);
+//       ASSERT_TRUE(ret.HasValue());
+//       ASSERT_TRUE(*ret);
+//     }
+
+//     // Check edges
+//     {
+//       auto ret = vertex1->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->InEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->InDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex1->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex1->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et3);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//     }
+//     ASSERT_EQ(vertex1->OutEdges(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     ASSERT_EQ(vertex1->OutDegree(View::NEW).GetError(), memgraph::storage::Error::DELETED_OBJECT);
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->InDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et1);
+//         ASSERT_EQ(e.FromVertex(), *vertex1);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       std::sort(edges.begin(), edges.end(), [](const auto &a, const auto &b) { return a.EdgeType() < b.EdgeType();
+//       }); ASSERT_EQ(edges.size(), 2); ASSERT_EQ(*vertex2->OutDegree(View::OLD), 2);
+//       {
+//         auto e = edges[0];
+//         ASSERT_EQ(e.EdgeType(), et2);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex1);
+//       }
+//       {
+//         auto e = edges[1];
+//         ASSERT_EQ(e.EdgeType(), et4);
+//         ASSERT_EQ(e.FromVertex(), *vertex2);
+//         ASSERT_EQ(e.ToVertex(), *vertex2);
+//       }
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check dataset
+//   {
+//     auto acc = store.Access();
+//     auto vertex1 = acc.FindVertex(gid_vertex1, View::NEW);
+//     auto vertex2 = acc.FindVertex(gid_vertex2, View::NEW);
+//     ASSERT_FALSE(vertex1);
+//     ASSERT_TRUE(vertex2);
+
+//     auto et4 = acc.NameToEdgeType("et4");
+
+//     // Check edges
+//     {
+//       auto ret = vertex2->InEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->InEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->InDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::OLD);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::OLD), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//     {
+//       auto ret = vertex2->OutEdges(View::NEW);
+//       ASSERT_TRUE(ret.HasValue());
+//       auto edges = ret.GetValue();
+//       ASSERT_EQ(edges.size(), 1);
+//       ASSERT_EQ(*vertex2->OutDegree(View::NEW), 1);
+//       auto e = edges[0];
+//       ASSERT_EQ(e.EdgeType(), et4);
+//       ASSERT_EQ(e.FromVertex(), *vertex2);
+//       ASSERT_EQ(e.ToVertex(), *vertex2);
+//     }
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST(StorageWithProperties, EdgePropertyCommit) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = true}});
+//   memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("temporary"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_TRUE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "temporary");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "temporary");
+//     }
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("nandare"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue());
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue());
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_TRUE(old_value->IsNull());
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST(StorageWithProperties, EdgePropertyAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = true}});
+//   memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+
+//   // Create the vertex.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Set property 5 to "nandare", but abort the transaction.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("temporary"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_TRUE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "temporary");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "temporary");
+//     }
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("nandare"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     acc.Abort();
+//   }
+
+//   // Check that property 5 is null.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+
+//   // Set property 5 to "nandare".
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("temporary"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_TRUE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "temporary");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "temporary");
+//     }
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue("nandare"));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check that property 5 is "nandare".
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+
+//   // Set property 5 to null, but abort the transaction.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue());
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     acc.Abort();
+//   }
+
+//   // Check that property 5 is "nandare".
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+
+//   // Set property 5 to null.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::NEW)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     {
+//       auto old_value = edge.SetProperty(property, memgraph::storage::PropertyValue());
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_FALSE(old_value->IsNull());
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property, View::OLD)->ValueString(), "nandare");
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property].ValueString(), "nandare");
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   // Check that property 5 is null.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST(StorageWithProperties, EdgePropertySerializationError) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = true}});
+//   memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+
+//   auto acc1 = store.Access();
+//   auto acc2 = store.Access();
+
+//   // Set property 1 to 123 in accessor 1.
+//   {
+//     auto vertex = acc1.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property1 = acc1.NameToProperty("property1");
+//     auto property2 = acc1.NameToProperty("property2");
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto old_value = edge.SetProperty(property1, memgraph::storage::PropertyValue(123));
+//       ASSERT_TRUE(old_value.HasValue());
+//       ASSERT_TRUE(old_value->IsNull());
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::OLD)->IsNull());
+//     ASSERT_EQ(edge.GetProperty(property1, View::NEW)->ValueInt(), 123);
+//     ASSERT_TRUE(edge.GetProperty(property2, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property1].ValueInt(), 123);
+//     }
+//   }
+
+//   // Set property 2 to "nandare" in accessor 2.
+//   {
+//     auto vertex = acc2.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property1 = acc2.NameToProperty("property1");
+//     auto property2 = acc2.NameToProperty("property2");
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto res = edge.SetProperty(property2, memgraph::storage::PropertyValue("nandare"));
+//       ASSERT_TRUE(res.HasError());
+//       ASSERT_EQ(res.GetError(), memgraph::storage::Error::SERIALIZATION_ERROR);
+//     }
+//   }
+
+//   // Finalize both accessors.
+//   ASSERT_FALSE(acc1.Commit().HasError());
+//   acc2.Abort();
+
+//   // Check which properties exist.
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property1 = acc.NameToProperty("property1");
+//     auto property2 = acc.NameToProperty("property2");
+
+//     ASSERT_EQ(edge.GetProperty(property1, View::OLD)->ValueInt(), 123);
+//     ASSERT_TRUE(edge.GetProperty(property2, View::OLD)->IsNull());
+//     {
+//       auto properties = edge.Properties(View::OLD).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property1].ValueInt(), 123);
+//     }
+
+//     ASSERT_EQ(edge.GetProperty(property1, View::NEW)->ValueInt(), 123);
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     {
+//       auto properties = edge.Properties(View::NEW).GetValue();
+//       ASSERT_EQ(properties.size(), 1);
+//       ASSERT_EQ(properties[property1].ValueInt(), 123);
+//     }
+
+//     acc.Abort();
+//   }
+// }
+
+// TEST(StorageWithProperties, EdgePropertyClear) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = true}});
+//   memgraph::storage::Gid gid;
+//   auto property1 = store.NameToProperty("property1");
+//   auto property2 = store.NameToProperty("property2");
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+
+//     auto old_value = edge.SetProperty(property1, memgraph::storage::PropertyValue("value"));
+//     ASSERT_TRUE(old_value.HasValue());
+//     ASSERT_TRUE(old_value->IsNull());
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     ASSERT_EQ(edge.GetProperty(property1, View::OLD)->ValueString(), "value");
+//     ASSERT_TRUE(edge.GetProperty(property2, View::OLD)->IsNull());
+//     ASSERT_THAT(edge.Properties(View::OLD).GetValue(),
+//                 UnorderedElementsAre(std::pair(property1, memgraph::storage::PropertyValue("value"))));
+
+//     {
+//       auto old_values = edge.ClearProperties();
+//       ASSERT_TRUE(old_values.HasValue());
+//       ASSERT_FALSE(old_values->empty());
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW).GetValue().size(), 0);
+
+//     {
+//       auto old_values = edge.ClearProperties();
+//       ASSERT_TRUE(old_values.HasValue());
+//       ASSERT_TRUE(old_values->empty());
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW).GetValue().size(), 0);
+
+//     acc.Abort();
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto old_value = edge.SetProperty(property2, memgraph::storage::PropertyValue(42));
+//     ASSERT_TRUE(old_value.HasValue());
+//     ASSERT_TRUE(old_value->IsNull());
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     ASSERT_EQ(edge.GetProperty(property1, View::OLD)->ValueString(), "value");
+//     ASSERT_EQ(edge.GetProperty(property2, View::OLD)->ValueInt(), 42);
+//     ASSERT_THAT(edge.Properties(View::OLD).GetValue(),
+//                 UnorderedElementsAre(std::pair(property1, memgraph::storage::PropertyValue("value")),
+//                                      std::pair(property2, memgraph::storage::PropertyValue(42))));
+
+//     {
+//       auto old_values = edge.ClearProperties();
+//       ASSERT_TRUE(old_values.HasValue());
+//       ASSERT_FALSE(old_values->empty());
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW).GetValue().size(), 0);
+
+//     {
+//       auto old_values = edge.ClearProperties();
+//       ASSERT_TRUE(old_values.HasValue());
+//       ASSERT_TRUE(old_values->empty());
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW).GetValue().size(), 0);
+
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     ASSERT_TRUE(edge.GetProperty(property1, View::NEW)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(property2, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW).GetValue().size(), 0);
+
+//     acc.Abort();
+//   }
+// }
+
+// // NOLINTNEXTLINE(hicpp-special-member-functions)
+// TEST(StorageWithoutProperties, EdgePropertyAbort) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = false}});
+//   memgraph::storage::Gid gid = memgraph::storage::Gid::FromUint(std::numeric_limits<uint64_t>::max());
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto res = edge.SetProperty(property, memgraph::storage::PropertyValue("temporary"));
+//       ASSERT_TRUE(res.HasError());
+//       ASSERT_EQ(res.GetError(), memgraph::storage::Error::PROPERTIES_DISABLED);
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     {
+//       auto res = edge.SetProperty(property, memgraph::storage::PropertyValue("nandare"));
+//       ASSERT_TRUE(res.HasError());
+//       ASSERT_EQ(res.GetError(), memgraph::storage::Error::PROPERTIES_DISABLED);
+//     }
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     acc.Abort();
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     auto property = acc.NameToProperty("property5");
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::OLD)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::OLD)->size(), 0);
+
+//     ASSERT_TRUE(edge.GetProperty(property, View::NEW)->IsNull());
+//     ASSERT_EQ(edge.Properties(View::NEW)->size(), 0);
+
+//     auto other_property = acc.NameToProperty("other");
+
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::OLD)->IsNull());
+//     ASSERT_TRUE(edge.GetProperty(other_property, View::NEW)->IsNull());
+
+//     acc.Abort();
+//   }
+// }
+
+// TEST(StorageWithoutProperties, EdgePropertyClear) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = false}});
+//   memgraph::storage::Gid gid;
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.CreateVertex();
+//     gid = vertex.Gid();
+//     auto et = acc.NameToEdgeType("et5");
+//     auto edge = acc.CreateEdge(&vertex, &vertex, et).GetValue();
+//     ASSERT_EQ(edge.EdgeType(), et);
+//     ASSERT_EQ(edge.FromVertex(), vertex);
+//     ASSERT_EQ(edge.ToVertex(), vertex);
+//     ASSERT_FALSE(acc.Commit().HasError());
+//   }
+//   {
+//     auto acc = store.Access();
+//     auto vertex = acc.FindVertex(gid, View::OLD);
+//     ASSERT_TRUE(vertex);
+//     auto edge = vertex->OutEdges(View::NEW).GetValue()[0];
+
+//     ASSERT_EQ(edge.ClearProperties().GetError(), memgraph::storage::Error::PROPERTIES_DISABLED);
+
+//     acc.Abort();
+//   }
+// }
+
+// TEST(StorageWithProperties, EdgeNonexistentPropertyAPI) {
+//   memgraph::storage::Storage store({.items = {.properties_on_edges = true}});
+
+//   auto property = store.NameToProperty("property");
+
+//   auto acc = store.Access();
+//   auto vertex = acc.CreateVertex();
+//   auto edge = acc.CreateEdge(&vertex, &vertex, acc.NameToEdgeType("edge"));
+//   ASSERT_TRUE(edge.HasValue());
+
+//   // Check state before (OLD view).
+//   ASSERT_EQ(edge->Properties(View::OLD).GetError(), memgraph::storage::Error::NONEXISTENT_OBJECT);
+//   ASSERT_EQ(edge->GetProperty(property, View::OLD).GetError(), memgraph::storage::Error::NONEXISTENT_OBJECT);
+
+//   // Check state before (NEW view).
+//   ASSERT_EQ(edge->Properties(View::NEW)->size(), 0);
+//   ASSERT_EQ(*edge->GetProperty(property, View::NEW), memgraph::storage::PropertyValue());
+
+//   // Modify edge.
+//   ASSERT_TRUE(edge->SetProperty(property, memgraph::storage::PropertyValue("value"))->IsNull());
+
+//   // Check state after (OLD view).
+//   ASSERT_EQ(edge->Properties(View::OLD).GetError(), memgraph::storage::Error::NONEXISTENT_OBJECT);
+//   ASSERT_EQ(edge->GetProperty(property, View::OLD).GetError(), memgraph::storage::Error::NONEXISTENT_OBJECT);
+
+//   // Check state after (NEW view).
+//   ASSERT_EQ(edge->Properties(View::NEW)->size(), 1);
+//   ASSERT_EQ(*edge->GetProperty(property, View::NEW), memgraph::storage::PropertyValue("value"));
+
+//   ASSERT_FALSE(acc.Commit().HasError());
+// }
+}  // namespace memgraph::storage::v3::tests