Add deterministic disk vertex_count and edge_count ()

* Add exact vertex_count and edge_count to disk storage
This commit is contained in:
Andi 2023-08-29 13:07:23 +02:00 committed by GitHub
parent c526ff2a8f
commit a6ec81b179
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 257 additions and 50 deletions

View File

@ -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.

View File

@ -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() {

View File

@ -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

View File

@ -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_;

View File

@ -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)

View File

@ -10,7 +10,6 @@
# licenses/APL.txt.
import sys
import typing
import pytest
from common import connect, execute_and_fetch_all

View File

@ -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"]))

View File

@ -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"]

View File

@ -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)

View File

@ -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);
}