diff --git a/src/storage/v2/disk/rocksdb_storage.hpp b/src/storage/v2/disk/rocksdb_storage.hpp
index f58de3763..f3e05a44c 100644
--- a/src/storage/v2/disk/rocksdb_storage.hpp
+++ b/src/storage/v2/disk/rocksdb_storage.hpp
@@ -50,18 +50,6 @@ struct RocksDBStorage {
   rocksdb::ColumnFamilyHandle *vertex_chandle = nullptr;
   rocksdb::ColumnFamilyHandle *edge_chandle = nullptr;
   rocksdb::ColumnFamilyHandle *default_chandle = nullptr;
-
-  uint64_t ApproximateVertexCount() const {
-    uint64_t estimate_num_keys = 0;
-    db_->GetIntProperty(vertex_chandle, "rocksdb.estimate-num-keys", &estimate_num_keys);
-    return estimate_num_keys;
-  }
-
-  uint64_t ApproximateEdgeCount() const {
-    uint64_t estimate_num_keys = 0;
-    db_->GetIntProperty(edge_chandle, "rocksdb.estimate-num-keys", &estimate_num_keys);
-    return estimate_num_keys;
-  }
 };
 
 /// RocksDB comparator that compares keys with timestamps.
diff --git a/src/storage/v2/disk/storage.cpp b/src/storage/v2/disk/storage.cpp
index a6910fa62..e5d9d4517 100644
--- a/src/storage/v2/disk/storage.cpp
+++ b/src/storage/v2/disk/storage.cpp
@@ -9,6 +9,7 @@
 // by the Apache License, Version 2.0, included in the file
 // licenses/APL.txt.
 
+#include <atomic>
 #include <limits>
 #include <optional>
 #include <stdexcept>
@@ -30,6 +31,7 @@
 #include "storage/v2/disk/rocksdb_storage.hpp"
 #include "storage/v2/disk/storage.hpp"
 #include "storage/v2/disk/unique_constraints.hpp"
+#include "storage/v2/edge.hpp"
 #include "storage/v2/id_types.hpp"
 #include "storage/v2/mvcc.hpp"
 #include "storage/v2/property_store.hpp"
@@ -65,6 +67,8 @@ constexpr const char *vertexHandle = "vertex";
 constexpr const char *edgeHandle = "edge";
 constexpr const char *defaultHandle = "default";
 constexpr const char *lastTransactionStartTimeStamp = "last_transaction_start_timestamp";
+constexpr const char *vertex_count_descr = "vertex_count";
+constexpr const char *edge_count_descr = "edge_count";
 constexpr const char *label_index_str = "label_index";
 constexpr const char *label_property_index_str = "label_property_index";
 constexpr const char *existence_constraints_str = "existence_constraints";
@@ -194,6 +198,18 @@ void DiskStorage::LoadTimestampIfExists() {
   }
 }
 
