Compare commits

...

27 Commits

Author SHA1 Message Date
Andi Skrgat
c785bed20c rocksdb operations to accessors api 2023-04-14 14:06:04 +02:00
Andi Skrgat
b34044459e Rocks to v2 storage 2023-04-14 10:54:18 +02:00
Andi Skrgat
a22b011736 Compaction filter poc 2023-04-14 10:09:38 +02:00
Aidar Samerkhanov
ed76943dff Fix build of mg-query 2023-04-13 22:47:09 +00:00
Aidar Samerkhanov
e0da73780a Fix db_accessor build in query engine 2023-04-13 19:56:09 +00:00
Aidar Samerkhanov
18bf1c237b Fix some compilation errors in query engine module 2023-04-12 22:16:54 +00:00
Aidar Samerkhanov
faa3e542b6 Fix storage unit tests 2023-04-12 21:50:26 +00:00
Aidar Samerkhanov
2250cadd28 Fix build for storage tests 2023-04-12 21:38:52 +00:00
Aidar Samerkhanov
3c0ae50a15 Fix build of storage library. 2023-04-12 20:53:48 +00:00
Andi Skrgat
876359f4b6 Code edits 2023-04-12 13:43:47 +02:00
Aidar Samerkhanov
c64b607478 Add abstract classes for Accessor, VertexAccessor and EdgeAccessor. 2023-04-11 12:34:09 +00:00
Andi Skrgat
c441b7de0b Solved memory leaks 2023-04-07 15:10:59 +02:00
Andi Skrgat
b6beffa9e2 Delete api 2023-04-07 09:56:00 +02:00
Andi Skrgat
1bed62da4a Code cleaning 2023-04-06 16:28:29 +02:00
Andi Skrgat
2019f5cc72 in and out edges 2023-04-06 11:31:50 +02:00
Andi Skrgat
f070329e77 Edge serializaiton 2023-04-05 15:29:44 +02:00
Andi Skrgat
b8d956294b Buffer serialization of properties 2023-04-05 12:04:05 +02:00
Andi Skrgat
72371cc8ee Dummy properties serialization 2023-04-05 09:33:47 +02:00
Andi Skrgat
e193d4036e Multiple labels serialization test cases 2023-04-04 11:12:58 +02:00
Andi Skrgat
ceb3e67f8e Merge branch 'add-on-disk-transactional-storage' into disk-storage-poc 2023-04-03 11:13:25 +02:00
Andi Skrgat
da8aa68c86 Merge branch 'master' into add-on-disk-transactional-storage 2023-04-03 11:12:46 +02:00
Andi Skrgat
cc765f0ebd Merge branch 'master' of github.com:memgraph/memgraph 2023-04-03 11:09:57 +02:00
Andi Skrgat
f7c1923ab2 ScanAll 2023-04-03 11:08:47 +02:00
Josipmrden
7fe17ba9ef
[E216 < T1245] Add subqueries (#794) (#851)
Add subqueries

Co-authored-by: Bruno Sačarić <bruno.sacaric@gmail.com>
2023-03-31 15:24:02 +02:00
Marko Budiselic
3f209e3863 Merge branch 'master' into add-on-disk-transactional-storage 2023-03-31 13:04:25 +00:00
Andi Skrgat
7503b304f2 POC started 2023-03-31 15:00:08 +02:00
Marko Budiselic
81a329ce56 Add ON_DISK_TRANSACTIONAL storage 2023-03-29 20:46:20 +00:00
41 changed files with 6755 additions and 3162 deletions

View File

@ -98,6 +98,8 @@
#ifdef MG_ENTERPRISE
#include "audit/log.hpp"
#endif
// Disk storage includes
#include "storage/v2/disk/storage.hpp"
constexpr const char *kMgUser = "MEMGRAPH_USER";
constexpr const char *kMgPassword = "MEMGRAPH_PASSWORD";
@ -773,6 +775,7 @@ int main(int argc, char **argv) {
auto gil = memgraph::py::EnsureGIL();
// NOLINTNEXTLINE(hicpp-signed-bitwise)
auto *flag = PyLong_FromLong(RTLD_NOW | RTLD_DEEPBIND);
// auto *flag = PyLong_FromLong(RTLD_NOW);
auto *setdl = PySys_GetObject("setdlopenflags");
MG_ASSERT(setdl);
auto *arg = PyTuple_New(1);
@ -907,10 +910,15 @@ int main(int argc, char **argv) {
}
db_config.durability.snapshot_interval = std::chrono::seconds(FLAGS_storage_snapshot_interval_sec);
}
// here in the future, a specific instantiation of storage type will be created
memgraph::storage::Storage db(db_config);
// for experiments, I will first bind together both storages to make sure we solve serialization correctly
// if we decide to use dynamic polymorphism, the concrete storage object should care about RocksDB details
memgraph::storage::rocks::RocksDBStorage disk_db;
memgraph::query::InterpreterContext interpreter_context{
&db,
&disk_db,
{.query = {.allow_load_csv = FLAGS_allow_load_csv},
.execution_timeout_sec = FLAGS_query_execution_timeout_sec,
.replication_replica_check_frequency = std::chrono::seconds(FLAGS_replication_replica_check_frequency_sec),

View File

@ -20,6 +20,7 @@
#include "query/parameters.hpp"
#include "query/plan/profile.hpp"
#include "query/trigger.hpp"
#include "storage/v2/disk/storage.hpp"
#include "utils/async_timer.hpp"
namespace memgraph::query {
@ -86,6 +87,7 @@ struct ExecutionContext {
#ifdef MG_ENTERPRISE
std::unique_ptr<FineGrainedAuthChecker> auth_checker{nullptr};
#endif
storage::rocks::RocksDBStorage *disk_db;
};
static_assert(std::is_move_assignable_v<ExecutionContext>, "ExecutionContext must be move assignable!");

View File

@ -63,6 +63,10 @@ class EdgeAccessor final {
auto Properties(storage::View view) const { return impl_.Properties(view); }
auto PropertyStore() const { return impl_.PropertyStore(); }
void SetPropertyStore(const std::string_view buffer) const { impl_.SetPropertyStore(buffer); }
storage::Result<storage::PropertyValue> GetProperty(storage::View view, storage::PropertyId key) const {
return impl_.GetProperty(key, view);
}
@ -93,6 +97,8 @@ class EdgeAccessor final {
storage::Gid Gid() const noexcept { return impl_.Gid(); }
void SetGid(storage::Gid gid_) { impl_.SetGid(gid_); }
bool operator==(const EdgeAccessor &e) const noexcept { return impl_ == e.impl_; }
bool operator!=(const EdgeAccessor &e) const noexcept { return !(*this == e); }
@ -121,6 +127,10 @@ class VertexAccessor final {
auto Properties(storage::View view) const { return impl_.Properties(view); }
auto PropertyStore() const { return impl_.PropertyStore(); }
void SetPropertyStore(const std::string_view buffer) const { impl_.SetPropertyStore(buffer); }
storage::Result<storage::PropertyValue> GetProperty(storage::View view, storage::PropertyId key) const {
return impl_.GetProperty(key, view);
}
@ -182,6 +192,8 @@ class VertexAccessor final {
storage::Gid Gid() const noexcept { return impl_.Gid(); }
void SetGid(storage::Gid gid) { impl_.SetGid(gid); }
bool operator==(const VertexAccessor &v) const noexcept {
static_assert(noexcept(impl_ == v.impl_));
return impl_ == v.impl_;

View File

@ -50,6 +50,7 @@
#include "query/stream/common.hpp"
#include "query/trigger.hpp"
#include "query/typed_value.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_value.hpp"
@ -1015,6 +1016,7 @@ PullPlan::PullPlan(const std::shared_ptr<CachedPlan> plan, const Parameters &par
cursor_(plan->plan().MakeCursor(execution_memory)),
frame_(plan->symbol_table().max_position(), execution_memory),
memory_limit_(memory_limit) {
ctx_.disk_db = interpreter_context->disk_db;
ctx_.db_accessor = dba;
ctx_.symbol_table = plan->symbol_table();
ctx_.evaluation_context.timestamp = QueryTimestamp();
@ -1127,9 +1129,13 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *strea
using RWType = plan::ReadWriteTypeChecker::RWType;
} // namespace
InterpreterContext::InterpreterContext(storage::Storage *db, const InterpreterConfig config,
const std::filesystem::path &data_directory)
: db(db), trigger_store(data_directory / "triggers"), config(config), streams{this, data_directory / "streams"} {}
InterpreterContext::InterpreterContext(storage::Storage *db, storage::rocks::RocksDBStorage *disk_db,
const InterpreterConfig config, const std::filesystem::path &data_directory)
: db(db),
disk_db(disk_db),
trigger_store(data_directory / "triggers"),
config(config),
streams{this, data_directory / "streams"} {}
Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_context_(interpreter_context) {
MG_ASSERT(interpreter_context_, "Interpreter context must not be NULL");

View File

@ -32,6 +32,7 @@
#include "query/stream/streams.hpp"
#include "query/trigger.hpp"
#include "query/typed_value.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/isolation_level.hpp"
#include "utils/event_counter.hpp"
#include "utils/logging.hpp"
@ -208,10 +209,11 @@ class Interpreter;
*
*/
struct InterpreterContext {
explicit InterpreterContext(storage::Storage *db, InterpreterConfig config,
explicit InterpreterContext(storage::Storage *db, storage::rocks::RocksDBStorage *disk_db, InterpreterConfig config,
const std::filesystem::path &data_directory);
storage::Storage *db;
storage::rocks::RocksDBStorage *disk_db;
// ANTLR has singleton instance that is shared between threads. It is
// protected by locks inside of ANTLR. Unfortunately, they are not protected

View File

@ -492,7 +492,8 @@ UniqueCursorPtr ScanAll::MakeCursor(utils::MemoryResource *mem) const {
auto vertices = [this](Frame &, ExecutionContext &context) {
auto *db = context.db_accessor;
return std::make_optional(db->Vertices(view_));
auto vertices = std::make_optional(db->Vertices(view_));
return vertices;
};
return MakeUniqueCursorPtr<ScanAllCursor<decltype(vertices)>>(mem, output_symbol_, input_->MakeCursor(mem), view_,
std::move(vertices), "ScanAll");

View File

@ -845,7 +845,8 @@ bool SharedLibraryModule::Load(const std::filesystem::path &file_path) {
file_path_ = file_path;
dlerror(); // Clear any existing error.
// NOLINTNEXTLINE(hicpp-signed-bitwise)
handle_ = dlopen(file_path.c_str(), RTLD_NOW | RTLD_LOCAL | RTLD_DEEPBIND);
// handle_ = dlopen(file_path.c_str(), RTLD_NOW | RTLD_LOCAL | RTLD_DEEPBIND);
handle_ = dlopen(file_path.c_str(), RTLD_NOW | RTLD_LOCAL);
if (!handle_) {
spdlog::error(
utils::MessageWithLink("Unable to load module {}; {}.", file_path, dlerror(), "https://memgr.ph/modules"));

3
src/storage/README.md Normal file
View File

@ -0,0 +1,3 @@
# Storage Modes
* `ON_DISK_TRANSACTIONAL`

View File

@ -8,9 +8,20 @@ set(storage_v2_src_files
durability/wal.cpp
edge_accessor.cpp
indices.cpp
inmemory/edge_accessor.cpp
inmemory/storage.cpp
inmemory/vertex_accessor.cpp
property_store.cpp
vertex_accessor.cpp
storage.cpp)
disk/compaction_filter.cpp
disk/old_storage.hpp
disk/old_storage.cpp
disk/new_storage.hpp
disk/vertex_accessor.cpp
disk/vertex_accessor.hpp
disk/edge_accessor.cpp
disk/edge_accessor.hpp
storage.cpp
vertex_accessor.cpp)
set(storage_v2_src_files

View File

@ -0,0 +1,27 @@
// 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 <rocksdb/cache.h>
#include <rocksdb/compaction_filter.h>
#include <rocksdb/options.h>
#include <rocksdb/slice_transform.h>
class TimestampCompactionFilter : public rocksdb::CompactionFilter {
public:
const char *Name() const override { return "TimestampCompactionFilter"; }
/// Return true if the key-value pair should be removed from the database during compaction.
/// Filters KV entries that are older than the specified timestamp.
bool Filter(int level, const rocksdb::Slice &key, const rocksdb::Slice &existing_value, std::string *new_value,
bool *value_changed) const override {
return true;
}
};

View File

@ -0,0 +1,273 @@
// 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 "storage/v2/disk/edge_accessor.hpp"
#include <memory>
#include <tuple>
#include "storage/v2/disk/vertex_accessor.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/exceptions.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage {
bool DiskEdgeAccessor::IsVisible(const View view) const {
/*
bool exists = true;
bool deleted = true;
// When edges don't have properties, their isolation level is still dictated by MVCC ->
// iterate over the deltas of the from_vertex_ and see which deltas can be applied on edges.
if (!config_.properties_on_edges) {
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(from_vertex_->lock);
// Initialize deleted by checking if out edges contain edge_
deleted = std::find_if(from_vertex_->out_edges.begin(), from_vertex_->out_edges.end(), [&](const auto &out_edge) {
return std::get<2>(out_edge) == edge_;
}) == from_vertex_->out_edges.end();
delta = from_vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::RECREATE_OBJECT:
case Delta::Action::DELETE_OBJECT:
break;
case Delta::Action::ADD_OUT_EDGE: { // relevant for the from_vertex_ -> we just deleted the edge
if (delta.vertex_edge.edge == edge_) {
deleted = false;
}
break;
}
case Delta::Action::REMOVE_OUT_EDGE: { // also relevant for the from_vertex_ -> we just added the edge
if (delta.vertex_edge.edge == edge_) {
exists = false;
}
break;
}
}
});
return exists && (for_deleted_ || !deleted);
}
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return exists && (for_deleted_ || !deleted);
*/
throw utils::NotYetImplemented("IsVisible for edges without properties is not implemented yet.");
}
std::unique_ptr<VertexAccessor> DiskEdgeAccessor::FromVertex() const {
// return std::make_unique<DiskVertexAccessor>(from_vertex_, transaction_, indices_, constraints_, config_);
throw utils::NotYetImplemented("FromVertex is not implemented yet.");
}
std::unique_ptr<VertexAccessor> DiskEdgeAccessor::ToVertex() const {
// return std::make_unique<DiskVertexAccessor>(to_vertex_, transaction_, indices_, constraints_, config_);
throw utils::NotYetImplemented("ToVertex is not implemented yet.");
}
Result<storage::PropertyValue> DiskEdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
/*utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto current_value = edge_.ptr->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, current_value);
edge_.ptr->properties.SetProperty(property, value);
return std::move(current_value);
*/
throw utils::NotYetImplemented("SetProperty is not implemented yet.");
}
Result<bool> DiskEdgeAccessor::InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
/*utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
if (!edge_.ptr->properties.InitProperties(properties)) return false;
for (const auto &[property, _] : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, PropertyValue());
}
return true;
*/
throw utils::NotYetImplemented("InitProperties is not implemented yet.");
}
Result<std::map<PropertyId, PropertyValue>> DiskEdgeAccessor::ClearProperties() {
// if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;//
//
// std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
//
// if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
//
// if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
//
// auto properties = edge_.ptr->properties.Properties();
// for (const auto &property : properties) {
// CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property.first, property.second);
// }
//
// edge_.ptr->properties.ClearProperties();
//
// return std::move(properties);
throw utils::NotYetImplemented("ClearProperties is not implemented yet.");
}
Result<PropertyValue> DiskEdgeAccessor::GetProperty(PropertyId property, View view) const {
/*
if (!config_.properties_on_edges) return PropertyValue();
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
value = edge_.ptr->properties.GetProperty(property);
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
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::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 std::move(value);
*/
throw utils::NotYetImplemented("GetProperty is not implemented yet.");
}
Result<std::map<PropertyId, PropertyValue>> DiskEdgeAccessor::Properties(View view) const {
/*if (!config_.properties_on_edges) return std::map<PropertyId, PropertyValue>{};
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
properties = edge_.ptr->properties.Properties();
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
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::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 std::move(properties);
*/
throw utils::NotYetImplemented("Properties is not implemented yet.");
}
} // namespace memgraph::storage

View File

@ -0,0 +1,100 @@
// 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.
#pragma once
#include <optional>
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/view.hpp"
namespace memgraph::storage {
struct Vertex;
class VertexAccessor;
struct Indices;
struct Constraints;
class DiskEdgeAccessor final : public EdgeAccessor {
private:
friend class DiskStorage;
public:
DiskEdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
Indices *indices, Constraints *constraints, Config::Items config, bool for_deleted = false)
: EdgeAccessor(edge_type, transaction, config, for_deleted),
edge_(edge),
from_vertex_(from_vertex),
to_vertex_(to_vertex),
indices_(indices),
constraints_(constraints) {}
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const override;
std::unique_ptr<VertexAccessor> FromVertex() const override;
std::unique_ptr<VertexAccessor> ToVertex() const override;
EdgeTypeId EdgeType() const { return edge_type_; }
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<storage::PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) override;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) override;
/// Remove all properties and return old values for each removed property.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties() override;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const override;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const override;
storage::Gid Gid() const noexcept override {
if (config_.properties_on_edges) {
return edge_.ptr->gid;
} else {
return edge_.gid;
}
}
bool IsCycle() const override { return from_vertex_ == to_vertex_; }
bool operator==(const EdgeAccessor &other) const noexcept override {
const auto *otherEdge = dynamic_cast<const DiskEdgeAccessor *>(&other);
if (otherEdge == nullptr) return false;
return edge_ == otherEdge->edge_ && transaction_ == otherEdge->transaction_;
}
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
private:
EdgeRef edge_;
Vertex *from_vertex_;
Vertex *to_vertex_;
Indices *indices_;
Constraints *constraints_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,358 @@
// 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.
#pragma once
#include <algorithm>
#include <atomic>
#include <memory>
#include <mutex>
#include <variant>
#include "storage/v2/result.hpp"
#include "storage/v2/storage.hpp"
#include <gflags/gflags.h>
#include <spdlog/spdlog.h>
#include "io/network/endpoint.hpp"
#include "storage/v2/disk/edge_accessor.hpp"
#include "storage/v2/disk/old_storage.hpp"
#include "storage/v2/disk/vertex_accessor.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/replication_persistence_helper.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/exceptions.hpp"
#include "utils/file.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
#include "utils/message.hpp"
#include "utils/rw_lock.hpp"
#include "utils/spin_lock.hpp"
#include "utils/stat.hpp"
#include "utils/uuid.hpp"
/// REPLICATION ///
#include "storage/v2/replication/replication_client.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/storage_error.hpp"
/// RocksDB
#include <rocksdb/db.h>
#include <rocksdb/iterator.h>
#include <rocksdb/options.h>
#include <rocksdb/status.h>
namespace memgraph::storage {
using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler;
namespace {
inline constexpr uint16_t kEpochHistoryRetention = 1000;
// /// Use it for operations that must successfully finish.
inline void AssertRocksDBStatus(const rocksdb::Status &status) {
MG_ASSERT(status.ok(), "rocksdb: {}", status.ToString());
}
inline bool CheckRocksDBStatus(const rocksdb::Status &status) {
if (!status.ok()) [[unlikely]] {
spdlog::error("rocksdb: {}", status.ToString());
}
return status.ok();
}
} // namespace
DiskStorage::DiskAccessor::DiskAccessor(DiskStorage *storage, IsolationLevel isolation_level)
: storage_(storage),
// The lock must be acquired before creating the transaction object to
// prevent freshly created transactions from dangling in an active state
// during exclusive operations.
storage_guard_(storage_->main_lock_),
transaction_(storage->CreateTransaction(isolation_level)),
is_transaction_active_(true),
config_(storage->config_.items) {}
DiskStorage::DiskAccessor::DiskAccessor(DiskAccessor &&other) noexcept
: storage_(other.storage_),
storage_guard_(std::move(other.storage_guard_)),
transaction_(std::move(other.transaction_)),
commit_timestamp_(other.commit_timestamp_),
is_transaction_active_(other.is_transaction_active_),
config_(other.config_) {
// Don't allow the other accessor to abort our transaction in destructor.
other.is_transaction_active_ = false;
other.commit_timestamp_.reset();
}
DiskStorage::DiskAccessor::~DiskAccessor() {
if (is_transaction_active_) {
Abort();
}
FinalizeTransaction();
}
/*
This function will be the same for both storage modes. We can move it on a different place later.
*/
std::unique_ptr<VertexAccessor> DiskStorage::DiskAccessor::CreateVertex() {
OOMExceptionEnabler oom_exception;
auto gid = storage_->vertex_id_.fetch_add(1, std::memory_order_acq_rel);
auto acc = storage_->vertices_.access();
auto delta = CreateDeleteObjectDelta(&transaction_);
auto [it, inserted] = acc.insert(Vertex{storage::Gid::FromUint(gid), delta});
MG_ASSERT(inserted, "The vertex must be inserted here!");
MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!");
delta->prev.Set(&*it);
return std::make_unique<DiskVertexAccessor>(&*it, &transaction_, &storage_->indices_, &storage_->constraints_,
config_);
}
/*
Same as function above, but with a given GID.
*/
std::unique_ptr<VertexAccessor> DiskStorage::DiskAccessor::CreateVertex(storage::Gid gid) {
OOMExceptionEnabler oom_exception;
// NOTE: When we update the next `vertex_id_` here we perform a RMW
// (read-modify-write) operation that ISN'T atomic! But, that isn't an issue
// because this function is only called from the replication delta applier
// that runs single-threadedly and while this instance is set-up to apply
// threads (it is the replica), it is guaranteed that no other writes are
// possible.
storage_->vertex_id_.store(std::max(storage_->vertex_id_.load(std::memory_order_acquire), gid.AsUint() + 1),
std::memory_order_release);
auto acc = storage_->vertices_.access();
auto delta = CreateDeleteObjectDelta(&transaction_);
auto [it, inserted] = acc.insert(Vertex{gid, delta});
MG_ASSERT(inserted, "The vertex must be inserted here!");
MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!");
delta->prev.Set(&*it);
return std::make_unique<DiskVertexAccessor>(&*it, &transaction_, &storage_->indices_, &storage_->constraints_,
config_);
}
std::unique_ptr<VertexAccessor> DiskStorage::DiskAccessor::DeserializeVertex(const std::string_view key,
const std::string_view value) {
/// Create vertex
spdlog::info("Key to deserialize: {}", key);
const auto vertex_parts = utils::Split(key, "|");
auto impl = CreateVertex(storage::Gid::FromUint(std::stoull(vertex_parts[1])));
// Deserialize labels
if (!vertex_parts[0].empty()) {
const auto labels = utils::Split(vertex_parts[0], ",");
for (const auto &label : labels) {
const storage::LabelId label_id = storage::LabelId::FromUint(std::stoull(label));
// TODO: introduce error handling
impl->AddLabel(label_id);
}
}
impl->SetPropertyStore(value);
return impl;
}
std::unique_ptr<VertexAccessor> DiskStorage::DiskAccessor::FindVertex(storage::Gid gid, View /*view*/) {
auto it =
std::unique_ptr<rocksdb::Iterator>(storage_->db_->NewIterator(rocksdb::ReadOptions(), storage_->vertex_chandle));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
const auto &key = it->key().ToString();
if (const auto vertex_parts = utils::Split(key, "|"); vertex_parts[1] == SerializeIdType(gid)) {
return DeserializeVertex(key, it->value().ToStringView());
}
}
return {};
}
Result<std::unique_ptr<VertexAccessor>> DiskStorage::DiskAccessor::DeleteVertex(VertexAccessor *vertex) {
if (!CheckRocksDBStatus(
storage_->db_->Delete(rocksdb::WriteOptions(), storage_->vertex_chandle, SerializeVertex(vertex)))) {
return Error::SERIALIZATION_ERROR;
}
return Result<std::unique_ptr<VertexAccessor>>{std::unique_ptr<VertexAccessor>(vertex)};
}
Result<std::optional<std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>>>
DiskStorage::DiskAccessor::DetachDeleteVertex(VertexAccessor *vertex) {
using ReturnType = std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>;
auto del_vertex = DeleteVertex(vertex);
if (del_vertex.HasError()) {
return del_vertex.GetError();
}
auto out_edges = vertex->OutEdges(storage::View::OLD);
auto in_edges = vertex->InEdges(storage::View::OLD);
if (out_edges.HasError() || in_edges.HasError()) {
return out_edges.GetError();
}
// if (auto del_edges = DeleteEdges(*out_edges), del_in_edges = DeleteEdges(*in_edges);
// del_edges.has_value() && del_in_edges.has_value()) {
// del_edges->insert(del_in_edges->end(), std::make_move_iterator(del_in_edges->begin()),
// std::make_move_iterator(del_in_edges->end()));
// return std::make_pair(*del_vertex, *del_edges);
// }
return Error::SERIALIZATION_ERROR;
}
Result<std::unique_ptr<EdgeAccessor>> DiskStorage::DiskAccessor::CreateEdge(VertexAccessor *from, VertexAccessor *to,
EdgeTypeId edge_type) {
throw utils::NotYetImplemented("CreateEdge");
}
Result<std::unique_ptr<EdgeAccessor>> DiskStorage::DiskAccessor::CreateEdge(VertexAccessor *from, VertexAccessor *to,
EdgeTypeId edge_type, storage::Gid gid) {
throw utils::NotYetImplemented("CreateEdge");
}
Result<std::unique_ptr<EdgeAccessor>> DiskStorage::DiskAccessor::DeleteEdge(EdgeAccessor *edge) {
throw utils::NotYetImplemented("DeleteEdge");
}
const std::string &DiskStorage::DiskAccessor::LabelToName(LabelId label) const { return storage_->LabelToName(label); }
const std::string &DiskStorage::DiskAccessor::PropertyToName(PropertyId property) const {
return storage_->PropertyToName(property);
}
const std::string &DiskStorage::DiskAccessor::EdgeTypeToName(EdgeTypeId edge_type) const {
return storage_->EdgeTypeToName(edge_type);
}
LabelId DiskStorage::DiskAccessor::NameToLabel(const std::string_view name) { return storage_->NameToLabel(name); }
PropertyId DiskStorage::DiskAccessor::NameToProperty(const std::string_view name) {
return storage_->NameToProperty(name);
}
EdgeTypeId DiskStorage::DiskAccessor::NameToEdgeType(const std::string_view name) {
return storage_->NameToEdgeType(name);
}
void DiskStorage::DiskAccessor::AdvanceCommand() { ++transaction_.command_id; }
utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::Commit(
const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("Commit");
}
void DiskStorage::DiskAccessor::Abort() { throw utils::NotYetImplemented("Abort"); }
void DiskStorage::DiskAccessor::FinalizeTransaction() { throw utils::NotYetImplemented("FinalizeTransaction"); }
std::optional<uint64_t> DiskStorage::DiskAccessor::GetTransactionId() const {
throw utils::NotYetImplemented("GetTransactionId");
}
const std::string &DiskStorage::LabelToName(LabelId label) const { return name_id_mapper_.IdToName(label.AsUint()); }
const std::string &DiskStorage::PropertyToName(PropertyId property) const {
return name_id_mapper_.IdToName(property.AsUint());
}
const std::string &DiskStorage::EdgeTypeToName(EdgeTypeId edge_type) const {
return name_id_mapper_.IdToName(edge_type.AsUint());
}
LabelId DiskStorage::NameToLabel(const std::string_view name) {
return LabelId::FromUint(name_id_mapper_.NameToId(name));
}
PropertyId DiskStorage::NameToProperty(const std::string_view name) {
return PropertyId::FromUint(name_id_mapper_.NameToId(name));
}
EdgeTypeId DiskStorage::NameToEdgeType(const std::string_view name) {
return EdgeTypeId::FromUint(name_id_mapper_.NameToId(name));
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::CreateIndex(
LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("CreateIndex");
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::CreateIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("CreateIndex");
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DropIndex(
LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("DropIndex");
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DropIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("DropIndex");
}
IndicesInfo DiskStorage::ListAllIndices() const { throw utils::NotYetImplemented("ListAllIndices"); }
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> DiskStorage::CreateExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("CreateExistenceConstraint");
}
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DiskStorage::DropExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("DropExistenceConstraint");
}
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
DiskStorage::CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("CreateUniqueConstraint");
}
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
DiskStorage::DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> desired_commit_timestamp) {
throw utils::NotYetImplemented("DropUniqueConstraint");
}
ConstraintsInfo DiskStorage::ListAllConstraints() const { throw utils::NotYetImplemented("ListAllConstraints"); }
StorageInfo DiskStorage::GetInfo() const { throw utils::NotYetImplemented("GetInfo"); }
VerticesIterable DiskStorage::DiskAccessor::Vertices(LabelId label, View view) {
throw utils::NotYetImplemented("Vertices");
}
VerticesIterable DiskStorage::DiskAccessor::Vertices(LabelId label, PropertyId property, View view) {
throw utils::NotYetImplemented("Vertices");
}
VerticesIterable DiskStorage::DiskAccessor::Vertices(LabelId label, PropertyId property, const PropertyValue &value,
View view) {
throw utils::NotYetImplemented("Vertices");
}
VerticesIterable DiskStorage::DiskAccessor::Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound,
View view) {
throw utils::NotYetImplemented("Vertices");
}
Transaction DiskStorage::CreateTransaction(IsolationLevel isolation_level) {
throw utils::NotYetImplemented("CreateTransaction");
}
} // namespace memgraph::storage

View File

@ -0,0 +1,555 @@
// 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.
#pragma once
#include <atomic>
#include <cstdint>
#include <filesystem>
#include <optional>
#include <shared_mutex>
#include <span>
#include <variant>
#include "io/network/endpoint.hpp"
#include "kvstore/kvstore.hpp"
#include "storage/v2/commit_log.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/disk/vertex_accessor.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/rw_lock.hpp"
#include "utils/scheduler.hpp"
#include "utils/skip_list.hpp"
#include "utils/synchronized.hpp"
#include "utils/uuid.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/storage_error.hpp"
/// ROCKSDB
#include <rocksdb/db.h>
#include <rocksdb/iterator.h>
#include <rocksdb/options.h>
#include <rocksdb/status.h>
namespace memgraph::storage {
class DiskStorage final : public Storage {
public:
/// @throw std::system_error
/// @throw std::bad_alloc
explicit DiskStorage();
~DiskStorage();
class DiskAccessor final : public Storage::Accessor {
private:
friend class DiskStorage;
explicit DiskAccessor(DiskStorage *storage, IsolationLevel isolation_level);
public:
DiskAccessor(const DiskAccessor &) = delete;
DiskAccessor &operator=(const DiskAccessor &) = delete;
DiskAccessor &operator=(DiskAccessor &&other) = delete;
// NOTE: After the accessor is moved, all objects derived from it (accessors
// and iterators) are *invalid*. You have to get all derived objects again.
DiskAccessor(DiskAccessor &&other) noexcept;
~DiskAccessor() override;
/// @throw std::alloc
std::unique_ptr<VertexAccessor> CreateVertex() override;
std::unique_ptr<VertexAccessor> FindVertex(Gid gid, View view) override;
VerticesIterable Vertices(View view) override {
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
&storage_->indices_, &storage_->constraints_,
storage_->config_.items));
}
VerticesIterable Vertices(LabelId label, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override;
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount() const override { return storage_->vertices_.size(); }
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label) const override {
return storage_->indices_.label_index.ApproximateVertexCount(label);
}
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
}
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
}
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
}
std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const override {
return storage_->indices_.label_property_index.GetIndexStats(label, property);
}
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() override {
return storage_->indices_.label_property_index.ClearIndexStats();
}
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(
const std::span<std::string> labels) override {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
std::for_each(labels.begin(), labels.end(), [this, &deleted_indexes](const auto &label_str) {
std::vector<std::pair<LabelId, PropertyId>> loc_results =
storage_->indices_.label_property_index.DeleteIndexStatsForLabel(NameToLabel(label_str));
deleted_indexes.insert(deleted_indexes.end(), std::make_move_iterator(loc_results.begin()),
std::make_move_iterator(loc_results.end()));
});
return deleted_indexes;
}
void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const IndexStats &stats) override {
storage_->indices_.label_property_index.SetIndexStats(label, property, stats);
}
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::unique_ptr<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) override;
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>>>
DetachDeleteVertex(VertexAccessor *vertex) override;
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> CreateEdge(VertexAccessor *from, VertexAccessor *to,
EdgeTypeId edge_type) override;
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) override;
const std::string &LabelToName(LabelId label) const override;
const std::string &PropertyToName(PropertyId property) const override;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const override;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name) override;
bool LabelIndexExists(LabelId label) const override { return storage_->indices_.label_index.IndexExists(label); }
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const override {
return storage_->indices_.label_property_index.IndexExists(label, property);
}
IndicesInfo ListAllIndices() const override {
return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
}
ConstraintsInfo ListAllConstraints() const override {
return {ListExistenceConstraints(storage_->constraints_),
storage_->constraints_.unique_constraints.ListConstraints()};
}
void AdvanceCommand() override;
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
/// @throw std::bad_alloc
void Abort() override;
void FinalizeTransaction() override;
std::optional<uint64_t> GetTransactionId() const override;
private:
/// @throw std::bad_alloc
std::unique_ptr<VertexAccessor> CreateVertex(storage::Gid gid);
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type,
storage::Gid gid);
inline std::string SerializeIdType(const auto &id) { return std::to_string(id.AsUint()); }
// (De)serialization utility methods
std::string SerializeLabels(const auto &&labels) {
if (labels.HasError() || (*labels).empty()) {
return "";
}
std::string result = std::to_string((*labels)[0].AsUint());
std::string ser_labels = std::accumulate(
std::next((*labels).begin()), (*labels).end(), result,
[](const std::string &join, const auto &label_id) { return join + "," + std::to_string(label_id.AsUint()); });
return ser_labels;
}
std::string SerializeVertex(VertexAccessor *vertex_acc) {
std::string result = SerializeLabels(vertex_acc->Labels(storage::View::OLD)) + "|";
result += SerializeIdType(vertex_acc->Gid());
return result;
}
std::unique_ptr<VertexAccessor> DeserializeVertex(std::string_view key, std::string_view value);
DiskStorage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
Config::Items config_;
};
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override {
return std::unique_ptr<DiskAccessor>(new DiskAccessor{this, override_isolation_level.value_or(isolation_level_)});
}
const std::string &LabelToName(LabelId label) const override;
const std::string &PropertyToName(PropertyId property) const override;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const override;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
IndicesInfo ListAllIndices() const override;
/// Returns void if the existence constraint has been created.
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there is already a vertex existing that would break this new constraint.
/// * `ConstraintDefinitionError`: the constraint already exists.
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing existence constraint.
/// Returns void if the existence constraint has been dropped.
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintDefinitionError`: the constraint did not exists.
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an unique constraint.
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there are already vertices violating the constraint.
/// Returns `UniqueConstraints::CreationStatus` otherwise. Value can be:
/// * `SUCCESS` if the constraint was successfully created,
/// * `ALREADY_EXISTS` if the constraint already existed,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
/// @throw std::bad_alloc
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
/// Removes an existing unique constraint.
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// Returns `UniqueConstraints::DeletionStatus` otherwise. Value can be:
/// * `SUCCESS` if constraint was successfully removed,
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
ConstraintsInfo ListAllConstraints() const override;
StorageInfo GetInfo() const override;
bool LockPath() override;
bool UnlockPath() override;
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) override;
bool SetMainReplicationRole() override;
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config) override;
/// @pre The instance should have a MAIN role
bool UnregisterReplica(const std::string &name) override;
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name) override;
ReplicationRole GetReplicationRole() const override;
std::vector<ReplicaInfo> ReplicasInfo() override;
void FreeMemory() override;
void SetIsolationLevel(IsolationLevel isolation_level) override;
utils::BasicResult<CreateSnapshotError> CreateSnapshot() override;
private:
Transaction CreateTransaction(IsolationLevel isolation_level);
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage();
bool InitializeWalFile();
void FinalizeWalFile();
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
void RestoreReplicas();
bool ShouldStoreAndRestoreReplicas() const;
// Main storage lock.
//
// Accessors take a shared lock when starting, so it is possible to block
// creation of new accessors by taking a unique lock. This is used when doing
// operations on storage that affect the global state, for example index
// creation.
mutable utils::RWLock main_lock_{utils::RWLock::Priority::WRITE};
// Main object storage
utils::SkipList<storage::Vertex> vertices_;
utils::SkipList<storage::Edge> edges_;
std::atomic<uint64_t> vertex_id_{0};
std::atomic<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.
std::atomic<uint64_t> edge_count_{0};
NameIdMapper name_id_mapper_;
Constraints constraints_;
Indices indices_;
// Transaction engine
utils::SpinLock engine_lock_;
uint64_t timestamp_{kTimestampInitialId};
uint64_t transaction_id_{kTransactionInitialId};
// TODO: This isn't really a commit log, it doesn't even care if a
// transaction commited or aborted. We could probably combine this with
// `timestamp_` in a sensible unit, something like TransactionClock or
// whatever.
std::optional<CommitLog> commit_log_;
utils::Synchronized<std::list<Transaction>, utils::SpinLock> committed_transactions_;
IsolationLevel isolation_level_;
Config config_;
utils::Scheduler gc_runner_;
std::mutex gc_lock_;
// Undo buffers that were unlinked and now are waiting to be freed.
utils::Synchronized<std::list<std::pair<uint64_t, std::list<Delta>>>, utils::SpinLock> garbage_undo_buffers_;
// Vertices that are logically deleted but still have to be removed from
// indices before removing them from the main storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_vertices_;
// Vertices that are logically deleted and removed from indices and now wait
// to be removed from the main storage.
std::list<std::pair<uint64_t, Gid>> garbage_vertices_;
// Edges that are logically deleted and wait to be removed from the main
// storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_edges_;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path wal_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
std::unique_ptr<kvstore::KVStore> storage_;
utils::Scheduler snapshot_runner_;
utils::SpinLock snapshot_lock_;
// UUID used to distinguish snapshots and to link snapshots to WALs
std::string uuid_;
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_;
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
utils::FileRetainer file_retainer_;
// Global locker that is used for clients file locking
utils::FileRetainer::FileLocker global_locker_;
// Last commited timestamp
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
class ReplicationServer;
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
class ReplicationClient;
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<ReplicationRole> replication_role_{ReplicationRole::MAIN};
rocksdb::Options options_;
rocksdb::DB *db_;
rocksdb::ColumnFamilyHandle *vertex_chandle = nullptr;
rocksdb::ColumnFamilyHandle *edge_chandle = nullptr;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,389 @@
// 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.
#pragma once
#include <rocksdb/db.h>
#include <rocksdb/iterator.h>
#include <rocksdb/options.h>
#include <rocksdb/status.h>
#include "query/db_accessor.hpp"
#include "utils/file.hpp"
#include "utils/string.hpp"
namespace memgraph::storage::rocks {
constexpr const char *vertexHandle = "vertex";
constexpr const char *edgeHandle = "edge";
constexpr const char *outEdgeDirection = "0";
constexpr const char *inEdgeDirection = "1";
// /// Use it for operations that must successfully finish.
inline void AssertRocksDBStatus(const rocksdb::Status &status) {
MG_ASSERT(status.ok(), "rocksdb: {}", status.ToString());
}
// inline bool CheckRocksDBStatus(const rocksdb::Status &status) {
// if (!status.ok()) [[unlikely]] {
// spdlog::error("rocksdb: {}", status.ToString());
// }
// return status.ok();
// }
class RocksDBStorage {
public:
explicit RocksDBStorage() {
options_.create_if_missing = true;
// options_.OptimizeLevelStyleCompaction();
std::filesystem::path rocksdb_path = "./rocks_experiment_unit";
MG_ASSERT(utils::EnsureDir(rocksdb_path), "Unable to create storage folder on the disk.");
AssertRocksDBStatus(rocksdb::DB::Open(options_, rocksdb_path, &db_));
AssertRocksDBStatus(db_->CreateColumnFamily(rocksdb::ColumnFamilyOptions(), vertexHandle, &vertex_chandle));
AssertRocksDBStatus(db_->CreateColumnFamily(rocksdb::ColumnFamilyOptions(), edgeHandle, &edge_chandle));
}
RocksDBStorage(const RocksDBStorage &) = delete;
RocksDBStorage &operator=(const RocksDBStorage &) = delete;
RocksDBStorage &operator=(RocksDBStorage &&) = delete;
RocksDBStorage(RocksDBStorage &&) = delete;
~RocksDBStorage() {
AssertRocksDBStatus(db_->DropColumnFamily(vertex_chandle));
AssertRocksDBStatus(db_->DropColumnFamily(edge_chandle));
AssertRocksDBStatus(db_->DestroyColumnFamilyHandle(vertex_chandle));
AssertRocksDBStatus(db_->DestroyColumnFamilyHandle(edge_chandle));
AssertRocksDBStatus(db_->Close());
delete db_;
}
// // EDGE ACCESSOR FUNCTIONALITIES
// // -----------------------------------------------------------
// /// fetch the edge's source vertex by its GID
// std::optional<query::VertexAccessor> FromVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
// return FindVertex(SerializeIdType(edge_acc.From().Gid()), dba);
// }
// /// fetch the edge's destination vertex by its GID
// std::optional<query::VertexAccessor> ToVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
// return FindVertex(SerializeIdType(edge_acc.To().Gid()), dba);
// }
// /// VERTEX ACCESSOR FUNCTIONALITIES
// /// ------------------------------------------------------------
// /// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
// /// src_gid | other_vertex_gid | 0 | ...
// /// other_vertex_gid | src_gid | 1 | ...
// /// We use the firt way since this should be possible to optimize using Bloom filters and prefix search
// std::vector<query::EdgeAccessor> OutEdges(const query::VertexAccessor &vertex_acc, query::DbAccessor &dba) {
// const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
// std::vector<query::EdgeAccessor> out_edges;
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
// const std::string_view key = it->key().ToStringView();
// const auto vertex_parts = utils::Split(key, "|");
// if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == outEdgeDirection) {
// out_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
// }
// }
// return out_edges;
// }
// /// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
// /// other_vertex_gid | dest_gid | 0 | ...
// /// dest_gid | other_verte_gid | 1 | ...
// /// we use the second way since this should be possible to optimize using Bloom filters and prefix search.
// std::vector<query::EdgeAccessor> InEdges(const query::VertexAccessor &vertex_acc, query::DbAccessor &dba) {
// const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
// std::vector<query::EdgeAccessor> in_edges;
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
// const std::string_view key = it->key().ToStringView();
// const auto vertex_parts = utils::Split(key, "|");
// if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == inEdgeDirection) {
// in_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
// }
// }
// return in_edges;
// }
// /// TODO: how will we handle new vertex creation
// /// STORAGE ACCESSOR FUNCTIONALITIES
// /// -----------------------------------------------------------
// /// TODO: how will we handle new edge creation
// /// @return Accessor to the deleted edge if a deletion took place, std::nullopt otherwise.
// /// Delete two edge entries since on edge is represented on a two-fold level.
// /// Edges are deleted from logical partition containing edges.
// std::optional<query::EdgeAccessor> DeleteEdge(const query::EdgeAccessor &edge_acc) {
// auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
// if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, src_dest_key)) ||
// !CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, dest_src_key))) {
// return std::nullopt;
// }
// return edge_acc;
// }
// /// Helper function, not used in the real accessor.
// std::optional<std::vector<query::EdgeAccessor>> DeleteEdges(const auto &edge_accessors) {
// std::vector<query::EdgeAccessor> edge_accs;
// for (auto &&it : edge_accessors) {
// if (const auto deleted_edge_res = DeleteEdge(it); !deleted_edge_res.has_value()) {
// return std::nullopt;
// }
// edge_accs.push_back(it);
// }
// return edge_accs;
// }
// /// @return A reference to the deleted vertex accessor if deleted, otherwise std::nullopt.
// /// Delete vertex from logical partition containing vertices.
// std::optional<query::VertexAccessor> DeleteVertex(const query::VertexAccessor &vertex_acc) {
// if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc)))) {
// return std::nullopt;
// }
// return vertex_acc;
// }
// /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise.
// /// Delete vertex from logical partition containing vertices.
// /// For each edge delete two key-value entries from logical partition containing edges.
// std::optional<std::pair<query::VertexAccessor, std::vector<query::EdgeAccessor>>> DetachDeleteVertex(
// const query::VertexAccessor &vertex_acc) {
// auto del_vertex = DeleteVertex(vertex_acc);
// if (!del_vertex.has_value()) {
// return std::nullopt;
// }
// auto out_edges = vertex_acc.OutEdges(storage::View::OLD);
// auto in_edges = vertex_acc.InEdges(storage::View::OLD);
// if (out_edges.HasError() || in_edges.HasError()) {
// return std::nullopt;
// }
// if (auto del_edges = DeleteEdges(*out_edges), del_in_edges = DeleteEdges(*in_edges);
// del_edges.has_value() && del_in_edges.has_value()) {
// del_edges->insert(del_in_edges->end(), std::make_move_iterator(del_in_edges->begin()),
// std::make_move_iterator(del_in_edges->end()));
// return std::make_pair(*del_vertex, *del_edges);
// }
// return std::nullopt;
// }
// /// STORING
// /// -----------------------------------------------------------
// /// Serialize and store in-memory vertex to the disk.
// /// Properties are serialized as the value
// void StoreVertex(const query::VertexAccessor &vertex_acc) {
// AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc),
// SerializeProperties(vertex_acc.PropertyStore())));
// }
// /// Store edge as two key-value entries in the RocksDB.
// void StoreEdge(const query::EdgeAccessor &edge_acc) {
// auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
// const std::string value = SerializeProperties(edge_acc.PropertyStore());
// AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), edge_chandle, src_dest_key, value));
// AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), edge_chandle, dest_src_key, value));
// }
// /// UPDATE PART
// /// -----------------------------------------------------------
// /// Clear all entries from the database.
// /// TODO: check if this deletes all entries, or you also need to specify handle here
// /// TODO: This will not be needed in the production code and can possibly removed in testing
// void Clear() {
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions()));
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
// db_->Delete(rocksdb::WriteOptions(), it->key().ToString());
// }
// }
// /// READ PART
// /// -----------------------------------------------------------
// /// TODO: if the need comes for using also a GID object, use std::variant
// /// This should again be changed when we have mulitple same vertices
// std::optional<query::VertexAccessor> FindVertex(const std::string_view gid, query::DbAccessor &dba) {
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), vertex_chandle));
// std::optional<query::VertexAccessor> result = {};
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
// const auto &key = it->key().ToString();
// if (const auto vertex_parts = utils::Split(key, "|"); vertex_parts[1] == gid) {
// result = DeserializeVertex(key, it->value().ToStringView(), dba);
// break;
// }
// }
// return result;
// }
// /// Get all vertices by a label.
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::LabelId &label_id) {
// return Vertices(dba, [label_id](const auto &vertex) {
// const auto res = vertex.HasLabel(storage::View::OLD, label_id);
// return !res.HasError() && *res;
// });
// }
// /// Read all vertices stored in the database by a property
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::PropertyId &property_id,
// const storage::PropertyValue &property_value) {
// return Vertices(dba, [property_id, property_value](const auto &vertex) {
// const auto res = vertex.GetProperty(storage::View::OLD, property_id);
// return !res.HasError() && *res == property_value;
// });
// }
// /// Get all vertices.
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba) {
// return Vertices(dba, [](const auto & /*vertex*/) { return true; });
// }
// /// Read all vertices stored in the database and filter them by a lambda function.
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const auto &vertex_filter) {
// std::vector<query::VertexAccessor> vertices;
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), vertex_chandle));
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
// auto vertex = DeserializeVertex(it->key().ToStringView(), it->value().ToStringView(), dba);
// if (vertex_filter(vertex)) {
// vertices.push_back(vertex);
// }
// }
// return vertices;
// }
// private:
// /// Serialization of properties is done by saving the property store buffer
// /// If the data is stored in the local buffer of the property store, data from the buffer is copied to the string
// /// If the data is stored in some external buffer, the data is read from that location and copied to the string
// inline std::string SerializeProperties(const auto &&properties) { return properties; }
// /// Serialize labels delimitied by | to string
// std::string SerializeLabels(const auto &&labels) {
// if (labels.HasError() || (*labels).empty()) {
// return "";
// }
// std::string result = std::to_string((*labels)[0].AsUint());
// std::string ser_labels = std::accumulate(
// std::next((*labels).begin()), (*labels).end(), result,
// [](const std::string &join, const auto &label_id) { return join + "," + std::to_string(label_id.AsUint());
// });
// return ser_labels;
// }
// /// Serializes id type to string
// inline std::string SerializeIdType(const auto &id) { return std::to_string(id.AsUint()); }
// /// Serialize vertex to string
// /// The format: | label1,label2,label3 | gid
// std::string SerializeVertex(const query::VertexAccessor &vertex_acc) {
// std::string result = SerializeLabels(vertex_acc.Labels(storage::View::OLD)) + "|";
// result += SerializeIdType(vertex_acc.Gid());
// return result;
// }
// /// Deserialize vertex from string
// /// Properties are read from value and set to the vertex later
// query::VertexAccessor DeserializeVertex(const std::string_view key, const std::string_view value,
// query::DbAccessor &dba) {
// /// Create vertex
// auto impl = dba.InsertVertex();
// spdlog::info("Key to deserialize: {}", key);
// const auto vertex_parts = utils::Split(key, "|");
// // Deserialize labels
// if (!vertex_parts[0].empty()) {
// const auto labels = utils::Split(vertex_parts[0], ",");
// for (const auto &label : labels) {
// const storage::LabelId label_id = storage::LabelId::FromUint(std::stoull(label));
// auto maybe_error = impl.AddLabel(label_id);
// if (maybe_error.HasError()) {
// switch (maybe_error.GetError()) {
// case storage::Error::SERIALIZATION_ERROR:
// throw utils::BasicException("Serialization");
// case storage::Error::DELETED_OBJECT:
// throw utils::BasicException("Trying to set a label on a deleted node.");
// case storage::Error::VERTEX_HAS_EDGES:
// case storage::Error::PROPERTIES_DISABLED:
// case storage::Error::NONEXISTENT_OBJECT:
// throw utils::BasicException("Unexpected error when setting a label.");
// }
// }
// }
// }
// impl.SetGid(storage::Gid::FromUint(std::stoull(vertex_parts[1])));
// impl.SetPropertyStore(value);
// return impl;
// }
// /// Serializes edge accessor to obtain a key for the key-value store.
// /// @return two strings because there will be two keys since edge is stored in both directions.
// // | from_gid | to_gid | direction | edge_type | edge_gid
// std::pair<std::string, std::string> SerializeEdge(const query::EdgeAccessor &edge_acc) {
// // Serialized objects
// auto from_gid = SerializeIdType(edge_acc.From().Gid());
// auto to_gid = SerializeIdType(edge_acc.To().Gid());
// auto edge_type = SerializeIdType(edge_acc.EdgeType());
// auto edge_gid = SerializeIdType(edge_acc.Gid());
// // source->destination key
// std::string src_dest_key = from_gid + "|";
// src_dest_key += to_gid + "|";
// src_dest_key += outEdgeDirection;
// src_dest_key += "|" + edge_type + "|";
// src_dest_key += edge_gid;
// // destination->source key
// std::string dest_src_key = to_gid + "|";
// dest_src_key += from_gid + "|";
// dest_src_key += inEdgeDirection;
// dest_src_key += "|" + edge_type + "|";
// dest_src_key += edge_gid;
// return {src_dest_key, dest_src_key};
// }
// /// Deserialize edge from the given key-value.
// /// Properties are read from value and set to the edge later.
// query::EdgeAccessor DeserializeEdge(const std::string_view key, const std::string_view value,
// query::DbAccessor &dba) {
// const auto edge_parts = utils::Split(key, "|");
// auto [from_gid, to_gid] = std::invoke(
// [&](const auto &edge_parts) {
// if (edge_parts[2] == "0") { // out edge
// return std::make_pair(edge_parts[0], edge_parts[1]);
// }
// // in edge
// return std::make_pair(edge_parts[1], edge_parts[0]);
// },
// edge_parts);
// // load vertex accessors
// auto from_acc = FindVertex(from_gid, dba);
// auto to_acc = FindVertex(to_gid, dba);
// if (!from_acc.has_value() || !to_acc.has_value()) {
// throw utils::BasicException("Non-existing vertices during edge deserialization");
// }
// const auto edge_type_id = storage::EdgeTypeId::FromUint(std::stoull(edge_parts[3]));
// auto maybe_edge = dba.InsertEdge(&*from_acc, &*to_acc, edge_type_id);
// MG_ASSERT(maybe_edge.HasValue());
// // in the new storage API, setting gid must be done atomically
// maybe_edge->SetGid(storage::Gid::FromUint(std::stoull(edge_parts[4])));
// maybe_edge->SetPropertyStore(value);
// return *maybe_edge;
// }
rocksdb::Options options_;
rocksdb::DB *db_;
rocksdb::ColumnFamilyHandle *vertex_chandle = nullptr;
rocksdb::ColumnFamilyHandle *edge_chandle = nullptr;
};
} // namespace memgraph::storage::rocks

View File

@ -0,0 +1,621 @@
// 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 "storage/v2/disk/vertex_accessor.hpp"
#include <memory>
#include "storage/v2/disk/edge_accessor.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/exceptions.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage {
namespace detail {
namespace {
std::pair<bool, bool> IsVisible(Vertex *vertex, Transaction *transaction, View view) {
bool exists = true;
bool deleted = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex->lock);
deleted = vertex->deleted;
delta = vertex->delta;
}
ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return {exists, deleted};
}
} // namespace
} // namespace detail
std::unique_ptr<DiskVertexAccessor> DiskVertexAccessor::Create(Vertex *vertex, Transaction *transaction,
Indices *indices, Constraints *constraints,
Config::Items config, View view) {
// if (const auto [exists, deleted] = detail::IsVisible(vertex, transaction, view); !exists || deleted) {
// return {};
// }
//
// return std::make_unique<DiskVertexAccessor>(vertex, transaction, indices, constraints, config);
throw utils::NotYetImplemented("DiskVertexAccessor::Create");
}
bool DiskVertexAccessor::IsVisible(View view) const {
// const auto [exists, deleted] = detail::IsVisible(vertex_, transaction_, view);
// return exists && (for_deleted_ || !deleted);
throw utils::NotYetImplemented("DiskVertexAccessor::IsVisible");
}
Result<bool> DiskVertexAccessor::AddLabel(LabelId label) {
// utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
//
// if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
//
// if (vertex_->deleted) return Error::DELETED_OBJECT;
//
// if (std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end()) return false;
//
// CreateAndLinkDelta(transaction_, vertex_, Delta::RemoveLabelTag(), label);
//
// vertex_->labels.push_back(label);
//
// UpdateOnAddLabel(indices_, label, vertex_, *transaction_);
//
// return true;
throw utils::NotYetImplemented("DiskVertexAccessor::AddLabel");
}
Result<bool> DiskVertexAccessor::RemoveLabel(LabelId label) {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
//
// if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
//
// if (vertex_->deleted) return Error::DELETED_OBJECT;
//
// auto it = std::find(vertex_->labels.begin(), vertex_->labels.end(), label);
// if (it == vertex_->labels.end()) return false;
//
// CreateAndLinkDelta(transaction_, vertex_, Delta::AddLabelTag(), label);
//
// std::swap(*it, *vertex_->labels.rbegin());
// vertex_->labels.pop_back();
// return true;
throw utils::NotYetImplemented("DiskVertexAccessor::RemoveLabel");
}
Result<bool> DiskVertexAccessor::HasLabel(LabelId label, View view) const {
// bool exists = true;
// bool deleted = false;
// bool has_label = false;
// Delta *delta = nullptr;
// {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
// deleted = vertex_->deleted;
// has_label = std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end();
// delta = vertex_->delta;
// }
// ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &has_label, label](const Delta &delta) {
// switch (delta.action) {
// case Delta::Action::REMOVE_LABEL: {
// if (delta.label == label) {
// MG_ASSERT(has_label, "Invalid database state!");
// has_label = false;
// }
// break;
// }
// case Delta::Action::ADD_LABEL: {
// if (delta.label == label) {
// MG_ASSERT(!has_label, "Invalid database state!");
// has_label = true;
// }
// break;
// }
// case Delta::Action::DELETE_OBJECT: {
// exists = false;
// break;
// }
// case Delta::Action::RECREATE_OBJECT: {
// deleted = false;
// break;
// }
// 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 has_label;
throw utils::NotYetImplemented("DiskVertexAccessor::HasLabel");
}
Result<std::vector<LabelId>> DiskVertexAccessor::Labels(View view) const {
// bool exists = true;
// bool deleted = false;
// std::vector<LabelId> labels;
// Delta *delta = nullptr;
// {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
// deleted = vertex_->deleted;
// labels = vertex_->labels;
// delta = vertex_->delta;
// }
// ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &labels](const Delta &delta) {
// switch (delta.action) {
// case Delta::Action::REMOVE_LABEL: {
// // Remove the label because we don't see the addition.
// auto it = std::find(labels.begin(), labels.end(), delta.label);
// MG_ASSERT(it != labels.end(), "Invalid database state!");
// std::swap(*it, *labels.rbegin());
// labels.pop_back();
// break;
// }
// case Delta::Action::ADD_LABEL: {
// // Add the label because we don't see the removal.
// auto it = std::find(labels.begin(), labels.end(), delta.label);
// MG_ASSERT(it == labels.end(), "Invalid database state!");
// labels.push_back(delta.label);
// break;
// }
// case Delta::Action::DELETE_OBJECT: {
// exists = false;
// break;
// }
// case Delta::Action::RECREATE_OBJECT: {
// deleted = false;
// break;
// }
// 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 std::move(labels);
throw utils::NotYetImplemented("DiskVertexAccessor::Labels");
}
Result<PropertyValue> DiskVertexAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
// utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
//
// if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
//
// if (vertex_->deleted) return Error::DELETED_OBJECT;
//
// auto current_value = vertex_->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
// CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
// vertex_->properties.SetProperty(property, value);
//
// UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
//
// return std::move(current_value);
throw utils::NotYetImplemented("DiskVertexAccessor::SetProperty");
}
Result<bool> DiskVertexAccessor::InitProperties(
const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
// utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
//
// if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
//
// if (vertex_->deleted) return Error::DELETED_OBJECT;
//
// if (!vertex_->properties.InitProperties(properties)) return false;
// for (const auto &[property, value] : properties) {
// CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, PropertyValue());
// UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
// }
//
// return true;
throw utils::NotYetImplemented("DiskVertexAccessor::InitProperties");
}
Result<std::map<PropertyId, PropertyValue>> DiskVertexAccessor::ClearProperties() {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
//
// if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
//
// if (vertex_->deleted) return Error::DELETED_OBJECT;
//
// auto properties = vertex_->properties.Properties();
// for (const auto &property : properties) {
// CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property.first, property.second);
// UpdateOnSetProperty(indices_, property.first, PropertyValue(), vertex_, *transaction_);
// }
//
// vertex_->properties.ClearProperties();
//
// return std::move(properties);
throw utils::NotYetImplemented("DiskVertexAccessor::ClearProperties");
}
Result<PropertyValue> DiskVertexAccessor::GetProperty(PropertyId property, View view) const {
// bool exists = true;
// bool deleted = false;
// PropertyValue value;
// Delta *delta = nullptr;
// {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
// deleted = vertex_->deleted;
// value = vertex_->properties.GetProperty(property);
// delta = vertex_->delta;
// }
// ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
// switch (delta.action) {
// case Delta::Action::SET_PROPERTY: {
// if (delta.property.key == property) {
// value = delta.property.value;
// }
// break;
// }
// 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::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 std::move(value);
throw utils::NotYetImplemented("DiskVertexAccessor::GetProperty");
}
Result<std::map<PropertyId, PropertyValue>> DiskVertexAccessor::Properties(View view) const {
// bool exists = true;
// bool deleted = false;
// std::map<PropertyId, PropertyValue> properties;
// Delta *delta = nullptr;
// {
// std::lock_guard<utils::SpinLock> guard(vertex_->lock);
// deleted = vertex_->deleted;
// properties = vertex_->properties.Properties();
// delta = vertex_->delta;
// }
// ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
// switch (delta.action) {
// case Delta::Action::SET_PROPERTY: {
// auto it = properties.find(delta.property.key);
// if (it != properties.end()) {
// if (delta.property.value.IsNull()) {
// // remove the property
// properties.erase(it);
// } else {
// // set the value
// it->second = delta.property.value;
// }
// } else if (!delta.property.value.IsNull()) {
// properties.emplace(delta.property.key, delta.property.value);
// }
// break;
// }
// 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::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 std::move(properties);
throw utils::NotYetImplemented("DiskVertexAccessor::Properties");
}
Result<std::vector<std::unique_ptr<EdgeAccessor>>> DiskVertexAccessor::InEdges(
View view, const std::vector<EdgeTypeId> &edge_types, const VertexAccessor *destination) const {
/*
auto *destVA = dynamic_cast<const DiskVertexAccessor *>(destination);
MG_ASSERT(!destination || destVA, "Target VertexAccessor must be from the same storage as the storage accessor!");
MG_ASSERT(!destVA || destVA->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destVA) {
in_edges = vertex_->in_edges;
} else {
for (const auto &item : vertex_->in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
if (destVA && from_vertex != destVA->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
in_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destVA](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE: {
if (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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 (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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());
in_edges.pop_back();
break;
}
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<std::unique_ptr<EdgeAccessor>> ret;
ret.reserve(in_edges.size());
for (const auto &item : in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
ret.emplace_back(std::make_unique<DiskEdgeAccessor>(edge, edge_type, from_vertex, vertex_, transaction_,
indices_, constraints_, config_));
}
return std::move(ret);*/
throw utils::NotYetImplemented("DiskVertexAccessor::InEdges");
}
Result<std::vector<std::unique_ptr<EdgeAccessor>>> DiskVertexAccessor::OutEdges(
View view, const std::vector<EdgeTypeId> &edge_types, const VertexAccessor *destination) const {
/*auto *destVA = dynamic_cast<const DiskVertexAccessor *>(destination);
MG_ASSERT(!destination || destVA, "Target VertexAccessor must be from the same storage as the storage accessor!");
MG_ASSERT(!destVA || destVA->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destVA) {
out_edges = vertex_->out_edges;
} else {
for (const auto &item : vertex_->out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
if (destVA && to_vertex != destVA->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
out_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destVA](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE: {
if (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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 (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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());
out_edges.pop_back();
break;
}
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<std::unique_ptr<EdgeAccessor>> ret;
ret.reserve(out_edges.size());
for (const auto &item : out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
ret.emplace_back(std::make_unique<DiskEdgeAccessor>(edge, edge_type, vertex_, to_vertex, transaction_, indices_,
constraints_, config_));
}
return std::move(ret);*/
throw utils::NotYetImplemented("DiskVertexAccessor::OutEdges");
}
Result<size_t> DiskVertexAccessor::InDegree(View view) const {
/*bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->in_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE:
++degree;
break;
case Delta::Action::REMOVE_IN_EDGE:
--degree;
break;
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
Result<size_t> DiskVertexAccessor::OutDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->out_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE:
++degree;
break;
case Delta::Action::REMOVE_OUT_EDGE:
--degree;
break;
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
*/
throw utils::NotYetImplemented("DiskVertexAccessor::OutDegree");
}
} // namespace memgraph::storage

View File

@ -0,0 +1,117 @@
// 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.
#pragma once
#include <memory>
#include "storage/v2/vertex.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "storage/v2/view.hpp"
namespace memgraph::storage {
class EdgeAccessor;
class Storage;
struct Indices;
struct Constraints;
class DiskVertexAccessor final : public VertexAccessor {
private:
friend class DiskStorage;
public:
DiskVertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config, bool for_deleted = false)
: VertexAccessor(transaction, config, for_deleted),
vertex_(vertex),
indices_(indices),
constraints_(constraints) {}
static std::unique_ptr<DiskVertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const override;
/// Add a label and return `true` if insertion took place.
/// `false` is returned if the label already existed.
/// @throw std::bad_alloc
Result<bool> AddLabel(LabelId label) override;
/// Remove a label and return `true` if deletion took place.
/// `false` is returned if the vertex did not have a label already.
/// @throw std::bad_alloc
Result<bool> RemoveLabel(LabelId label) override;
Result<bool> HasLabel(LabelId label, View view) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<LabelId>> Labels(View view) const override;
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) override;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) override;
/// Remove all properties and return the values of the removed properties.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties() override;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const override;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const override;
Result<size_t> InDegree(View view) const override;
Result<size_t> OutDegree(View view) const override;
storage::Gid Gid() const noexcept override { return vertex_->gid; }
bool operator==(const VertexAccessor &other) const noexcept override {
const auto *otherVertex = dynamic_cast<const DiskVertexAccessor *>(&other);
if (otherVertex == nullptr) return false;
return vertex_ == otherVertex->vertex_ && transaction_ == otherVertex->transaction_;
}
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
private:
Vertex *vertex_;
Indices *indices_;
Constraints *constraints_;
};
} // namespace memgraph::storage

View File

@ -10,247 +10,16 @@
// licenses/APL.txt.
#include "storage/v2/edge_accessor.hpp"
#include <memory>
#include <tuple>
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/memory_tracker.hpp"
#include "storage/v2/inmemory/edge_accessor.hpp"
namespace memgraph::storage {
bool EdgeAccessor::IsVisible(const View view) const {
bool exists = true;
bool deleted = true;
// When edges don't have properties, their isolation level is still dictated by MVCC ->
// iterate over the deltas of the from_vertex_ and see which deltas can be applied on edges.
if (!config_.properties_on_edges) {
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(from_vertex_->lock);
// Initialize deleted by checking if out edges contain edge_
deleted = std::find_if(from_vertex_->out_edges.begin(), from_vertex_->out_edges.end(), [&](const auto &out_edge) {
return std::get<2>(out_edge) == edge_;
}) == from_vertex_->out_edges.end();
delta = from_vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::RECREATE_OBJECT:
case Delta::Action::DELETE_OBJECT:
break;
case Delta::Action::ADD_OUT_EDGE: { // relevant for the from_vertex_ -> we just deleted the edge
if (delta.vertex_edge.edge == edge_) {
deleted = false;
}
break;
}
case Delta::Action::REMOVE_OUT_EDGE: { // also relevant for the from_vertex_ -> we just added the edge
if (delta.vertex_edge.edge == edge_) {
exists = false;
}
break;
}
}
});
return exists && (for_deleted_ || !deleted);
}
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return exists && (for_deleted_ || !deleted);
std::unique_ptr<EdgeAccessor> EdgeAccessor::Create(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex,
Vertex *to_vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, bool for_deleted) {
return std::make_unique<InMemoryEdgeAccessor>(edge, edge_type, from_vertex, to_vertex, transaction, indices,
constraints, config, for_deleted);
}
VertexAccessor EdgeAccessor::FromVertex() const {
return VertexAccessor{from_vertex_, transaction_, indices_, constraints_, config_};
}
VertexAccessor EdgeAccessor::ToVertex() const {
return VertexAccessor{to_vertex_, transaction_, indices_, constraints_, config_};
}
Result<storage::PropertyValue> EdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto current_value = edge_.ptr->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, current_value);
edge_.ptr->properties.SetProperty(property, value);
return std::move(current_value);
}
Result<bool> EdgeAccessor::InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
if (!edge_.ptr->properties.InitProperties(properties)) return false;
for (const auto &[property, _] : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, PropertyValue());
}
return true;
}
Result<std::map<PropertyId, PropertyValue>> EdgeAccessor::ClearProperties() {
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto properties = edge_.ptr->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property.first, property.second);
}
edge_.ptr->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> EdgeAccessor::GetProperty(PropertyId property, View view) const {
if (!config_.properties_on_edges) return PropertyValue();
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
value = edge_.ptr->properties.GetProperty(property);
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
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::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 std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> EdgeAccessor::Properties(View view) const {
if (!config_.properties_on_edges) return std::map<PropertyId, PropertyValue>{};
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
properties = edge_.ptr->properties.Properties();
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
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::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 std::move(properties);
}
} // namespace memgraph::storage

View File

@ -11,91 +11,73 @@
#pragma once
#include <memory>
#include <optional>
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/view.hpp"
#include "storage/v2/vertex_accessor.hpp"
namespace memgraph::storage {
struct Vertex;
class VertexAccessor;
struct Indices;
struct Constraints;
struct Transaction;
class EdgeAccessor final {
class EdgeAccessor {
private:
friend class Storage;
public:
EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *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),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config),
for_deleted_(for_deleted) {}
EdgeAccessor(EdgeTypeId edge_type, Transaction *transaction, Config::Items config, bool for_deleted = false)
: edge_type_(edge_type), transaction_(transaction), config_(config), for_deleted_(for_deleted) {}
virtual ~EdgeAccessor() {}
static std::unique_ptr<EdgeAccessor> Create(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex,
Vertex *to_vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, bool for_deleted = false);
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const;
virtual bool IsVisible(View view) const = 0;
VertexAccessor FromVertex() const;
virtual std::unique_ptr<VertexAccessor> FromVertex() const = 0;
VertexAccessor ToVertex() const;
virtual std::unique_ptr<VertexAccessor> ToVertex() const = 0;
EdgeTypeId EdgeType() const { return edge_type_; }
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<storage::PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
virtual Result<storage::PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) = 0;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties);
virtual Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) = 0;
/// Remove all properties and return old values for each removed property.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
virtual Result<std::map<PropertyId, PropertyValue>> ClearProperties() = 0;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const;
virtual Result<PropertyValue> GetProperty(PropertyId property, View view) const = 0;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const;
virtual Result<std::map<PropertyId, PropertyValue>> Properties(View view) const = 0;
Gid Gid() const noexcept {
if (config_.properties_on_edges) {
return edge_.ptr->gid;
} else {
return edge_.gid;
}
}
virtual storage::Gid Gid() const noexcept = 0;
bool IsCycle() const { return from_vertex_ == to_vertex_; }
virtual bool IsCycle() const = 0;
bool operator==(const EdgeAccessor &other) const noexcept {
return edge_ == other.edge_ && transaction_ == other.transaction_;
}
virtual bool operator==(const EdgeAccessor &other) const noexcept = 0;
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
private:
EdgeRef edge_;
protected:
EdgeTypeId edge_type_;
Vertex *from_vertex_;
Vertex *to_vertex_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
// if the accessor was created for a deleted edge.

View File

@ -0,0 +1,258 @@
// 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 "storage/v2/inmemory/edge_accessor.hpp"
#include <memory>
#include <tuple>
#include "storage/v2/inmemory/vertex_accessor.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage {
bool InMemoryEdgeAccessor::IsVisible(const View view) const {
bool exists = true;
bool deleted = true;
// When edges don't have properties, their isolation level is still dictated by MVCC ->
// iterate over the deltas of the from_vertex_ and see which deltas can be applied on edges.
if (!config_.properties_on_edges) {
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(from_vertex_->lock);
// Initialize deleted by checking if out edges contain edge_
deleted = std::find_if(from_vertex_->out_edges.begin(), from_vertex_->out_edges.end(), [&](const auto &out_edge) {
return std::get<2>(out_edge) == edge_;
}) == from_vertex_->out_edges.end();
delta = from_vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::RECREATE_OBJECT:
case Delta::Action::DELETE_OBJECT:
break;
case Delta::Action::ADD_OUT_EDGE: { // relevant for the from_vertex_ -> we just deleted the edge
if (delta.vertex_edge.edge == edge_) {
deleted = false;
}
break;
}
case Delta::Action::REMOVE_OUT_EDGE: { // also relevant for the from_vertex_ -> we just added the edge
if (delta.vertex_edge.edge == edge_) {
exists = false;
}
break;
}
}
});
return exists && (for_deleted_ || !deleted);
}
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return exists && (for_deleted_ || !deleted);
}
std::unique_ptr<VertexAccessor> InMemoryEdgeAccessor::FromVertex() const {
return std::make_unique<InMemoryVertexAccessor>(from_vertex_, transaction_, indices_, constraints_, config_);
}
std::unique_ptr<VertexAccessor> InMemoryEdgeAccessor::ToVertex() const {
return std::make_unique<InMemoryVertexAccessor>(to_vertex_, transaction_, indices_, constraints_, config_);
}
Result<storage::PropertyValue> InMemoryEdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto current_value = edge_.ptr->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, current_value);
edge_.ptr->properties.SetProperty(property, value);
return std::move(current_value);
}
Result<bool> InMemoryEdgeAccessor::InitProperties(
const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
if (!edge_.ptr->properties.InitProperties(properties)) return false;
for (const auto &[property, _] : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, PropertyValue());
}
return true;
}
Result<std::map<PropertyId, PropertyValue>> InMemoryEdgeAccessor::ClearProperties() {
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto properties = edge_.ptr->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property.first, property.second);
}
edge_.ptr->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> InMemoryEdgeAccessor::GetProperty(PropertyId property, View view) const {
if (!config_.properties_on_edges) return PropertyValue();
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
value = edge_.ptr->properties.GetProperty(property);
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
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::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 std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> InMemoryEdgeAccessor::Properties(View view) const {
if (!config_.properties_on_edges) return std::map<PropertyId, PropertyValue>{};
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
properties = edge_.ptr->properties.Properties();
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
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::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 std::move(properties);
}
} // namespace memgraph::storage

View File

@ -0,0 +1,102 @@
// 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.
#pragma once
#include <optional>
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/view.hpp"
namespace memgraph::storage {
struct Vertex;
class VertexAccessor;
struct Indices;
struct Constraints;
class InMemoryEdgeAccessor final : public EdgeAccessor {
private:
friend class InMemoryStorage;
public:
InMemoryEdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex,
Transaction *transaction, Indices *indices, Constraints *constraints, Config::Items config,
bool for_deleted = false)
: EdgeAccessor(edge_type, transaction, config, for_deleted),
edge_(edge),
from_vertex_(from_vertex),
to_vertex_(to_vertex),
indices_(indices),
constraints_(constraints) {}
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const override;
std::unique_ptr<VertexAccessor> FromVertex() const override;
std::unique_ptr<VertexAccessor> ToVertex() const override;
EdgeTypeId EdgeType() const { return edge_type_; }
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<storage::PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) override;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) override;
/// Remove all properties and return old values for each removed property.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties() override;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const override;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const override;
storage::Gid Gid() const noexcept override {
if (config_.properties_on_edges) {
return edge_.ptr->gid;
} else {
return edge_.gid;
}
}
bool IsCycle() const override { return from_vertex_ == to_vertex_; }
bool operator==(const EdgeAccessor &other) const noexcept override {
const auto *otherEdge = dynamic_cast<const InMemoryEdgeAccessor *>(&other);
if (otherEdge == nullptr) return false;
return edge_ == otherEdge->edge_ && transaction_ == otherEdge->transaction_;
}
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
private:
EdgeRef edge_;
Vertex *from_vertex_;
Vertex *to_vertex_;
Indices *indices_;
Constraints *constraints_;
};
} // namespace memgraph::storage

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,526 @@
// 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.
#pragma once
#include <atomic>
#include <cstdint>
#include <filesystem>
#include <optional>
#include <shared_mutex>
#include <span>
#include <variant>
#include "io/network/endpoint.hpp"
#include "kvstore/kvstore.hpp"
#include "storage/v2/commit_log.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/rw_lock.hpp"
#include "utils/scheduler.hpp"
#include "utils/skip_list.hpp"
#include "utils/synchronized.hpp"
#include "utils/uuid.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/storage_error.hpp"
namespace memgraph::storage {
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
// The paper implements a fully serializable storage, in our implementation we
// only implement snapshot isolation for transactions.
class InMemoryStorage final : public Storage {
public:
/// @throw std::system_error
/// @throw std::bad_alloc
explicit InMemoryStorage(Config config = Config());
~InMemoryStorage();
class InMemoryAccessor final : public Storage::Accessor {
private:
friend class InMemoryStorage;
explicit InMemoryAccessor(InMemoryStorage *storage, IsolationLevel isolation_level);
public:
InMemoryAccessor(const InMemoryAccessor &) = delete;
InMemoryAccessor &operator=(const InMemoryAccessor &) = delete;
InMemoryAccessor &operator=(InMemoryAccessor &&other) = delete;
// NOTE: After the accessor is moved, all objects derived from it (accessors
// and iterators) are *invalid*. You have to get all derived objects again.
InMemoryAccessor(InMemoryAccessor &&other) noexcept;
~InMemoryAccessor() override;
/// @throw std::bad_alloc
std::unique_ptr<VertexAccessor> CreateVertex() override;
std::unique_ptr<VertexAccessor> FindVertex(Gid gid, View view) override;
VerticesIterable Vertices(View view) override {
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
&storage_->indices_, &storage_->constraints_,
storage_->config_.items));
}
VerticesIterable Vertices(LabelId label, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override;
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount() const override { return storage_->vertices_.size(); }
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label) const override {
return storage_->indices_.label_index.ApproximateVertexCount(label);
}
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
}
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
}
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const override {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
}
std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const override {
return storage_->indices_.label_property_index.GetIndexStats(label, property);
}
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() override {
return storage_->indices_.label_property_index.ClearIndexStats();
}
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(
const std::span<std::string> labels) override {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
std::for_each(labels.begin(), labels.end(), [this, &deleted_indexes](const auto &label_str) {
std::vector<std::pair<LabelId, PropertyId>> loc_results =
storage_->indices_.label_property_index.DeleteIndexStatsForLabel(NameToLabel(label_str));
deleted_indexes.insert(deleted_indexes.end(), std::make_move_iterator(loc_results.begin()),
std::make_move_iterator(loc_results.end()));
});
return deleted_indexes;
}
void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const IndexStats &stats) override {
storage_->indices_.label_property_index.SetIndexStats(label, property, stats);
}
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::unique_ptr<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) override;
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>>>
DetachDeleteVertex(VertexAccessor *vertex) override;
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> CreateEdge(VertexAccessor *from, VertexAccessor *to,
EdgeTypeId edge_type) override;
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) override;
const std::string &LabelToName(LabelId label) const override;
const std::string &PropertyToName(PropertyId property) const override;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const override;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name) override;
bool LabelIndexExists(LabelId label) const override { return storage_->indices_.label_index.IndexExists(label); }
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const override {
return storage_->indices_.label_property_index.IndexExists(label, property);
}
IndicesInfo ListAllIndices() const override {
return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
}
ConstraintsInfo ListAllConstraints() const override {
return {ListExistenceConstraints(storage_->constraints_),
storage_->constraints_.unique_constraints.ListConstraints()};
}
void AdvanceCommand() override;
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
/// @throw std::bad_alloc
void Abort() override;
void FinalizeTransaction() override;
std::optional<uint64_t> GetTransactionId() const override;
private:
/// @throw std::bad_alloc
std::unique_ptr<VertexAccessor> CreateVertex(storage::Gid gid);
/// @throw std::bad_alloc
Result<std::unique_ptr<EdgeAccessor>> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type,
storage::Gid gid);
InMemoryStorage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
Config::Items config_;
};
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override {
return std::unique_ptr<InMemoryAccessor>(
new InMemoryAccessor{this, override_isolation_level.value_or(isolation_level_)});
}
const std::string &LabelToName(LabelId label) const override;
const std::string &PropertyToName(PropertyId property) const override;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const override;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name) override;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
IndicesInfo ListAllIndices() const override;
/// Returns void if the existence constraint has been created.
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there is already a vertex existing that would break this new constraint.
/// * `ConstraintDefinitionError`: the constraint already exists.
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing existence constraint.
/// Returns void if the existence constraint has been dropped.
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintDefinitionError`: the constraint did not exists.
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an unique constraint.
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there are already vertices violating the constraint.
/// Returns `UniqueConstraints::CreationStatus` otherwise. Value can be:
/// * `SUCCESS` if the constraint was successfully created,
/// * `ALREADY_EXISTS` if the constraint already existed,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
/// @throw std::bad_alloc
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
/// Removes an existing unique constraint.
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// Returns `UniqueConstraints::DeletionStatus` otherwise. Value can be:
/// * `SUCCESS` if constraint was successfully removed,
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
ConstraintsInfo ListAllConstraints() const override;
StorageInfo GetInfo() const override;
bool LockPath() override;
bool UnlockPath() override;
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) override;
bool SetMainReplicationRole() override;
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config) override;
/// @pre The instance should have a MAIN role
bool UnregisterReplica(const std::string &name) override;
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name) override;
ReplicationRole GetReplicationRole() const override;
std::vector<ReplicaInfo> ReplicasInfo() override;
void FreeMemory() override;
void SetIsolationLevel(IsolationLevel isolation_level) override;
utils::BasicResult<CreateSnapshotError> CreateSnapshot() override;
private:
Transaction CreateTransaction(IsolationLevel isolation_level);
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage();
bool InitializeWalFile();
void FinalizeWalFile();
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
void RestoreReplicas();
bool ShouldStoreAndRestoreReplicas() const;
// Main storage lock.
//
// Accessors take a shared lock when starting, so it is possible to block
// creation of new accessors by taking a unique lock. This is used when doing
// operations on storage that affect the global state, for example index
// creation.
mutable utils::RWLock main_lock_{utils::RWLock::Priority::WRITE};
// Main object storage
utils::SkipList<storage::Vertex> vertices_;
utils::SkipList<storage::Edge> edges_;
std::atomic<uint64_t> vertex_id_{0};
std::atomic<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.
std::atomic<uint64_t> edge_count_{0};
NameIdMapper name_id_mapper_;
Constraints constraints_;
Indices indices_;
// Transaction engine
utils::SpinLock engine_lock_;
uint64_t timestamp_{kTimestampInitialId};
uint64_t transaction_id_{kTransactionInitialId};
// TODO: This isn't really a commit log, it doesn't even care if a
// transaction commited or aborted. We could probably combine this with
// `timestamp_` in a sensible unit, something like TransactionClock or
// whatever.
std::optional<CommitLog> commit_log_;
utils::Synchronized<std::list<Transaction>, utils::SpinLock> committed_transactions_;
IsolationLevel isolation_level_;
Config config_;
utils::Scheduler gc_runner_;
std::mutex gc_lock_;
// Undo buffers that were unlinked and now are waiting to be freed.
utils::Synchronized<std::list<std::pair<uint64_t, std::list<Delta>>>, utils::SpinLock> garbage_undo_buffers_;
// Vertices that are logically deleted but still have to be removed from
// indices before removing them from the main storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_vertices_;
// Vertices that are logically deleted and removed from indices and now wait
// to be removed from the main storage.
std::list<std::pair<uint64_t, Gid>> garbage_vertices_;
// Edges that are logically deleted and wait to be removed from the main
// storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_edges_;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path wal_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
std::unique_ptr<kvstore::KVStore> storage_;
utils::Scheduler snapshot_runner_;
utils::SpinLock snapshot_lock_;
// UUID used to distinguish snapshots and to link snapshots to WALs
std::string uuid_;
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_;
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
utils::FileRetainer file_retainer_;
// Global locker that is used for clients file locking
utils::FileRetainer::FileLocker global_locker_;
// Last commited timestamp
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
class ReplicationServer;
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
class ReplicationClient;
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<ReplicationRole> replication_role_{ReplicationRole::MAIN};
};
} // namespace memgraph::storage