+void DiskStorage::LoadVertexAndEdgeCountIfExists() {
+  if (!utils::DirExists(config_.disk.durability_directory)) {
+    return;
+  }
+  if (auto vertex_count = durability_kvstore_->Get(vertex_count_descr); vertex_count.has_value()) {
+    vertex_count_ = std::stoull(vertex_count.value());
+  }
+  if (auto edge_count = durability_kvstore_->Get(edge_count_descr); edge_count.has_value()) {
+    edge_count_ = std::stoull(edge_count.value());
+  }
+}
+
 void DiskStorage::LoadIndexInfoIfExists() const {
   if (utils::DirExists(config_.disk.durability_directory)) {
     LoadLabelIndexInfoIfExists();
@@ -246,6 +262,7 @@ DiskStorage::DiskStorage(Config config)
       kvstore_(std::make_unique<RocksDBStorage>()),
       durability_kvstore_(std::make_unique<kvstore::KVStore>(config.disk.durability_directory)) {
   LoadTimestampIfExists();
+  LoadVertexAndEdgeCountIfExists();
   LoadIndexInfoIfExists();
   LoadConstraintsInfoIfExists();
   kvstore_->options_.create_if_missing = true;
@@ -279,6 +296,8 @@ DiskStorage::DiskStorage(Config config)
 
 DiskStorage::~DiskStorage() {
   durability_kvstore_->Put(lastTransactionStartTimeStamp, std::to_string(timestamp_));
+  durability_kvstore_->Put(vertex_count_descr, std::to_string(vertex_count_.load(std::memory_order_acquire)));
+  durability_kvstore_->Put(edge_count_descr, std::to_string(edge_count_.load(std::memory_order_acquire)));
   logging::AssertRocksDBStatus(kvstore_->db_->DestroyColumnFamilyHandle(kvstore_->vertex_chandle));
   logging::AssertRocksDBStatus(kvstore_->db_->DestroyColumnFamilyHandle(kvstore_->edge_chandle));
   if (kvstore_->default_chandle) {
@@ -324,8 +343,8 @@ std::optional<storage::VertexAccessor> DiskStorage::DiskAccessor::LoadVertexToMa
   }
   std::vector<LabelId> labels_id{utils::DeserializeLabelsFromMainDiskStorage(key)};
   PropertyStore properties{utils::DeserializePropertiesFromMainDiskStorage(value)};
-  return CreateVertex(main_storage_accessor, gid, std::move(labels_id), std::move(properties),
-                      CreateDeleteDeserializedObjectDelta(&transaction_, key, ts));
+  return CreateVertexFromDisk(main_storage_accessor, gid, std::move(labels_id), std::move(properties),
+                              CreateDeleteDeserializedObjectDelta(&transaction_, key, ts));
 }
 
 std::optional<storage::VertexAccessor> DiskStorage::DiskAccessor::LoadVertexToLabelIndexCache(
@@ -335,11 +354,11 @@ std::optional<storage::VertexAccessor> DiskStorage::DiskAccessor::LoadVertexToLa
   if (VertexExistsInCache(index_accessor, gid)) {
     return std::nullopt;
   }
-
+  /// TODO: (andi) I think this is now changed with one PR
   std::vector<LabelId> labels_id{utils::DeserializeLabelsFromLabelIndexStorage(value)};
   labels_id.push_back(indexing_label);
   PropertyStore properties{utils::DeserializePropertiesFromLabelIndexStorage(value)};
-  return CreateVertex(index_accessor, gid, std::move(labels_id), std::move(properties), index_delta);
+  return CreateVertexFromDisk(index_accessor, gid, std::move(labels_id), std::move(properties), index_delta);
 }
 
 std::optional<storage::VertexAccessor> DiskStorage::DiskAccessor::LoadVertexToLabelPropertyIndexCache(
@@ -349,11 +368,11 @@ std::optional<storage::VertexAccessor> DiskStorage::DiskAccessor::LoadVertexToLa
   if (VertexExistsInCache(index_accessor, gid)) {
     return std::nullopt;
   }
-
+  /// TODO: (andi) I think this is now changed with one PR
   std::vector<LabelId> labels_id{utils::DeserializeLabelsFromLabelPropertyIndexStorage(value)};
   labels_id.push_back(indexing_label);
   PropertyStore properties{utils::DeserializePropertiesFromLabelPropertyIndexStorage(value)};
-  return CreateVertex(index_accessor, gid, std::move(labels_id), std::move(properties), index_delta);
+  return CreateVertexFromDisk(index_accessor, gid, std::move(labels_id), std::move(properties), index_delta);
 }
 
 std::optional<EdgeAccessor> DiskStorage::DiskAccessor::DeserializeEdge(const rocksdb::Slice &key,
@@ -384,8 +403,8 @@ std::optional<EdgeAccessor> DiskStorage::DiskAccessor::DeserializeEdge(const roc
     throw utils::BasicException("Non-existing vertices found during edge deserialization");
   }
   const auto edge_type_id = storage::EdgeTypeId::FromUint(std::stoull(edge_parts[3]));
-  auto maybe_edge =
-      CreateEdge(&*from_acc, &*to_acc, edge_type_id, edge_gid, value.ToStringView(), key.ToString(), ts.ToString());
+  auto maybe_edge = CreateEdgeFromDisk(&*from_acc, &*to_acc, edge_type_id, edge_gid, value.ToStringView(),
+                                       key.ToString(), ts.ToString());
   MG_ASSERT(maybe_edge.HasValue());
 
   return *maybe_edge;
@@ -700,7 +719,7 @@ void DiskStorage::DiskAccessor::LoadVerticesFromDiskLabelPropertyIndexForInterva
 
 uint64_t DiskStorage::DiskAccessor::ApproximateVertexCount() const {
   auto *disk_storage = static_cast<DiskStorage *>(storage_);
-  return disk_storage->kvstore_->ApproximateVertexCount();
+  return disk_storage->vertex_count_.load(std::memory_order_acquire);
 }
 
 bool DiskStorage::PersistLabelIndexCreation(LabelId label) const {
@@ -795,19 +814,21 @@ uint64_t DiskStorage::GetDiskSpaceUsage() const {
 }
 
 StorageInfo DiskStorage::GetInfo() const {
-  auto vertex_count = kvstore_->ApproximateVertexCount();
-  auto edge_count = kvstore_->ApproximateEdgeCount();
+  uint64_t edge_count = edge_count_.load(std::memory_order_acquire);
+  uint64_t vertex_count = vertex_count_.load(std::memory_order_acquire);
   double average_degree = 0.0;
   if (vertex_count) {
     // NOLINTNEXTLINE(bugprone-narrowing-conversions, cppcoreguidelines-narrowing-conversions)
-    average_degree = 2.0 * static_cast<double>(edge_count) / vertex_count;
+    average_degree = 2.0 * edge_count / static_cast<double>(vertex_count);
   }
 
   return {vertex_count, edge_count, average_degree, utils::GetMemoryUsage(), GetDiskSpaceUsage()};
 }
 
 VertexAccessor DiskStorage::DiskAccessor::CreateVertex() {
-  auto gid = storage_->vertex_id_.fetch_add(1, std::memory_order_acq_rel);
+  OOMExceptionEnabler oom_exception;
+  auto *disk_storage = static_cast<DiskStorage *>(storage_);
+  auto gid = disk_storage->vertex_id_.fetch_add(1, std::memory_order_acq_rel);
   auto acc = vertices_.access();
 
   auto *delta = CreateDeleteObjectDelta(&transaction_);
@@ -819,12 +840,13 @@ VertexAccessor DiskStorage::DiskAccessor::CreateVertex() {
     delta->prev.Set(&*it);
   }
 
+  disk_storage->vertex_count_.fetch_add(1, std::memory_order_acq_rel);
   return {&*it, &transaction_, &storage_->indices_, &storage_->constraints_, config_};
 }
 
-VertexAccessor DiskStorage::DiskAccessor::CreateVertex(utils::SkipList<Vertex>::Accessor &accessor, storage::Gid gid,
-                                                       std::vector<LabelId> &&label_ids, PropertyStore &&properties,
-                                                       Delta *delta) {
+VertexAccessor DiskStorage::DiskAccessor::CreateVertexFromDisk(utils::SkipList<Vertex>::Accessor &accessor,
+                                                               storage::Gid gid, std::vector<LabelId> &&label_ids,
+                                                               PropertyStore &&properties, Delta *delta) {
   OOMExceptionEnabler oom_exception;
   auto *disk_storage = static_cast<DiskStorage *>(storage_);
   disk_storage->vertex_id_.store(std::max(disk_storage->vertex_id_.load(std::memory_order_acquire), gid.AsUint() + 1),
@@ -886,6 +908,9 @@ Result<std::optional<VertexAccessor>> DiskStorage::DiskAccessor::DeleteVertex(Ve
   vertices_to_delete_.emplace_back(utils::SerializeIdType(vertex_ptr->gid), utils::SerializeVertex(*vertex_ptr));
   transaction_.manyDeltasCache.Invalidate(vertex_ptr);
 
+  auto *disk_storage = static_cast<DiskStorage *>(storage_);
+  disk_storage->vertex_count_.fetch_sub(1, std::memory_order_acq_rel);
+
   return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_,
                                             config_, true);
 }
@@ -935,6 +960,8 @@ DiskStorage::DiskAccessor::DetachDeleteVertex(VertexAccessor *vertex) {
   CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag());
   vertex_ptr->deleted = true;
   vertices_to_delete_.emplace_back(utils::SerializeIdType(vertex_ptr->gid), utils::SerializeVertex(*vertex_ptr));
+  auto *disk_storage = static_cast<DiskStorage *>(storage_);
+  disk_storage->vertex_count_.fetch_sub(1, std::memory_order_acq_rel);
   transaction_.manyDeltasCache.Invalidate(vertex_ptr);
 
   return std::make_optional<ReturnType>(
@@ -995,11 +1022,11 @@ void DiskStorage::DiskAccessor::PrefetchOutEdges(const VertexAccessor &vertex_ac
   PrefetchEdges(vertex_acc, EdgeDirection::OUT);
 }
 
-Result<EdgeAccessor> DiskStorage::DiskAccessor::CreateEdge(const VertexAccessor *from, const VertexAccessor *to,
-                                                           EdgeTypeId edge_type, storage::Gid gid,
-                                                           const std::string_view properties,
-                                                           const std::string &old_disk_key,
-                                                           const std::string &read_ts) {
+Result<EdgeAccessor> DiskStorage::DiskAccessor::CreateEdgeFromDisk(const VertexAccessor *from, const VertexAccessor *to,
+                                                                   EdgeTypeId edge_type, storage::Gid gid,
+                                                                   const std::string_view properties,
+                                                                   const std::string &old_disk_key,
+                                                                   const std::string &read_ts) {
   OOMExceptionEnabler oom_exception;
   auto *from_vertex = from->vertex_;
   auto *to_vertex = to->vertex_;
@@ -1030,8 +1057,6 @@ Result<EdgeAccessor> DiskStorage::DiskAccessor::CreateEdge(const VertexAccessor
   transaction_.manyDeltasCache.Invalidate(from_vertex, edge_type, EdgeDirection::OUT);
   transaction_.manyDeltasCache.Invalidate(to_vertex, edge_type, EdgeDirection::IN);
 
-  storage_->edge_count_.fetch_add(1, std::memory_order_acq_rel);
-
   return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_,
                       &storage_->constraints_, config_);
 }
@@ -1065,7 +1090,6 @@ Result<EdgeAccessor> DiskStorage::DiskAccessor::CreateEdge(VertexAccessor *from,
   transaction_.manyDeltasCache.Invalidate(from_vertex, edge_type, EdgeDirection::OUT);
   transaction_.manyDeltasCache.Invalidate(to_vertex, edge_type, EdgeDirection::IN);
 
-  // Increment edge count.
   storage_->edge_count_.fetch_add(1, std::memory_order_acq_rel);
 
   return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_,
@@ -1079,9 +1103,7 @@ Result<std::optional<EdgeAccessor>> DiskStorage::DiskAccessor::DeleteEdge(EdgeAc
   const auto edge_ref = edge->edge_;
   const auto edge_type = edge->edge_type_;
 
-  if (config_.properties_on_edges) {
-    if (edge_ref.ptr->deleted) return std::optional<EdgeAccessor>{};
-  }
+  if (config_.properties_on_edges && edge_ref.ptr->deleted) return std::optional<EdgeAccessor>{};
 
   auto *from_vertex = edge->from_vertex_;
   auto *to_vertex = edge->to_vertex_;
@@ -1133,8 +1155,7 @@ Result<std::optional<EdgeAccessor>> DiskStorage::DiskAccessor::DeleteEdge(EdgeAc
   transaction_.manyDeltasCache.Invalidate(from_vertex, edge_type, EdgeDirection::OUT);
   transaction_.manyDeltasCache.Invalidate(to_vertex, edge_type, EdgeDirection::IN);
 
-  // Decrement edge count.
-  storage_->edge_count_.fetch_add(-1, std::memory_order_acq_rel);
+  storage_->edge_count_.fetch_sub(1, std::memory_order_acq_rel);
 
   return std::make_optional<EdgeAccessor>(edge_ref, edge_type, from_vertex, to_vertex, &transaction_,
                                           &storage_->indices_, &storage_->constraints_, config_, true);
@@ -1516,6 +1537,60 @@ std::vector<std::pair<std::string, std::string>> DiskStorage::SerializeVerticesF
   return vertices_to_be_indexed;
 }
 
+void DiskStorage::DiskAccessor::UpdateObjectsCountOnAbort() {
+  auto *disk_storage = static_cast<DiskStorage *>(storage_);
+  uint64_t transaction_id = transaction_.transaction_id.load(std::memory_order_acquire);
+
+  for (const auto &delta : transaction_.deltas) {
+    auto prev = delta.prev.Get();
+    switch (prev.type) {
+      case PreviousPtr::Type::VERTEX: {
+        auto *vertex = prev.vertex;
+        Delta *current = vertex->delta;
+        while (current != nullptr && current->timestamp->load(std::memory_order_acquire) == transaction_id) {
+          switch (current->action) {
+            case Delta::Action::DELETE_DESERIALIZED_OBJECT:
+            case Delta::Action::DELETE_OBJECT: {
+              disk_storage->vertex_count_.fetch_sub(1, std::memory_order_acq_rel);
+              break;
+            }
+            case Delta::Action::RECREATE_OBJECT: {
+              disk_storage->vertex_count_.fetch_add(1, std::memory_order_acq_rel);
+              break;
+            }
+            case Delta::Action::REMOVE_IN_EDGE:
+            case Delta::Action::REMOVE_LABEL:
+            case Delta::Action::ADD_LABEL:
+            case Delta::Action::SET_PROPERTY:
+            case Delta::Action::ADD_IN_EDGE: {
+              break;
+            }
+            case Delta::Action::ADD_OUT_EDGE: {
+              storage_->edge_count_.fetch_add(1, std::memory_order_acq_rel);
+              break;
+            }
+            case Delta::Action::REMOVE_OUT_EDGE: {
+              storage_->edge_count_.fetch_sub(1, std::memory_order_acq_rel);
+              break;
+            }
+          }
+          current = current->next.load(std::memory_order_acquire);
+        }
+        vertex->delta = current;
+        if (current != nullptr) {
+          current->prev.Set(vertex);
+        }
+
+        break;
+      }
+      case PreviousPtr::Type::EDGE:
+      case PreviousPtr::Type::DELTA:
+      case PreviousPtr::Type::NULLPTR:
+        break;
+    }
+  }
+}
+
 /// TODO: what to do with all that?
 void DiskStorage::DiskAccessor::Abort() {
   MG_ASSERT(is_transaction_active_, "The transaction is already terminated!");
@@ -1527,8 +1602,8 @@ void DiskStorage::DiskAccessor::Abort() {
   disk_transaction_->ClearSnapshot();
   delete disk_transaction_;
   disk_transaction_ = nullptr;
-
   is_transaction_active_ = false;
+  UpdateObjectsCountOnAbort();
 }
 
 void DiskStorage::DiskAccessor::FinalizeTransaction() {
diff --git a/src/storage/v2/disk/storage.hpp b/src/storage/v2/disk/storage.hpp
index 601761ca4..b01849b12 100644
--- a/src/storage/v2/disk/storage.hpp
+++ b/src/storage/v2/disk/storage.hpp
@@ -193,6 +193,8 @@ class DiskStorage final : public Storage {
     utils::BasicResult<StorageDataManipulationError, void> Commit(
         std::optional<uint64_t> desired_commit_timestamp = {}) override;
 
+    void UpdateObjectsCountOnAbort();
+
     void Abort() override;
 
     void FinalizeTransaction() override;
@@ -212,17 +214,16 @@ class DiskStorage final : public Storage {
                                                          const rocksdb::Slice &ts);
 
    private:
-    VertexAccessor CreateVertex(utils::SkipList<Vertex>::Accessor &accessor, storage::Gid gid,
-                                std::vector<LabelId> &&label_ids, PropertyStore &&properties, Delta *delta);
+    VertexAccessor CreateVertexFromDisk(utils::SkipList<Vertex>::Accessor &accessor, storage::Gid gid,
+                                        std::vector<LabelId> &&label_ids, PropertyStore &&properties, Delta *delta);
 
     bool PrefetchEdgeFilter(const std::string_view disk_edge_key_str, const VertexAccessor &vertex_acc,
                             EdgeDirection edge_direction);
     void PrefetchEdges(const VertexAccessor &vertex_acc, EdgeDirection edge_direction);
 
-    Result<EdgeAccessor> CreateEdge(const VertexAccessor *from, const VertexAccessor *to, EdgeTypeId edge_type,
-                                    storage::Gid gid, std::string_view properties, const std::string &old_disk_key,
-                                    const std::string &ts);
-
+    Result<EdgeAccessor> CreateEdgeFromDisk(const VertexAccessor *from, const VertexAccessor *to, EdgeTypeId edge_type,
+                                            storage::Gid gid, std::string_view properties,
+                                            const std::string &old_disk_key, const std::string &ts);
     /// Flushes vertices and edges to the disk with the commit timestamp.
     /// At the time of calling, the commit_timestamp_ must already exist.
     /// After this method, the vertex and edge caches are cleared.
@@ -321,6 +322,8 @@ class DiskStorage final : public Storage {
 
   void LoadTimestampIfExists();
 
+  void LoadVertexAndEdgeCountIfExists();
+
   [[nodiscard]] std::optional<ConstraintViolation> CheckExistingVerticesBeforeCreatingExistenceConstraint(
       LabelId label, PropertyId property) const;
 
@@ -354,6 +357,7 @@ class DiskStorage final : public Storage {
  private:
   std::unique_ptr<RocksDBStorage> kvstore_;
   std::unique_ptr<kvstore::KVStore> durability_kvstore_;
+  std::atomic<uint64_t> vertex_count_{0};
 };
 
 }  // namespace memgraph::storage
diff --git a/src/storage/v2/storage.hpp b/src/storage/v2/storage.hpp
index 38b86350f..f2b10e5c8 100644
--- a/src/storage/v2/storage.hpp
+++ b/src/storage/v2/storage.hpp
@@ -302,7 +302,8 @@ class Storage {
 
   // 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.
+  // keep a separate count of edges that is always updated. This counter is also used
+  // for disk storage.
   std::atomic<uint64_t> edge_count_{0};
 
   std::unique_ptr<NameIdMapper> name_id_mapper_;
diff --git a/tests/e2e/disk_storage/CMakeLists.txt b/tests/e2e/disk_storage/CMakeLists.txt
index 680f980f6..777277178 100644
--- a/tests/e2e/disk_storage/CMakeLists.txt
+++ b/tests/e2e/disk_storage/CMakeLists.txt
@@ -12,3 +12,4 @@ copy_disk_storage_e2e_python_files(replication_disabled.py)
 copy_disk_storage_e2e_python_files(snapshot_disabled.py)
 copy_disk_storage_e2e_python_files(lock_data_dir_disabled.py)
 copy_disk_storage_e2e_python_files(create_edge_from_indices.py)
+copy_disk_storage_e2e_python_files(storage_info.py)
diff --git a/tests/e2e/disk_storage/create_edge_from_indices.py b/tests/e2e/disk_storage/create_edge_from_indices.py
index a1bb0a35c..652ff688a 100644
--- a/tests/e2e/disk_storage/create_edge_from_indices.py
+++ b/tests/e2e/disk_storage/create_edge_from_indices.py
@@ -10,7 +10,6 @@
 # licenses/APL.txt.
 
 import sys
-import typing
 
 import pytest
 from common import connect, execute_and_fetch_all
diff --git a/tests/e2e/disk_storage/storage_info.py b/tests/e2e/disk_storage/storage_info.py
new file mode 100644
index 000000000..a08ceac51
--- /dev/null
+++ b/tests/e2e/disk_storage/storage_info.py
@@ -0,0 +1,69 @@
+# Copyright 2023 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.
+
+import sys
+
+import pytest
+from common import connect, execute_and_fetch_all
+
+
+def test_empty_show_storage_info(connect):
+    cursor = connect.cursor()
+    execute_and_fetch_all(cursor, "STORAGE MODE ON_DISK_TRANSACTIONAL")
+    results = execute_and_fetch_all(cursor, "SHOW STORAGE INFO")
+    results = dict(map(lambda pair: (pair[0], pair[1]), results))
+    assert results["vertex_count"] == 0
+    assert results["edge_count"] == 0
+
+
+def test_show_storage_info_after_initialization(connect):
+    cursor = connect.cursor()
+    execute_and_fetch_all(cursor, "STORAGE MODE ON_DISK_TRANSACTIONAL")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 1})")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 2})")
+    execute_and_fetch_all(cursor, "MATCH (n:User {id: 1}), (m:User {id: 2}) CREATE (n)-[r:FRIEND {id: 1}]->(m)")
+    results = execute_and_fetch_all(cursor, "SHOW STORAGE INFO")
+    results = dict(map(lambda pair: (pair[0], pair[1]), results))
+    assert results["vertex_count"] == 2
+    assert results["edge_count"] == 1
+    execute_and_fetch_all(cursor, "MATCH (n) DETACH DELETE n")
+
+
+def test_show_storage_info_detach_delete_vertex(connect):
+    cursor = connect.cursor()
+    execute_and_fetch_all(cursor, "STORAGE MODE ON_DISK_TRANSACTIONAL")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 1})")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 2})")
+    execute_and_fetch_all(cursor, "MATCH (n:User {id: 1}), (m:User {id: 2}) CREATE (n)-[r:FRIEND {id: 1}]->(m)")
+    execute_and_fetch_all(cursor, "MATCH (n:User {id: 1}) DETACH DELETE n;")
+    results = execute_and_fetch_all(cursor, "SHOW STORAGE INFO")
+    results = dict(map(lambda pair: (pair[0], pair[1]), results))
+    assert results["vertex_count"] == 1
+    assert results["edge_count"] == 0
+    execute_and_fetch_all(cursor, "MATCH (n) DETACH DELETE n")
+
+
+def test_show_storage_info_delete_edge(connect):
+    cursor = connect.cursor()
+    execute_and_fetch_all(cursor, "STORAGE MODE ON_DISK_TRANSACTIONAL")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 1})")
+    execute_and_fetch_all(cursor, "CREATE (n:User {id: 2})")
+    execute_and_fetch_all(cursor, "MATCH (n:User {id: 1}), (m:User {id: 2}) CREATE (n)-[r:FRIEND {id: 1}]->(m)")
+    execute_and_fetch_all(cursor, "MATCH (n:User {id: 1})-[r]->(m:User {id: 2}) DELETE r;")
+    results = execute_and_fetch_all(cursor, "SHOW STORAGE INFO")
+    results = dict(map(lambda pair: (pair[0], pair[1]), results))
+    assert results["vertex_count"] == 2
+    assert results["edge_count"] == 0
+    execute_and_fetch_all(cursor, "MATCH (n) DETACH DELETE n")
+
+
+if __name__ == "__main__":
+    sys.exit(pytest.main([__file__, "-rA"]))
diff --git a/tests/e2e/disk_storage/workloads.yaml b/tests/e2e/disk_storage/workloads.yaml
index 4f449e77d..32c558935 100644
--- a/tests/e2e/disk_storage/workloads.yaml
+++ b/tests/e2e/disk_storage/workloads.yaml
@@ -7,6 +7,11 @@ disk_storage: &disk_storage
       validation_queries: []
 
 workloads:
+  - name: "Tests SHOW STORAGE INFO clause"
+    binary: "tests/e2e/pytest_runner.sh"
+    args: ["disk_storage/storage_info.py"]
+    <<: *disk_storage
+
   - name: "Test that loading vertices from indices and creating edge with them works."
     binary: "tests/e2e/pytest_runner.sh"
     args: ["disk_storage/create_edge_from_indices.py"]
diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt
index 34a094c12..b08e3a7e4 100644
--- a/tests/unit/CMakeLists.txt
+++ b/tests/unit/CMakeLists.txt
@@ -348,6 +348,9 @@ target_link_libraries(${test_prefix}storage_v2_replication mg-storage-v2 fmt)
 add_unit_test(storage_v2_isolation_level.cpp)
 target_link_libraries(${test_prefix}storage_v2_isolation_level mg-storage-v2)
 
+add_unit_test(storage_v2_show_storage_info.cpp)
+target_link_libraries(${test_prefix}storage_v2_show_storage_info mg-storage-v2)
+
 add_unit_test(storage_v2_storage_mode.cpp)
 target_link_libraries(${test_prefix}storage_v2_storage_mode mg-storage-v2 storage_test_utils mg-query mg-glue)
 
diff --git a/tests/unit/storage_v2_show_storage_info.cpp b/tests/unit/storage_v2_show_storage_info.cpp
new file mode 100644
index 000000000..08ec954a0
--- /dev/null
+++ b/tests/unit/storage_v2_show_storage_info.cpp
@@ -0,0 +1,62 @@
+// Copyright 2023 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 <filesystem>
+
+#include "disk_test_utils.hpp"
+#include "storage/v2/disk/storage.hpp"
+
+// NOLINTNEXTLINE(google-build-using-namespace)
+using namespace memgraph::storage;
+
+using testing::Types;
+using testing::UnorderedElementsAre;
+
+// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
+#define ASSERT_NO_ERROR(result) ASSERT_FALSE((result).HasError())
+
+class ShowStorageInfoTest : public testing::Test {
+ public:
+  const std::string testSuite = "storage_v2__show_storage_info";
+
+  ShowStorageInfoTest() {
+    config_ = disk_test_utils::GenerateOnDiskConfig(testSuite);
+    storage = std::make_unique<memgraph::storage::DiskStorage>(config_);
+  }
+
+  void TearDown() override {
+    storage.reset(nullptr);
+    disk_test_utils::RemoveRocksDbDirs(testSuite);
+  }
+
+  std::unique_ptr<Storage> storage;
+  memgraph::storage::Config config_;
+};
+
+TEST_F(ShowStorageInfoTest, CountOnAbort) {
+  auto acc = this->storage->Access();
+  auto src_vertex = acc->CreateVertex();
+  auto dest_vertex = acc->CreateVertex();
+  auto et = acc->NameToEdgeType("et5");
+  auto edge = acc->CreateEdge(&src_vertex, &dest_vertex, et).GetValue();
+  ASSERT_EQ(edge.EdgeType(), et);
+  ASSERT_EQ(edge.FromVertex(), src_vertex);
+  ASSERT_EQ(edge.ToVertex(), dest_vertex);
+  memgraph::storage::StorageInfo info_before_abort = this->storage->GetInfo();
+  ASSERT_EQ(info_before_abort.vertex_count, 2);
+  ASSERT_EQ(info_before_abort.edge_count, 1);
+  acc->Abort();
+  memgraph::storage::StorageInfo info_after_abort = this->storage->GetInfo();
+  ASSERT_EQ(info_after_abort.vertex_count, 0);
+  ASSERT_EQ(info_after_abort.edge_count, 0);
+}