View File

@ -0,0 +1,604 @@
// 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 "storage/v2/inmemory/vertex_accessor.hpp"
#include <memory>
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/inmemory/edge_accessor.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage {
namespace detail {
namespace {
std::pair<bool, bool> IsVisible(Vertex *vertex, Transaction *transaction, View view) {
bool exists = true;
bool deleted = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex->lock);
deleted = vertex->deleted;
delta = vertex->delta;
}
ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return {exists, deleted};
}
} // namespace
} // namespace detail
std::unique_ptr<InMemoryVertexAccessor> InMemoryVertexAccessor::Create(Vertex *vertex, Transaction *transaction,
Indices *indices, Constraints *constraints,
Config::Items config, View view) {
if (const auto [exists, deleted] = detail::IsVisible(vertex, transaction, view); !exists || deleted) {
return {};
}
return std::make_unique<InMemoryVertexAccessor>(vertex, transaction, indices, constraints, config);
}
bool InMemoryVertexAccessor::IsVisible(View view) const {
const auto [exists, deleted] = detail::IsVisible(vertex_, transaction_, view);
return exists && (for_deleted_ || !deleted);
}
Result<bool> InMemoryVertexAccessor::AddLabel(LabelId label) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
if (std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::RemoveLabelTag(), label);
vertex_->labels.push_back(label);
UpdateOnAddLabel(indices_, label, vertex_, *transaction_);
return true;
}
Result<bool> InMemoryVertexAccessor::RemoveLabel(LabelId label) {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto it = std::find(vertex_->labels.begin(), vertex_->labels.end(), label);
if (it == vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::AddLabelTag(), label);
std::swap(*it, *vertex_->labels.rbegin());
vertex_->labels.pop_back();
return true;
}
Result<bool> InMemoryVertexAccessor::HasLabel(LabelId label, View view) const {
bool exists = true;
bool deleted = false;
bool has_label = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
has_label = std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &has_label, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
}
case Delta::Action::ADD_LABEL: {
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
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 has_label;
}
Result<std::vector<LabelId>> InMemoryVertexAccessor::Labels(View view) const {
bool exists = true;
bool deleted = false;
std::vector<LabelId> labels;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
labels = vertex_->labels;
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &labels](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
// Remove the label because we don't see the addition.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it != labels.end(), "Invalid database state!");
std::swap(*it, *labels.rbegin());
labels.pop_back();
break;
}
case Delta::Action::ADD_LABEL: {
// Add the label because we don't see the removal.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it == labels.end(), "Invalid database state!");
labels.push_back(delta.label);
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
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 std::move(labels);
}
Result<PropertyValue> InMemoryVertexAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto current_value = vertex_->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
vertex_->properties.SetProperty(property, value);
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
return std::move(current_value);
}
Result<bool> InMemoryVertexAccessor::InitProperties(
const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
if (!vertex_->properties.InitProperties(properties)) return false;
for (const auto &[property, value] : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, PropertyValue());
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
}
return true;
}
Result<std::map<PropertyId, PropertyValue>> InMemoryVertexAccessor::ClearProperties() {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto properties = vertex_->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property.first, property.second);
UpdateOnSetProperty(indices_, property.first, PropertyValue(), vertex_, *transaction_);
}
vertex_->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> InMemoryVertexAccessor::GetProperty(PropertyId property, View view) const {
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
value = vertex_->properties.GetProperty(property);
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
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::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 std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> InMemoryVertexAccessor::Properties(View view) const {
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
properties = vertex_->properties.Properties();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
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::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 std::move(properties);
}
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InMemoryVertexAccessor::InEdges(
View view, const std::vector<EdgeTypeId> &edge_types, const VertexAccessor *destination) const {
auto *destVA = dynamic_cast<const InMemoryVertexAccessor *>(destination);
MG_ASSERT(!destination || destVA, "Target VertexAccessor must be from the same storage as the storage accessor!");
MG_ASSERT(!destVA || destVA->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destVA) {
in_edges = vertex_->in_edges;
} else {
for (const auto &item : vertex_->in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
if (destVA && from_vertex != destVA->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
in_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destVA](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE: {
if (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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 (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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());
in_edges.pop_back();
break;
}
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<std::unique_ptr<EdgeAccessor>> ret;
ret.reserve(in_edges.size());
for (const auto &item : in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
ret.emplace_back(std::make_unique<InMemoryEdgeAccessor>(edge, edge_type, from_vertex, vertex_, transaction_,
indices_, constraints_, config_));
}
return std::move(ret);
}
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InMemoryVertexAccessor::OutEdges(
View view, const std::vector<EdgeTypeId> &edge_types, const VertexAccessor *destination) const {
auto *destVA = dynamic_cast<const InMemoryVertexAccessor *>(destination);
MG_ASSERT(!destination || destVA, "Target VertexAccessor must be from the same storage as the storage accessor!");
MG_ASSERT(!destVA || destVA->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destVA) {
out_edges = vertex_->out_edges;
} else {
for (const auto &item : vertex_->out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
if (destVA && to_vertex != destVA->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
out_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destVA](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE: {
if (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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 (destVA && delta.vertex_edge.vertex != destVA->vertex_) 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};
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());
out_edges.pop_back();
break;
}
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<std::unique_ptr<EdgeAccessor>> ret;
ret.reserve(out_edges.size());
for (const auto &item : out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
ret.emplace_back(std::make_unique<InMemoryEdgeAccessor>(edge, edge_type, vertex_, to_vertex, transaction_, indices_,
constraints_, config_));
}
return std::move(ret);
}
Result<size_t> InMemoryVertexAccessor::InDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->in_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE:
++degree;
break;
case Delta::Action::REMOVE_IN_EDGE:
--degree;
break;
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
Result<size_t> InMemoryVertexAccessor::OutDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->out_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE:
++degree;
break;
case Delta::Action::REMOVE_OUT_EDGE:
--degree;
break;
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,117 @@
// 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.
#pragma once
#include <memory>
#include "storage/v2/vertex.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "storage/v2/view.hpp"
namespace memgraph::storage {
class EdgeAccessor;
class Storage;
struct Indices;
struct Constraints;
class InMemoryVertexAccessor final : public VertexAccessor {
private:
friend class InMemoryStorage;
public:
InMemoryVertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config, bool for_deleted = false)
: VertexAccessor(transaction, config, for_deleted),
vertex_(vertex),
indices_(indices),
constraints_(constraints) {}
static std::unique_ptr<InMemoryVertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const override;
/// Add a label and return `true` if insertion took place.
/// `false` is returned if the label already existed.
/// @throw std::bad_alloc
Result<bool> AddLabel(LabelId label) override;
/// Remove a label and return `true` if deletion took place.
/// `false` is returned if the vertex did not have a label already.
/// @throw std::bad_alloc
Result<bool> RemoveLabel(LabelId label) override;
Result<bool> HasLabel(LabelId label, View view) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<LabelId>> Labels(View view) const override;
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) override;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) override;
/// Remove all properties and return the values of the removed properties.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties() override;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const override;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const override;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const override;
Result<size_t> InDegree(View view) const override;
Result<size_t> OutDegree(View view) const override;
storage::Gid Gid() const noexcept override { return vertex_->gid; }
bool operator==(const VertexAccessor &other) const noexcept override {
const auto *otherVertex = dynamic_cast<const InMemoryVertexAccessor *>(&other);
if (otherVertex == nullptr) return false;
return vertex_ == otherVertex->vertex_ && transaction_ == otherVertex->transaction_;
}
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
private:
Vertex *vertex_;
Indices *indices_;
Constraints *constraints_;
};
} // namespace memgraph::storage

View File

@ -11,9 +11,12 @@
#include "storage/v2/property_store.hpp"
#include <cstdint>
#include <cstring>
#include <iterator>
#include <limits>
#include <optional>
#include <sstream>
#include <tuple>
#include <type_traits>
#include <utility>
@ -1219,4 +1222,37 @@ bool PropertyStore::ClearProperties() {
return true;
}
std::string PropertyStore::StringBuffer() {
uint64_t size = 0;
uint8_t *data = nullptr;
std::tie(size, data) = GetSizeData(buffer_);
if (size % 8 != 0) { // We are storing the data in the local buffer.
size = sizeof(buffer_) - 1;
data = &buffer_[1];
}
std::string arr(size, ' ');
for (uint i = 0; i < size; ++i) {
arr[i] = static_cast<char>(data[i]);
}
return arr;
}
void PropertyStore::SetBuffer(const std::string_view buffer) {
uint64_t size = 0;
uint8_t *data = nullptr;
if (buffer.size() == sizeof(buffer_) - 1) { // use local buffer
buffer_[0] = kUseLocalBuffer;
size = buffer.size() - 1;
data = &buffer_[1];
} else {
size = buffer.size();
data = new uint8_t[size];
SetSizeData(buffer_, size, data);
}
for (uint i = 0; i < size; ++i) {
data[i] = static_cast<uint8_t>(buffer[i]);
}
}
} // namespace memgraph::storage

View File

@ -71,6 +71,12 @@ class PropertyStore {
/// @throw std::bad_alloc
bool ClearProperties();
/// Return property buffer as a string
std::string StringBuffer();
/// Sets buffer
void SetBuffer(std::string_view buffer);
private:
uint8_t buffer_[sizeof(uint64_t) + sizeof(uint8_t *)];
};

File diff suppressed because it is too large Load Diff

View File

@ -11,49 +11,27 @@
#pragma once
#include <atomic>
#include <cstdint>
#include <filesystem>
#include <optional>
#include <shared_mutex>
#include <set>
#include <span>
#include <variant>
#include "io/network/endpoint.hpp"
#include "kvstore/kvstore.hpp"
#include "storage/v2/commit_log.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/storage_error.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/rw_lock.hpp"
#include "utils/scheduler.hpp"
#include "utils/skip_list.hpp"
#include "utils/synchronized.hpp"
#include "utils/uuid.hpp"
#include "storage/v2/view.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/storage_error.hpp"
namespace memgraph::storage {
struct Transaction;
class EdgeAccessor;
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
// The paper implements a fully serializable storage, in our implementation we
@ -70,7 +48,7 @@ class AllVerticesIterable final {
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
std::optional<VertexAccessor> vertex_;
std::unique_ptr<VertexAccessor> vertex_;
public:
class Iterator final {
@ -80,7 +58,7 @@ class AllVerticesIterable final {
public:
Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it);
VertexAccessor operator*() const;
VertexAccessor *operator*() const;
Iterator &operator++();
@ -152,7 +130,7 @@ class VerticesIterable final {
~Iterator();
VertexAccessor operator*() const;
VertexAccessor *operator*() const;
Iterator &operator++();
@ -186,23 +164,11 @@ struct StorageInfo {
uint64_t disk_usage;
};
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
class Storage final {
class Storage {
public:
/// @throw std::system_error
/// @throw std::bad_alloc
explicit Storage(Config config = Config());
~Storage();
class Accessor final {
private:
friend class Storage;
explicit Accessor(Storage *storage, IsolationLevel isolation_level);
class Accessor {
public:
Accessor() {}
Accessor(const Accessor &) = delete;
Accessor &operator=(const Accessor &) = delete;
Accessor &operator=(Accessor &&other) = delete;
@ -211,130 +177,100 @@ class Storage final {
// and iterators) are *invalid*. You have to get all derived objects again.
Accessor(Accessor &&other) noexcept;
~Accessor();
virtual ~Accessor() {}
/// @throw std::bad_alloc
VertexAccessor CreateVertex();
virtual std::unique_ptr<VertexAccessor> CreateVertex() = 0;
std::optional<VertexAccessor> FindVertex(Gid gid, View view);
virtual std::unique_ptr<VertexAccessor> FindVertex(Gid gid, View view) = 0;
VerticesIterable Vertices(View view) {
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
&storage_->indices_, &storage_->constraints_,
storage_->config_.items));
}
virtual VerticesIterable Vertices(View view) = 0;
VerticesIterable Vertices(LabelId label, View view);
virtual VerticesIterable Vertices(LabelId label, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) = 0;
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount() const { return storage_->vertices_.size(); }
virtual int64_t ApproximateVertexCount() const = 0;
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label) const {
return storage_->indices_.label_index.ApproximateVertexCount(label);
}
virtual int64_t ApproximateVertexCount(LabelId label) const = 0;
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
}
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property) const = 0;
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
}
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const = 0;
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
}
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const {
return storage_->indices_.label_property_index.GetIndexStats(label, property);
}
virtual std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const = 0;
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() {
return storage_->indices_.label_property_index.ClearIndexStats();
}
virtual std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() = 0;
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(const std::span<std::string> labels) {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
std::for_each(labels.begin(), labels.end(), [this, &deleted_indexes](const auto &label_str) {
std::vector<std::pair<LabelId, PropertyId>> loc_results =
storage_->indices_.label_property_index.DeleteIndexStatsForLabel(NameToLabel(label_str));
deleted_indexes.insert(deleted_indexes.end(), std::make_move_iterator(loc_results.begin()),
std::make_move_iterator(loc_results.end()));
});
return deleted_indexes;
}
virtual std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(
const std::span<std::string> labels) = 0;
void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property, const IndexStats &stats) {
storage_->indices_.label_property_index.SetIndexStats(label, property, stats);
}
virtual void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const IndexStats &stats) = 0;
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
virtual Result<std::unique_ptr<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) = 0;
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex);
virtual Result<
std::optional<std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>>>
DetachDeleteVertex(VertexAccessor *vertex) = 0;
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
virtual Result<std::unique_ptr<EdgeAccessor>> CreateEdge(VertexAccessor *from, VertexAccessor *to,
EdgeTypeId edge_type) = 0;
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
virtual Result<std::unique_ptr<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) = 0;
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
virtual const std::string &LabelToName(LabelId label) const = 0;
virtual const std::string &PropertyToName(PropertyId property) const = 0;
virtual const std::string &EdgeTypeToName(EdgeTypeId edge_type) const = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name);
virtual LabelId NameToLabel(std::string_view name) = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name);
virtual PropertyId NameToProperty(std::string_view name) = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name);
virtual EdgeTypeId NameToEdgeType(std::string_view name) = 0;
bool LabelIndexExists(LabelId label) const { return storage_->indices_.label_index.IndexExists(label); }
virtual bool LabelIndexExists(LabelId label) const = 0;
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const {
return storage_->indices_.label_property_index.IndexExists(label, property);
}
virtual bool LabelPropertyIndexExists(LabelId label, PropertyId property) const = 0;
IndicesInfo ListAllIndices() const {
return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
}
virtual IndicesInfo ListAllIndices() const = 0;
ConstraintsInfo ListAllConstraints() const {
return {ListExistenceConstraints(storage_->constraints_),
storage_->constraints_.unique_constraints.ListConstraints()};
}
virtual ConstraintsInfo ListAllConstraints() const = 0;
void AdvanceCommand();
virtual void AdvanceCommand() = 0;
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
@ -342,83 +278,84 @@ class Storage final {
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) = 0;
/// @throw std::bad_alloc
void Abort();
virtual void Abort() = 0;
void FinalizeTransaction();
virtual void FinalizeTransaction() = 0;
std::optional<uint64_t> GetTransactionId() const;
private:
/// @throw std::bad_alloc
VertexAccessor CreateVertex(storage::Gid gid);
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, storage::Gid gid);
Storage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
Config::Items config_;
virtual std::optional<uint64_t> GetTransactionId() const = 0;
};
Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
return Accessor{this, override_isolation_level.value_or(isolation_level_)};
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> Access() { return Access(std::optional<IsolationLevel>{}); }
virtual const std::string &LabelToName(LabelId label) const = 0;
virtual const std::string &PropertyToName(PropertyId property) const = 0;
virtual const std::string &EdgeTypeToName(EdgeTypeId edge_type) const = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
virtual LabelId NameToLabel(std::string_view name) = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
virtual PropertyId NameToProperty(std::string_view name) = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
virtual EdgeTypeId NameToEdgeType(std::string_view name) = 0;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) {
return CreateIndex(label, std::optional<uint64_t>{});
}
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name);
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name);
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name);
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) {
return CreateIndex(label, property, std::optional<uint64_t>{});
}
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) {
return DropIndex(label, std::optional<uint64_t>{});
}
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
IndicesInfo ListAllIndices() const;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) {
return DropIndex(label, property, std::optional<uint64_t>{});
}
virtual IndicesInfo ListAllIndices() const;
/// Returns void if the existence constraint has been created.
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
@ -427,16 +364,26 @@ class Storage final {
/// * `ConstraintDefinitionError`: the constraint already exists.
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(LabelId label,
PropertyId property) {
return CreateExistenceConstraint(label, property, std::optional<uint64_t>{});
}
/// Drop an existing existence constraint.
/// Returns void if the existence constraint has been dropped.
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintDefinitionError`: the constraint did not exists.
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(LabelId label,
PropertyId property) {
return DropExistenceConstraint(label, property, std::optional<uint64_t>{});
}
/// Create an unique constraint.
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
@ -448,8 +395,14 @@ class Storage final {
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
/// @throw std::bad_alloc
virtual utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {});
LabelId label, const std::set<PropertyId> &properties) {
return CreateUniqueConstraint(label, properties, std::optional<uint64_t>{});
}
/// Removes an existing unique constraint.
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
@ -459,19 +412,29 @@ class Storage final {
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
virtual utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {});
LabelId label, const std::set<PropertyId> &properties) {
return DropUniqueConstraint(label, properties, std::optional<uint64_t>{});
}
ConstraintsInfo ListAllConstraints() const;
virtual ConstraintsInfo ListAllConstraints() const = 0;
StorageInfo GetInfo() const;
virtual StorageInfo GetInfo() const = 0;
bool LockPath();
bool UnlockPath();
virtual bool LockPath() = 0;
virtual bool UnlockPath() = 0;
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
virtual bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) = 0;
bool SetMainReplicationRole();
bool SetReplicaRole(io::network::Endpoint endpoint) {
return SetReplicaRole(endpoint, replication::ReplicationServerConfig{});
}
virtual bool SetMainReplicationRole() = 0;
enum class RegisterReplicaError : uint8_t {
NAME_EXISTS,
@ -482,15 +445,22 @@ class Storage final {
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
virtual utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config = {});
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config) = 0;
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode) {
return RegisterReplica(name, endpoint, replication_mode, registration_mode, replication::ReplicationClientConfig{});
}
/// @pre The instance should have a MAIN role
bool UnregisterReplica(const std::string &name);
virtual bool UnregisterReplica(const std::string &name) = 0;
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
virtual std::optional<replication::ReplicaState> GetReplicaState(std::string_view name) = 0;
ReplicationRole GetReplicationRole() const;
virtual ReplicationRole GetReplicationRole() const = 0;
struct TimestampInfo {
uint64_t current_timestamp_of_replica;
@ -505,167 +475,15 @@ class Storage final {
TimestampInfo timestamp_info;
};
std::vector<ReplicaInfo> ReplicasInfo();
virtual std::vector<ReplicaInfo> ReplicasInfo() = 0;
void FreeMemory();
virtual void FreeMemory() = 0;
void SetIsolationLevel(IsolationLevel isolation_level);
virtual void SetIsolationLevel(IsolationLevel isolation_level) = 0;
enum class CreateSnapshotError : uint8_t { DisabledForReplica };
utils::BasicResult<CreateSnapshotError> CreateSnapshot();
private:
Transaction CreateTransaction(IsolationLevel isolation_level);
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage();
bool InitializeWalFile();
void FinalizeWalFile();
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
void RestoreReplicas();
bool ShouldStoreAndRestoreReplicas() const;
// Main storage lock.
//
// Accessors take a shared lock when starting, so it is possible to block
// creation of new accessors by taking a unique lock. This is used when doing
// operations on storage that affect the global state, for example index
// creation.
mutable utils::RWLock main_lock_{utils::RWLock::Priority::WRITE};
// Main object storage
utils::SkipList<storage::Vertex> vertices_;
utils::SkipList<storage::Edge> edges_;
std::atomic<uint64_t> vertex_id_{0};
std::atomic<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.
std::atomic<uint64_t> edge_count_{0};
NameIdMapper name_id_mapper_;
Constraints constraints_;
Indices indices_;
// Transaction engine
utils::SpinLock engine_lock_;
uint64_t timestamp_{kTimestampInitialId};
uint64_t transaction_id_{kTransactionInitialId};
// TODO: This isn't really a commit log, it doesn't even care if a
// transaction commited or aborted. We could probably combine this with
// `timestamp_` in a sensible unit, something like TransactionClock or
// whatever.
std::optional<CommitLog> commit_log_;
utils::Synchronized<std::list<Transaction>, utils::SpinLock> committed_transactions_;
IsolationLevel isolation_level_;
Config config_;
utils::Scheduler gc_runner_;
std::mutex gc_lock_;
// Undo buffers that were unlinked and now are waiting to be freed.
utils::Synchronized<std::list<std::pair<uint64_t, std::list<Delta>>>, utils::SpinLock> garbage_undo_buffers_;
// Vertices that are logically deleted but still have to be removed from
// indices before removing them from the main storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_vertices_;
// Vertices that are logically deleted and removed from indices and now wait
// to be removed from the main storage.
std::list<std::pair<uint64_t, Gid>> garbage_vertices_;
// Edges that are logically deleted and wait to be removed from the main
// storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_edges_;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path wal_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
std::unique_ptr<kvstore::KVStore> storage_;
utils::Scheduler snapshot_runner_;
utils::SpinLock snapshot_lock_;
// UUID used to distinguish snapshots and to link snapshots to WALs
std::string uuid_;
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_;
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
utils::FileRetainer file_retainer_;
// Global locker that is used for clients file locking
utils::FileRetainer::FileLocker global_locker_;
// Last commited timestamp
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
class ReplicationServer;
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
class ReplicationClient;
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<ReplicationRole> replication_role_{ReplicationRole::MAIN};
virtual utils::BasicResult<CreateSnapshotError> CreateSnapshot() = 0;
};
} // namespace memgraph::storage

View File

@ -11,585 +11,32 @@
#include "storage/v2/vertex_accessor.hpp"
#include <memory>
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
#include "storage/v2/inmemory/edge_accessor.hpp"
#include "storage/v2/inmemory/vertex_accessor.hpp"
namespace memgraph::storage {
namespace detail {
namespace {
std::pair<bool, bool> IsVisible(Vertex *vertex, Transaction *transaction, View view) {
bool exists = true;
bool deleted = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex->lock);
deleted = vertex->deleted;
delta = vertex->delta;
}
ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) {
switch (delta.action) {
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return {exists, deleted};
}
} // namespace
} // namespace detail
std::optional<VertexAccessor> VertexAccessor::Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view) {
if (const auto [exists, deleted] = detail::IsVisible(vertex, transaction, view); !exists || deleted) {
return std::nullopt;
}
return VertexAccessor{vertex, transaction, indices, constraints, config};
std::unique_ptr<VertexAccessor> VertexAccessor::Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view) {
return InMemoryVertexAccessor::Create(vertex, transaction, indices, constraints, config, view);
}
bool VertexAccessor::IsVisible(View view) const {
const auto [exists, deleted] = detail::IsVisible(vertex_, transaction_, view);
return exists && (for_deleted_ || !deleted);
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::InEdges(
View view, const std::vector<EdgeTypeId> &edge_types) const {
return InEdges(view, edge_types, nullptr);
}
Result<bool> VertexAccessor::AddLabel(LabelId label) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
if (std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::RemoveLabelTag(), label);
vertex_->labels.push_back(label);
UpdateOnAddLabel(indices_, label, vertex_, *transaction_);
return true;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::InEdges(View view) const {
return InEdges(view, {}, nullptr);
}
Result<bool> VertexAccessor::RemoveLabel(LabelId label) {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto it = std::find(vertex_->labels.begin(), vertex_->labels.end(), label);
if (it == vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::AddLabelTag(), label);
std::swap(*it, *vertex_->labels.rbegin());
vertex_->labels.pop_back();
return true;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::OutEdges(
View view, const std::vector<EdgeTypeId> &edge_types) const {
return OutEdges(view, edge_types, nullptr);
}
Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
bool exists = true;
bool deleted = false;
bool has_label = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
has_label = std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &has_label, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
}
case Delta::Action::ADD_LABEL: {
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
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 has_label;
}
Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
bool exists = true;
bool deleted = false;
std::vector<LabelId> labels;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
labels = vertex_->labels;
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &labels](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
// Remove the label because we don't see the addition.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it != labels.end(), "Invalid database state!");
std::swap(*it, *labels.rbegin());
labels.pop_back();
break;
}
case Delta::Action::ADD_LABEL: {
// Add the label because we don't see the removal.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it == labels.end(), "Invalid database state!");
labels.push_back(delta.label);
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
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 std::move(labels);
}
Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto current_value = vertex_->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
vertex_->properties.SetProperty(property, value);
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
return std::move(current_value);
}
Result<bool> VertexAccessor::InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
if (!vertex_->properties.InitProperties(properties)) return false;
for (const auto &[property, value] : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, PropertyValue());
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
}
return true;
}
Result<std::map<PropertyId, PropertyValue>> VertexAccessor::ClearProperties() {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto properties = vertex_->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property.first, property.second);
UpdateOnSetProperty(indices_, property.first, PropertyValue(), vertex_, *transaction_);
}
vertex_->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> VertexAccessor::GetProperty(PropertyId property, View view) const {
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
value = vertex_->properties.GetProperty(property);
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
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::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 std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view) const {
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
properties = vertex_->properties.Properties();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
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::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 std::move(properties);
}
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!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destination) {
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 (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
in_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destination](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) 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};
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 (!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};
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());
in_edges.pop_back();
break;
}
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<EdgeAccessor> ret;
ret.reserve(in_edges.size());
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_);
}
return std::move(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!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destination) {
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 (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
out_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destination](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) 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};
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 (!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};
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());
out_edges.pop_back();
break;
}
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<EdgeAccessor> ret;
ret.reserve(out_edges.size());
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_);
}
return std::move(ret);
}
Result<size_t> VertexAccessor::InDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->in_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE:
++degree;
break;
case Delta::Action::REMOVE_IN_EDGE:
--degree;
break;
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_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
Result<size_t> VertexAccessor::OutDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->out_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE:
++degree;
break;
case Delta::Action::REMOVE_OUT_EDGE:
--degree;
break;
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::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::OutEdges(View view) const {
return OutEdges(view, {}, nullptr);
}
} // namespace memgraph::storage

View File

@ -12,10 +12,14 @@
#pragma once
#include <optional>
#include <string>
#include <string_view>
#include "storage/v2/id_types.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/view.hpp"
@ -23,94 +27,98 @@
namespace memgraph::storage {
class EdgeAccessor;
class Storage;
struct Indices;
struct Constraints;
class VertexAccessor final {
class VertexAccessor {
private:
friend class Storage;
public:
VertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config, bool for_deleted = false)
: vertex_(vertex),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config),
for_deleted_(for_deleted) {}
VertexAccessor(Transaction *transaction, Config::Items config, bool for_deleted = false)
: transaction_(transaction), config_(config), for_deleted_(for_deleted) {}
static std::optional<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
virtual ~VertexAccessor() {}
static std::unique_ptr<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const;
virtual bool IsVisible(View view) const = 0;
/// Add a label and return `true` if insertion took place.
/// `false` is returned if the label already existed.
/// @throw std::bad_alloc
Result<bool> AddLabel(LabelId label);
virtual Result<bool> AddLabel(LabelId label) = 0;
/// Remove a label and return `true` if deletion took place.
/// `false` is returned if the vertex did not have a label already.
/// @throw std::bad_alloc
Result<bool> RemoveLabel(LabelId label);
virtual Result<bool> RemoveLabel(LabelId label) = 0;
Result<bool> HasLabel(LabelId label, View view) const;
virtual Result<bool> HasLabel(LabelId label, View view) const = 0;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<LabelId>> Labels(View view) const;
virtual Result<std::vector<LabelId>> Labels(View view) const = 0;
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
virtual Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value) = 0;
/// Set property values only if property store is empty. Returns `true` if successully set all values,
/// `false` otherwise.
/// @throw std::bad_alloc
Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties);
virtual Result<bool> InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) = 0;
/// Remove all properties and return the values of the removed properties.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
virtual Result<std::map<PropertyId, PropertyValue>> ClearProperties() = 0;
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const;
virtual Result<PropertyValue> GetProperty(PropertyId property, View view) const = 0;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const;
virtual Result<std::map<PropertyId, PropertyValue>> Properties(View view) const = 0;
std::string PropertyStore() const;
void SetPropertyStore(std::string_view buffer) const;
/// @throw std::bad_alloc
/// @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;
virtual Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view,
const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const = 0;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view,
const std::vector<EdgeTypeId> &edge_types) const;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view) 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;
virtual Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view,
const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const = 0;
Result<size_t> InDegree(View view) const;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view,
const std::vector<EdgeTypeId> &edge_types) const;
Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view) const;
Result<size_t> OutDegree(View view) const;
virtual Result<size_t> InDegree(View view) const = 0;
Gid Gid() const noexcept { return vertex_->gid; }
virtual Result<size_t> OutDegree(View view) const = 0;
bool operator==(const VertexAccessor &other) const noexcept {
return vertex_ == other.vertex_ && transaction_ == other.transaction_;
}
virtual Gid Gid() const noexcept = 0;
virtual bool operator==(const VertexAccessor &other) const noexcept = 0;
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
private:
Vertex *vertex_;
protected:
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
// if the accessor was created for a deleted vertex.

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// 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
@ -21,6 +21,7 @@
class ExpansionBenchFixture : public benchmark::Fixture {
protected:
std::optional<memgraph::storage::Storage> db;
memgraph::storage::rocks::RocksDBStorage disk_db_;
std::optional<memgraph::query::InterpreterContext> interpreter_context;
std::optional<memgraph::query::Interpreter> interpreter;
std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "expansion-benchmark"};
@ -47,7 +48,7 @@ class ExpansionBenchFixture : public benchmark::Fixture {
MG_ASSERT(!db->CreateIndex(label).HasError());
interpreter_context.emplace(&*db, memgraph::query::InterpreterConfig{}, data_directory);
interpreter_context.emplace(&*db, &disk_db_, memgraph::query::InterpreterConfig{}, data_directory);
interpreter.emplace(&*interpreter_context);
}

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// 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
@ -27,11 +27,13 @@ int main(int argc, char *argv[]) {
}
memgraph::storage::Storage db;
memgraph::storage::rocks::RocksDBStorage disk_db;
auto data_directory = std::filesystem::temp_directory_path() / "single_query_test";
memgraph::utils::OnScopeExit([&data_directory] { std::filesystem::remove_all(data_directory); });
memgraph::license::global_license_checker.EnableTesting();
memgraph::query::InterpreterContext interpreter_context{&db, memgraph::query::InterpreterConfig{}, data_directory};
memgraph::query::InterpreterContext interpreter_context{&db, &disk_db, memgraph::query::InterpreterConfig{},
data_directory};
memgraph::query::Interpreter interpreter{&interpreter_context};
ResultStreamFaker stream(&db);

View File

@ -303,6 +303,9 @@ target_link_libraries(${test_prefix}storage_v2_decoder_encoder mg-storage-v2)
add_unit_test(storage_v2_durability.cpp)
target_link_libraries(${test_prefix}storage_v2_durability mg-storage-v2)
add_unit_test(storage_rocks.cpp)
target_link_libraries(${test_prefix}storage_rocks mg-storage-v2)
add_unit_test(storage_v2_edge.cpp)
target_link_libraries(${test_prefix}storage_v2_edge mg-storage-v2)

View File

@ -49,8 +49,9 @@ auto ToEdgeList(const memgraph::communication::bolt::Value &v) {
class InterpreterTest : public ::testing::Test {
public:
memgraph::storage::Storage db_;
memgraph::storage::rocks::RocksDBStorage disk_db;
std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "MG_tests_unit_interpreter"};
memgraph::query::InterpreterContext interpreter_context{&db_, {}, data_directory};
memgraph::query::InterpreterContext interpreter_context{&db_, &disk_db, {}, data_directory};
InterpreterFaker default_interpreter{&interpreter_context};
@ -1066,7 +1067,7 @@ TEST_F(InterpreterTest, AllowLoadCsvConfig) {
"row"};
memgraph::query::InterpreterContext csv_interpreter_context{
&db_, {.query = {.allow_load_csv = allow_load_csv}}, directory_manager.Path()};
&db_, &disk_db, {.query = {.allow_load_csv = allow_load_csv}}, directory_manager.Path()};
InterpreterFaker interpreter_faker{&csv_interpreter_context};
for (const auto &query : queries) {
if (allow_load_csv) {

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// 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
@ -202,8 +202,9 @@ DatabaseState GetState(memgraph::storage::Storage *db) {
}
auto Execute(memgraph::storage::Storage *db, const std::string &query) {
memgraph::storage::rocks::RocksDBStorage disk_db;
auto data_directory = std::filesystem::temp_directory_path() / "MG_tests_unit_query_dump";
memgraph::query::InterpreterContext context(db, memgraph::query::InterpreterConfig{}, data_directory);
memgraph::query::InterpreterContext context(db, &disk_db, memgraph::query::InterpreterConfig{}, data_directory);
memgraph::query::Interpreter interpreter(&context);
ResultStreamFaker stream(db);
@ -757,8 +758,10 @@ TEST(DumpTest, ExecuteDumpDatabase) {
class StatefulInterpreter {
public:
explicit StatefulInterpreter(memgraph::storage::Storage *db)
: db_(db), context_(db_, memgraph::query::InterpreterConfig{}, data_directory_), interpreter_(&context_) {}
explicit StatefulInterpreter(memgraph::storage::Storage *db, memgraph::storage::rocks::RocksDBStorage *disk_db)
: db_(db),
context_(db_, disk_db, memgraph::query::InterpreterConfig{}, data_directory_),
interpreter_(&context_) {}
auto Execute(const std::string &query) {
ResultStreamFaker stream(db_);
@ -785,7 +788,8 @@ const std::filesystem::path StatefulInterpreter::data_directory_{std::filesystem
// NOLINTNEXTLINE(hicpp-special-member-functions)
TEST(DumpTest, ExecuteDumpDatabaseInMulticommandTransaction) {
memgraph::storage::Storage db;
StatefulInterpreter interpreter(&db);
memgraph::storage::rocks::RocksDBStorage disk_db;
StatefulInterpreter interpreter(&db, &disk_db);
// Begin the transaction before the vertex is created.
interpreter.Execute("BEGIN");

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// 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
@ -28,6 +28,7 @@ DECLARE_bool(query_cost_planner);
class QueryExecution : public testing::Test {
protected:
std::optional<memgraph::storage::Storage> db_;
std::optional<memgraph::storage::rocks::RocksDBStorage> disk_db_;
std::optional<memgraph::query::InterpreterContext> interpreter_context_;
std::optional<memgraph::query::Interpreter> interpreter_;
@ -35,7 +36,8 @@ class QueryExecution : public testing::Test {
void SetUp() {
db_.emplace();
interpreter_context_.emplace(&*db_, memgraph::query::InterpreterConfig{}, data_directory);
disk_db_.emplace();
interpreter_context_.emplace(&*db_, &*disk_db_, memgraph::query::InterpreterConfig{}, data_directory);
interpreter_.emplace(&*interpreter_context_);
}

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// 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
@ -22,6 +22,7 @@
#include "query/config.hpp"
#include "query/interpreter.hpp"
#include "query/stream/streams.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/storage.hpp"
#include "test_utils.hpp"
@ -55,6 +56,7 @@ class StreamsTest : public ::testing::Test {
protected:
memgraph::storage::Storage db_;
memgraph::storage::rocks::RocksDBStorage disk_db_;
std::filesystem::path data_directory_{GetCleanDataDirectory()};
KafkaClusterMock mock_cluster_{std::vector<std::string>{kTopicName}};
// Though there is a Streams object in interpreter context, it makes more sense to use a separate object to test,
@ -62,7 +64,8 @@ class StreamsTest : public ::testing::Test {
// Streams constructor.
// InterpreterContext::auth_checker_ is used in the Streams object, but only in the message processing part. Because
// these tests don't send any messages, the auth_checker_ pointer can be left as nullptr.
memgraph::query::InterpreterContext interpreter_context_{&db_, memgraph::query::InterpreterConfig{}, data_directory_};
memgraph::query::InterpreterContext interpreter_context_{&db_, &disk_db_, memgraph::query::InterpreterConfig{},
data_directory_};
std::filesystem::path streams_data_directory_{data_directory_ / "separate-dir-for-test"};
std::optional<Streams> streams_;

View File

@ -0,0 +1,323 @@
// 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 <exception>
#include <unordered_set>
#include "query/common.hpp"
#include "query/db_accessor.hpp"
#include "storage/v2/delta.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "storage/v2/view.hpp"
class RocksDBStorageTest : public ::testing::TestWithParam<bool> {
public:
~RocksDBStorageTest() { db.Clear(); }
protected:
memgraph::storage::rocks::RocksDBStorage db;
memgraph::storage::Storage storage;
};
TEST_F(RocksDBStorageTest, SerializeVertexGID) {
// empty vertices, only gid is serialized
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
std::unordered_set<uint64_t> gids;
for (uint64_t i = 0; i < 5; ++i) {
gids.insert(i);
auto impl = dba.InsertVertex();
impl.SetGid(memgraph::storage::Gid::FromUint(i));
db.StoreVertex(impl);
}
// load vertices from disk
auto loaded_vertices = db.Vertices(dba);
ASSERT_EQ(loaded_vertices.size(), 5);
for (const auto &vertex_acc : loaded_vertices) {
ASSERT_TRUE(gids.contains(vertex_acc.Gid().AsUint()));
}
}
TEST_F(RocksDBStorageTest, SerializeVertexGIDLabels) {
// serialize vertex's gid with its single label
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
// save vertices on disk
std::unordered_set<uint64_t> gids;
std::vector<memgraph::storage::LabelId> label_ids{dba.NameToLabel("Player"), dba.NameToLabel("Person"),
dba.NameToLabel("Ball")};
for (int i = 0; i < 5; ++i) {
gids.insert(i);
auto impl = dba.InsertVertex();
impl.SetGid(memgraph::storage::Gid::FromUint(i));
impl.AddLabel(label_ids[i % 3]);
db.StoreVertex(impl);
}
// load vertices from disk
auto loaded_vertices = db.Vertices(dba);
ASSERT_EQ(loaded_vertices.size(), 5);
for (const auto &vertex_acc : loaded_vertices) {
ASSERT_TRUE(gids.contains(vertex_acc.Gid().AsUint()));
auto labels = vertex_acc.Labels(memgraph::storage::View::OLD);
ASSERT_EQ(labels->size(), 1);
ASSERT_TRUE(std::all_of(labels->begin(), labels->end(), [&label_ids](const auto &label_id) {
return std::find(label_ids.begin(), label_ids.end(), label_id) != label_ids.end();
}));
}
}
TEST_F(RocksDBStorageTest, SerializeVertexGIDMutlipleLabels) {
// serialize vertex's gid with multiple labels it contains
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
// save vertices on disk
std::unordered_set<uint64_t> gids;
std::vector<memgraph::storage::LabelId> label_ids{dba.NameToLabel("Player"), dba.NameToLabel("Person"),
dba.NameToLabel("Ball")};
for (int i = 0; i < 5; ++i) {
gids.insert(i);
auto impl = dba.InsertVertex();
impl.SetGid(memgraph::storage::Gid::FromUint(i));
impl.AddLabel(label_ids[i % 3]);
impl.AddLabel(label_ids[(i + 1) % 3]);
db.StoreVertex(impl);
}
// load vertices from disk
auto loaded_vertices = db.Vertices(dba);
ASSERT_EQ(loaded_vertices.size(), 5);
for (const auto &vertex_acc : loaded_vertices) {
ASSERT_TRUE(gids.contains(vertex_acc.Gid().AsUint()));
auto labels = vertex_acc.Labels(memgraph::storage::View::OLD);
ASSERT_EQ(labels->size(), 2);
ASSERT_TRUE(std::all_of(labels->begin(), labels->end(), [&label_ids](const auto &label_id) {
return std::find(label_ids.begin(), label_ids.end(), label_id) != label_ids.end();
}));
}
}
TEST_F(RocksDBStorageTest, GetVerticesByLabel) {
// search vertices by label
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
// prepare labels
std::vector<memgraph::storage::LabelId> label_ids{dba.NameToLabel("Player"), dba.NameToLabel("Player"),
dba.NameToLabel("Ball")};
// insert vertices
for (int i = 0; i < 5; ++i) {
auto impl = dba.InsertVertex();
impl.AddLabel(label_ids[i % 3]);
db.StoreVertex(impl);
}
// load vertices from disk
auto player_vertices = db.Vertices(dba, dba.NameToLabel("Player"));
auto ball_vertices = db.Vertices(dba, dba.NameToLabel("Ball"));
ASSERT_EQ(player_vertices.size(), 4);
ASSERT_EQ(ball_vertices.size(), 1);
}
TEST_F(RocksDBStorageTest, GetVerticesByProperty) {
// search vertices by property value
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
// prepare ssd properties
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> ssd_properties_1;
ssd_properties_1.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(225.84));
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> ssd_properties_2;
ssd_properties_2.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(226.84));
// prepare hdd properties
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> hdd_properties_1;
hdd_properties_1.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(125.84));
std::vector properties{ssd_properties_1, ssd_properties_2, hdd_properties_1, hdd_properties_1};
// insert vertices
for (int i = 0; i < 4; ++i) {
auto impl = dba.InsertVertex();
memgraph::query::MultiPropsInitChecked(&impl, properties[i]);
db.StoreVertex(impl);
}
// load vertices from disk
auto ssd_vertices_1 = db.Vertices(dba, dba.NameToProperty("price"), memgraph::storage::PropertyValue(225.84));
auto hdd_vertices = db.Vertices(dba, dba.NameToProperty("price"), memgraph::storage::PropertyValue(125.84));
auto hdd_vertices_non_existing =
db.Vertices(dba, dba.NameToProperty("price"), memgraph::storage::PropertyValue(125.81));
ASSERT_EQ(ssd_vertices_1.size(), 1);
ASSERT_EQ(hdd_vertices.size(), 2);
ASSERT_EQ(hdd_vertices_non_existing.size(), 0);
}
TEST_F(RocksDBStorageTest, DeleteVertex) {
// auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTE);
auto storage_dba = storage.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> properties;
// samo 1 property stane
properties.emplace(dba.NameToProperty("sum"), memgraph::storage::PropertyValue("2TB"));
properties.emplace(dba.NameToProperty("same_type"), memgraph::storage::PropertyValue(true));
// properties.emplace(dba.NameToProperty("cluster_price"), memgraph::storage::PropertyValue(2000.42));
// create vertex
auto impl = dba.InsertVertex();
impl.AddLabel(dba.NameToLabel("Player"));
memgraph::query::MultiPropsInitChecked(&impl, properties);
db.StoreVertex(impl);
// find vertex should work now
ASSERT_TRUE(db.FindVertex(std::to_string(impl.Gid().AsUint()), dba).has_value());
db.FindVertex(std::to_string(impl.Gid().AsUint()), dba);
// RocksDB doesn't physically delete entry so deletion will pass two times
ASSERT_TRUE(db.DeleteVertex(impl).has_value());
ASSERT_TRUE(db.DeleteVertex(impl).has_value());
// second time you shouldn't be able to find the vertex
ASSERT_FALSE(db.FindVertex(std::to_string(impl.Gid().AsUint()), dba).has_value());
}
TEST_F(RocksDBStorageTest, SerializeVertexGIDProperties) {
// serializes vertex's gid, multiple labels and properties
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
// prepare labels
std::vector<memgraph::storage::LabelId> label_ids{dba.NameToLabel("Player"), dba.NameToLabel("Person"),
dba.NameToLabel("Ball")};
// prepare properties
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> properties;
properties.emplace(dba.NameToProperty("name"), memgraph::storage::PropertyValue("disk"));
properties.emplace(dba.NameToProperty("memory"), memgraph::storage::PropertyValue("1TB"));
properties.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(1000.21));
properties.emplace(dba.NameToProperty("price2"), memgraph::storage::PropertyValue(1000.212));
// gids
std::unordered_set<uint64_t> gids;
for (int i = 0; i < 5; ++i) {
gids.insert(i);
auto impl = dba.InsertVertex();
impl.SetGid(memgraph::storage::Gid::FromUint(i));
impl.AddLabel(label_ids[i % 3]);
impl.AddLabel(label_ids[(i + 1) % 3]);
memgraph::query::MultiPropsInitChecked(&impl, properties);
db.StoreVertex(impl);
}
// load vertices from disk
auto loaded_vertices = db.Vertices(dba);
ASSERT_EQ(loaded_vertices.size(), 5);
for (const auto &vertex_acc : loaded_vertices) {
ASSERT_TRUE(gids.contains(vertex_acc.Gid().AsUint()));
// labels
auto labels = vertex_acc.Labels(memgraph::storage::View::OLD);
ASSERT_EQ(labels->size(), 2);
ASSERT_TRUE(std::all_of(labels->begin(), labels->end(), [&label_ids](const auto &label_id) {
return std::find(label_ids.begin(), label_ids.end(), label_id) != label_ids.end();
}));
// check properties
auto props = vertex_acc.Properties(memgraph::storage::View::OLD);
ASSERT_FALSE(props.HasError());
auto prop_name = vertex_acc.GetProperty(memgraph::storage::View::OLD, dba.NameToProperty("name"));
auto prop_memory = vertex_acc.GetProperty(memgraph::storage::View::OLD, dba.NameToProperty("memory"));
auto prop_price = vertex_acc.GetProperty(memgraph::storage::View::OLD, dba.NameToProperty("price"));
auto prop_unexisting = vertex_acc.GetProperty(memgraph::storage::View::OLD, dba.NameToProperty("random"));
ASSERT_TRUE(prop_name->IsString());
ASSERT_EQ(prop_name->ValueString(), "disk");
ASSERT_TRUE(prop_memory->IsString());
ASSERT_EQ(prop_memory->ValueString(), "1TB");
ASSERT_TRUE(prop_price->IsDouble());
ASSERT_DOUBLE_EQ(prop_price->ValueDouble(), 1000.21);
ASSERT_TRUE(prop_unexisting->IsNull());
}
}
TEST_F(RocksDBStorageTest, SerializeEdge) {
// create two vertices and edge between them
// search by one of the vertices, return edge
// check deserialization for both vertices and edge
auto storage_dba = storage.Access(memgraph::storage::IsolationLevel::READ_UNCOMMITTED);
memgraph::query::DbAccessor dba(&storage_dba);
std::vector<memgraph::storage::LabelId> label_ids{dba.NameToLabel("Player"), dba.NameToLabel("Referee")};
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> properties_1;
properties_1.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(221.84));
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> properties_2;
properties_2.emplace(dba.NameToProperty("price"), memgraph::storage::PropertyValue(222.84));
std::vector properties{properties_1, properties_2};
for (int i = 0; i < 2; ++i) {
auto impl = dba.InsertVertex();
impl.AddLabel(label_ids[i]);
memgraph::query::MultiPropsInitChecked(&impl, properties[i]);
db.StoreVertex(impl);
}
// prepare edge properties
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> edge_properties;
edge_properties.emplace(dba.NameToProperty("sum"), memgraph::storage::PropertyValue("2TB"));
edge_properties.emplace(dba.NameToProperty("same_type"), memgraph::storage::PropertyValue(true));
edge_properties.emplace(dba.NameToProperty("cluster_price"), memgraph::storage::PropertyValue(2000.42));
// Before inserting edge, find two vertices
// find source vertex by the property
auto src_vertices = db.Vertices(dba, dba.NameToProperty("price"), memgraph::storage::PropertyValue(221.84));
ASSERT_EQ(src_vertices.size(), 1);
auto src_vertex = src_vertices[0];
// find destination vertex by the property
auto dest_vertices = db.Vertices(dba, dba.NameToProperty("price"), memgraph::storage::PropertyValue(222.84));
ASSERT_EQ(dest_vertices.size(), 1);
auto dest_vertex = dest_vertices[0];
// insert the edge
uint64_t edge_gid = 2;
auto edge_type_id = "CONNECTION";
auto impl_edge = dba.InsertEdge(&src_vertex, &dest_vertex, dba.NameToEdgeType(edge_type_id));
ASSERT_FALSE(impl_edge.HasError());
(*impl_edge).SetGid(memgraph::storage::Gid::FromUint(edge_gid));
memgraph::query::MultiPropsInitChecked(&*impl_edge, edge_properties);
db.StoreEdge(*impl_edge);
// Test out edges of the source vertex
auto src_out_edges = db.OutEdges(src_vertex, dba);
ASSERT_EQ(src_out_edges.size(), 1);
auto src_out_edge = src_out_edges[0];
// test from edge accessor
auto from_out_edge_acc = src_out_edge.From();
ASSERT_EQ(from_out_edge_acc.Gid(), src_vertex.Gid());
ASSERT_EQ(from_out_edge_acc.Labels(memgraph::storage::View::OLD)->size(), 1);
ASSERT_EQ(from_out_edge_acc.Labels(memgraph::storage::View::OLD)->at(0), label_ids[0]);
ASSERT_EQ(*from_out_edge_acc.Properties(memgraph::storage::View::OLD), properties_1);
// test to edge accessor
auto to_out_edge_acc = src_out_edge.To();
ASSERT_EQ(to_out_edge_acc.Gid(), dest_vertex.Gid());
ASSERT_EQ(to_out_edge_acc.Labels(memgraph::storage::View::OLD)->size(), 1);
ASSERT_EQ(to_out_edge_acc.Labels(memgraph::storage::View::OLD)->at(0), label_ids[1]);
ASSERT_EQ(*to_out_edge_acc.Properties(memgraph::storage::View::OLD), properties_2);
// test edge accessor
ASSERT_EQ(src_out_edge.Gid().AsUint(), edge_gid);
ASSERT_EQ(src_out_edge.EdgeType(), dba.NameToEdgeType(edge_type_id));
ASSERT_EQ(*src_out_edge.Properties(memgraph::storage::View::OLD), edge_properties);
// Test in edge of the destination vertex
auto dest_in_edges = db.InEdges(dest_vertex, dba);
ASSERT_EQ(dest_in_edges.size(), 1);
auto dest_in_edge = dest_in_edges[0];
// test from edge accessor
auto from_in_edge_acc = dest_in_edge.From();
ASSERT_EQ(from_in_edge_acc.Gid(), from_out_edge_acc.Gid());
ASSERT_EQ(from_in_edge_acc.Labels(memgraph::storage::View::OLD)->size(), 1);
ASSERT_EQ(from_in_edge_acc.Labels(memgraph::storage::View::OLD)->at(0),
from_out_edge_acc.Labels(memgraph::storage::View::OLD)->at(0));
ASSERT_EQ(*from_in_edge_acc.Properties(memgraph::storage::View::OLD),
*from_out_edge_acc.Properties(memgraph::storage::View::OLD));
// test in edge accessors
auto to_in_edge_acc = dest_in_edge.To();
ASSERT_EQ(to_in_edge_acc.Gid(), to_out_edge_acc.Gid());
ASSERT_EQ(to_in_edge_acc.Labels(memgraph::storage::View::OLD)->size(), 1);
ASSERT_EQ(to_in_edge_acc.Labels(memgraph::storage::View::OLD)->at(0),
to_out_edge_acc.Labels(memgraph::storage::View::OLD)->at(0));
ASSERT_EQ(*to_in_edge_acc.Properties(memgraph::storage::View::OLD),
*to_out_edge_acc.Properties(memgraph::storage::View::OLD));
// test edge accessors
ASSERT_EQ(dest_in_edge.Gid(), src_out_edge.Gid());
ASSERT_EQ(dest_in_edge.EdgeType(), src_out_edge.EdgeType());
ASSERT_EQ(*dest_in_edge.Properties(memgraph::storage::View::OLD),
*src_out_edge.Properties(memgraph::storage::View::OLD));
}

View File

@ -26,8 +26,9 @@ corresponding interpreter/.
class TransactionQueueSimpleTest : public ::testing::Test {
protected:
memgraph::storage::Storage db_;
memgraph::storage::rocks::RocksDBStorage disk_db_;
std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "MG_tests_unit_transaction_queue_intr"};
memgraph::query::InterpreterContext interpreter_context{&db_, {}, data_directory};
memgraph::query::InterpreterContext interpreter_context{&db_, &disk_db_, {}, data_directory};
InterpreterFaker running_interpreter{&interpreter_context}, main_interpreter{&interpreter_context};
};

View File

@ -31,9 +31,10 @@ corresponding interpreter.
class TransactionQueueMultipleTest : public ::testing::Test {
protected:
memgraph::storage::Storage db_;
memgraph::storage::rocks::RocksDBStorage disk_db_;
std::filesystem::path data_directory{std::filesystem::temp_directory_path() /
"MG_tests_unit_transaction_queue_multiple_intr"};
memgraph::query::InterpreterContext interpreter_context{&db_, {}, data_directory};
memgraph::query::InterpreterContext interpreter_context{&db_, &disk_db_, {}, data_directory};
InterpreterFaker main_interpreter{&interpreter_context};
std::vector<InterpreterFaker *> running_interpreters;