rocksdb operations to accessors api
This commit is contained in:
commit
c785bed20c
@ -8,11 +8,20 @@ set(storage_v2_src_files
|
|||||||
durability/wal.cpp
|
durability/wal.cpp
|
||||||
edge_accessor.cpp
|
edge_accessor.cpp
|
||||||
indices.cpp
|
indices.cpp
|
||||||
|
inmemory/edge_accessor.cpp
|
||||||
|
inmemory/storage.cpp
|
||||||
|
inmemory/vertex_accessor.cpp
|
||||||
property_store.cpp
|
property_store.cpp
|
||||||
vertex_accessor.cpp
|
|
||||||
storage.cpp
|
|
||||||
disk/compaction_filter.cpp
|
disk/compaction_filter.cpp
|
||||||
disk/storage.hpp)
|
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
|
set(storage_v2_src_files
|
||||||
|
273
src/storage/v2/disk/edge_accessor.cpp
Normal file
273
src/storage/v2/disk/edge_accessor.cpp
Normal 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
|
100
src/storage/v2/disk/edge_accessor.hpp
Normal file
100
src/storage/v2/disk/edge_accessor.hpp
Normal 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
|
358
src/storage/v2/disk/old_storage.cpp
Normal file
358
src/storage/v2/disk/old_storage.cpp
Normal 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
|
555
src/storage/v2/disk/old_storage.hpp
Normal file
555
src/storage/v2/disk/old_storage.hpp
Normal 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
|
@ -17,6 +17,7 @@
|
|||||||
#include <rocksdb/status.h>
|
#include <rocksdb/status.h>
|
||||||
|
|
||||||
#include "query/db_accessor.hpp"
|
#include "query/db_accessor.hpp"
|
||||||
|
#include "utils/file.hpp"
|
||||||
#include "utils/string.hpp"
|
#include "utils/string.hpp"
|
||||||
|
|
||||||
namespace memgraph::storage::rocks {
|
namespace memgraph::storage::rocks {
|
||||||
@ -26,17 +27,17 @@ constexpr const char *edgeHandle = "edge";
|
|||||||
constexpr const char *outEdgeDirection = "0";
|
constexpr const char *outEdgeDirection = "0";
|
||||||
constexpr const char *inEdgeDirection = "1";
|
constexpr const char *inEdgeDirection = "1";
|
||||||
|
|
||||||
/// Use it for operations that must successfully finish.
|
// /// Use it for operations that must successfully finish.
|
||||||
inline void AssertRocksDBStatus(const rocksdb::Status &status) {
|
inline void AssertRocksDBStatus(const rocksdb::Status &status) {
|
||||||
MG_ASSERT(status.ok(), "rocksdb: {}", status.ToString());
|
MG_ASSERT(status.ok(), "rocksdb: {}", status.ToString());
|
||||||
}
|
}
|
||||||
|
|
||||||
inline bool CheckRocksDBStatus(const rocksdb::Status &status) {
|
// inline bool CheckRocksDBStatus(const rocksdb::Status &status) {
|
||||||
if (!status.ok()) [[unlikely]] {
|
// if (!status.ok()) [[unlikely]] {
|
||||||
spdlog::error("rocksdb: {}", status.ToString());
|
// spdlog::error("rocksdb: {}", status.ToString());
|
||||||
}
|
// }
|
||||||
return status.ok();
|
// return status.ok();
|
||||||
}
|
// }
|
||||||
|
|
||||||
class RocksDBStorage {
|
class RocksDBStorage {
|
||||||
public:
|
public:
|
||||||
@ -64,319 +65,320 @@ class RocksDBStorage {
|
|||||||
delete db_;
|
delete db_;
|
||||||
}
|
}
|
||||||
|
|
||||||
// EDGE ACCESSOR FUNCTIONALITIES
|
// // EDGE ACCESSOR FUNCTIONALITIES
|
||||||
// -----------------------------------------------------------
|
// // -----------------------------------------------------------
|
||||||
|
|
||||||
/// fetch the edge's source vertex by its GID
|
// /// fetch the edge's source vertex by its GID
|
||||||
std::optional<query::VertexAccessor> FromVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
|
// std::optional<query::VertexAccessor> FromVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
|
||||||
return FindVertex(SerializeIdType(edge_acc.From().Gid()), dba);
|
// return FindVertex(SerializeIdType(edge_acc.From().Gid()), dba);
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// fetch the edge's destination vertex by its GID
|
// /// fetch the edge's destination vertex by its GID
|
||||||
std::optional<query::VertexAccessor> ToVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
|
// std::optional<query::VertexAccessor> ToVertex(const query::EdgeAccessor &edge_acc, query::DbAccessor &dba) {
|
||||||
return FindVertex(SerializeIdType(edge_acc.To().Gid()), dba);
|
// return FindVertex(SerializeIdType(edge_acc.To().Gid()), dba);
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// VERTEX ACCESSOR FUNCTIONALITIES
|
// /// VERTEX ACCESSOR FUNCTIONALITIES
|
||||||
/// ------------------------------------------------------------
|
// /// ------------------------------------------------------------
|
||||||
|
|
||||||
/// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
|
// /// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
|
||||||
/// src_gid | other_vertex_gid | 0 | ...
|
// /// src_gid | other_vertex_gid | 0 | ...
|
||||||
/// other_vertex_gid | src_gid | 1 | ...
|
// /// other_vertex_gid | src_gid | 1 | ...
|
||||||
/// We use the firt way since this should be possible to optimize using Bloom filters and prefix search
|
// /// 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) {
|
// std::vector<query::EdgeAccessor> OutEdges(const query::VertexAccessor &vertex_acc, query::DbAccessor &dba) {
|
||||||
const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
|
// const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
|
||||||
std::vector<query::EdgeAccessor> out_edges;
|
// std::vector<query::EdgeAccessor> out_edges;
|
||||||
auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
|
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
|
||||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||||
const std::string_view key = it->key().ToStringView();
|
// const std::string_view key = it->key().ToStringView();
|
||||||
const auto vertex_parts = utils::Split(key, "|");
|
// const auto vertex_parts = utils::Split(key, "|");
|
||||||
if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == outEdgeDirection) {
|
// if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == outEdgeDirection) {
|
||||||
out_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
|
// out_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
return out_edges;
|
// return out_edges;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
|
// /// The VertexAccessor's out edge with gid src_gid has the following format in the RocksDB:
|
||||||
/// other_vertex_gid | dest_gid | 0 | ...
|
// /// other_vertex_gid | dest_gid | 0 | ...
|
||||||
/// dest_gid | other_verte_gid | 1 | ...
|
// /// dest_gid | other_verte_gid | 1 | ...
|
||||||
/// we use the second way since this should be possible to optimize using Bloom filters and prefix search.
|
// /// 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) {
|
// std::vector<query::EdgeAccessor> InEdges(const query::VertexAccessor &vertex_acc, query::DbAccessor &dba) {
|
||||||
const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
|
// const auto vertex_acc_gid = SerializeIdType(vertex_acc.Gid());
|
||||||
std::vector<query::EdgeAccessor> in_edges;
|
// std::vector<query::EdgeAccessor> in_edges;
|
||||||
auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
|
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), edge_chandle));
|
||||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||||
const std::string_view key = it->key().ToStringView();
|
// const std::string_view key = it->key().ToStringView();
|
||||||
const auto vertex_parts = utils::Split(key, "|");
|
// const auto vertex_parts = utils::Split(key, "|");
|
||||||
if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == inEdgeDirection) {
|
// if (vertex_parts[0] == vertex_acc_gid && vertex_parts[2] == inEdgeDirection) {
|
||||||
in_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
|
// in_edges.push_back(DeserializeEdge(key, it->value().ToStringView(), dba));
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
return in_edges;
|
// return in_edges;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// TODO: how will we handle new vertex creation
|
// /// TODO: how will we handle new vertex creation
|
||||||
|
|
||||||
/// STORAGE ACCESSOR FUNCTIONALITIES
|
// /// STORAGE ACCESSOR FUNCTIONALITIES
|
||||||
/// -----------------------------------------------------------
|
// /// -----------------------------------------------------------
|
||||||
|
|
||||||
/// TODO: how will we handle new edge creation
|
// /// TODO: how will we handle new edge creation
|
||||||
|
|
||||||
/// @return Accessor to the deleted edge if a deletion took place, std::nullopt otherwise.
|
// /// @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.
|
// /// Delete two edge entries since on edge is represented on a two-fold level.
|
||||||
/// Edges are deleted from logical partition containing edges.
|
// /// Edges are deleted from logical partition containing edges.
|
||||||
std::optional<query::EdgeAccessor> DeleteEdge(const query::EdgeAccessor &edge_acc) {
|
// std::optional<query::EdgeAccessor> DeleteEdge(const query::EdgeAccessor &edge_acc) {
|
||||||
auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
|
// auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
|
||||||
if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, src_dest_key)) ||
|
// if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, src_dest_key)) ||
|
||||||
!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, dest_src_key))) {
|
// !CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), edge_chandle, dest_src_key))) {
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
return edge_acc;
|
// return edge_acc;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Helper function, not used in the real accessor.
|
// /// Helper function, not used in the real accessor.
|
||||||
std::optional<std::vector<query::EdgeAccessor>> DeleteEdges(const auto &edge_accessors) {
|
// std::optional<std::vector<query::EdgeAccessor>> DeleteEdges(const auto &edge_accessors) {
|
||||||
std::vector<query::EdgeAccessor> edge_accs;
|
// std::vector<query::EdgeAccessor> edge_accs;
|
||||||
for (auto &&it : edge_accessors) {
|
// for (auto &&it : edge_accessors) {
|
||||||
if (const auto deleted_edge_res = DeleteEdge(it); !deleted_edge_res.has_value()) {
|
// if (const auto deleted_edge_res = DeleteEdge(it); !deleted_edge_res.has_value()) {
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
edge_accs.push_back(it);
|
// edge_accs.push_back(it);
|
||||||
}
|
// }
|
||||||
return edge_accs;
|
// return edge_accs;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// @return A reference to the deleted vertex accessor if deleted, otherwise std::nullopt.
|
// /// @return A reference to the deleted vertex accessor if deleted, otherwise std::nullopt.
|
||||||
/// Delete vertex from logical partition containing vertices.
|
// /// Delete vertex from logical partition containing vertices.
|
||||||
std::optional<query::VertexAccessor> DeleteVertex(const query::VertexAccessor &vertex_acc) {
|
// std::optional<query::VertexAccessor> DeleteVertex(const query::VertexAccessor &vertex_acc) {
|
||||||
if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc)))) {
|
// if (!CheckRocksDBStatus(db_->Delete(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc)))) {
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
return vertex_acc;
|
// return vertex_acc;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise.
|
// /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise.
|
||||||
/// Delete vertex from logical partition containing vertices.
|
// /// Delete vertex from logical partition containing vertices.
|
||||||
/// For each edge delete two key-value entries from logical partition containing edges.
|
// /// For each edge delete two key-value entries from logical partition containing edges.
|
||||||
std::optional<std::pair<query::VertexAccessor, std::vector<query::EdgeAccessor>>> DetachDeleteVertex(
|
// std::optional<std::pair<query::VertexAccessor, std::vector<query::EdgeAccessor>>> DetachDeleteVertex(
|
||||||
const query::VertexAccessor &vertex_acc) {
|
// const query::VertexAccessor &vertex_acc) {
|
||||||
auto del_vertex = DeleteVertex(vertex_acc);
|
// auto del_vertex = DeleteVertex(vertex_acc);
|
||||||
if (!del_vertex.has_value()) {
|
// if (!del_vertex.has_value()) {
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
auto out_edges = vertex_acc.OutEdges(storage::View::OLD);
|
// auto out_edges = vertex_acc.OutEdges(storage::View::OLD);
|
||||||
auto in_edges = vertex_acc.InEdges(storage::View::OLD);
|
// auto in_edges = vertex_acc.InEdges(storage::View::OLD);
|
||||||
if (out_edges.HasError() || in_edges.HasError()) {
|
// if (out_edges.HasError() || in_edges.HasError()) {
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
if (auto del_edges = DeleteEdges(*out_edges), del_in_edges = DeleteEdges(*in_edges);
|
// if (auto del_edges = DeleteEdges(*out_edges), del_in_edges = DeleteEdges(*in_edges);
|
||||||
del_edges.has_value() && del_in_edges.has_value()) {
|
// del_edges.has_value() && del_in_edges.has_value()) {
|
||||||
del_edges->insert(del_in_edges->end(), std::make_move_iterator(del_in_edges->begin()),
|
// del_edges->insert(del_in_edges->end(), std::make_move_iterator(del_in_edges->begin()),
|
||||||
std::make_move_iterator(del_in_edges->end()));
|
// std::make_move_iterator(del_in_edges->end()));
|
||||||
return std::make_pair(*del_vertex, *del_edges);
|
// return std::make_pair(*del_vertex, *del_edges);
|
||||||
}
|
// }
|
||||||
return std::nullopt;
|
// return std::nullopt;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// STORING
|
// /// STORING
|
||||||
/// -----------------------------------------------------------
|
// /// -----------------------------------------------------------
|
||||||
|
|
||||||
/// Serialize and store in-memory vertex to the disk.
|
// /// Serialize and store in-memory vertex to the disk.
|
||||||
/// Properties are serialized as the value
|
// /// Properties are serialized as the value
|
||||||
void StoreVertex(const query::VertexAccessor &vertex_acc) {
|
// void StoreVertex(const query::VertexAccessor &vertex_acc) {
|
||||||
AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc),
|
// AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), vertex_chandle, SerializeVertex(vertex_acc),
|
||||||
SerializeProperties(vertex_acc.PropertyStore())));
|
// SerializeProperties(vertex_acc.PropertyStore())));
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Store edge as two key-value entries in the RocksDB.
|
// /// Store edge as two key-value entries in the RocksDB.
|
||||||
void StoreEdge(const query::EdgeAccessor &edge_acc) {
|
// void StoreEdge(const query::EdgeAccessor &edge_acc) {
|
||||||
auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
|
// auto [src_dest_key, dest_src_key] = SerializeEdge(edge_acc);
|
||||||
const std::string value = SerializeProperties(edge_acc.PropertyStore());
|
// 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, src_dest_key, value));
|
||||||
AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), edge_chandle, dest_src_key, value));
|
// AssertRocksDBStatus(db_->Put(rocksdb::WriteOptions(), edge_chandle, dest_src_key, value));
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// UPDATE PART
|
// /// UPDATE PART
|
||||||
/// -----------------------------------------------------------
|
// /// -----------------------------------------------------------
|
||||||
|
|
||||||
/// Clear all entries from the database.
|
// /// Clear all entries from the database.
|
||||||
/// TODO: check if this deletes all entries, or you also need to specify handle here
|
// /// 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
|
// /// TODO: This will not be needed in the production code and can possibly removed in testing
|
||||||
void Clear() {
|
// void Clear() {
|
||||||
auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions()));
|
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions()));
|
||||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||||
db_->Delete(rocksdb::WriteOptions(), it->key().ToString());
|
// db_->Delete(rocksdb::WriteOptions(), it->key().ToString());
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// READ PART
|
// /// READ PART
|
||||||
/// -----------------------------------------------------------
|
// /// -----------------------------------------------------------
|
||||||
|
|
||||||
/// TODO: if the need comes for using also a GID object, use std::variant
|
// /// 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
|
// /// This should again be changed when we have mulitple same vertices
|
||||||
std::optional<query::VertexAccessor> FindVertex(const std::string_view gid, query::DbAccessor &dba) {
|
// 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));
|
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), vertex_chandle));
|
||||||
std::optional<query::VertexAccessor> result = {};
|
// std::optional<query::VertexAccessor> result = {};
|
||||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||||
const auto &key = it->key().ToString();
|
// const auto &key = it->key().ToString();
|
||||||
if (const auto vertex_parts = utils::Split(key, "|"); vertex_parts[1] == gid) {
|
// if (const auto vertex_parts = utils::Split(key, "|"); vertex_parts[1] == gid) {
|
||||||
result = DeserializeVertex(key, it->value().ToStringView(), dba);
|
// result = DeserializeVertex(key, it->value().ToStringView(), dba);
|
||||||
break;
|
// break;
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
return result;
|
// return result;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Get all vertices by a label.
|
// /// Get all vertices by a label.
|
||||||
std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::LabelId &label_id) {
|
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::LabelId &label_id) {
|
||||||
return Vertices(dba, [label_id](const auto &vertex) {
|
// return Vertices(dba, [label_id](const auto &vertex) {
|
||||||
const auto res = vertex.HasLabel(storage::View::OLD, label_id);
|
// const auto res = vertex.HasLabel(storage::View::OLD, label_id);
|
||||||
return !res.HasError() && *res;
|
// return !res.HasError() && *res;
|
||||||
});
|
// });
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Read all vertices stored in the database by a property
|
// /// Read all vertices stored in the database by a property
|
||||||
std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::PropertyId &property_id,
|
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba, const storage::PropertyId &property_id,
|
||||||
const storage::PropertyValue &property_value) {
|
// const storage::PropertyValue &property_value) {
|
||||||
return Vertices(dba, [property_id, property_value](const auto &vertex) {
|
// return Vertices(dba, [property_id, property_value](const auto &vertex) {
|
||||||
const auto res = vertex.GetProperty(storage::View::OLD, property_id);
|
// const auto res = vertex.GetProperty(storage::View::OLD, property_id);
|
||||||
return !res.HasError() && *res == property_value;
|
// return !res.HasError() && *res == property_value;
|
||||||
});
|
// });
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Get all vertices.
|
// /// Get all vertices.
|
||||||
std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba) {
|
// std::vector<query::VertexAccessor> Vertices(query::DbAccessor &dba) {
|
||||||
return Vertices(dba, [](const auto & /*vertex*/) { return true; });
|
// return Vertices(dba, [](const auto & /*vertex*/) { return true; });
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Read all vertices stored in the database and filter them by a lambda function.
|
// /// 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(query::DbAccessor &dba, const auto &vertex_filter) {
|
||||||
std::vector<query::VertexAccessor> vertices;
|
// std::vector<query::VertexAccessor> vertices;
|
||||||
auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), vertex_chandle));
|
// auto it = std::unique_ptr<rocksdb::Iterator>(db_->NewIterator(rocksdb::ReadOptions(), vertex_chandle));
|
||||||
for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
// for (it->SeekToFirst(); it->Valid(); it->Next()) {
|
||||||
auto vertex = DeserializeVertex(it->key().ToStringView(), it->value().ToStringView(), dba);
|
// auto vertex = DeserializeVertex(it->key().ToStringView(), it->value().ToStringView(), dba);
|
||||||
if (vertex_filter(vertex)) {
|
// if (vertex_filter(vertex)) {
|
||||||
vertices.push_back(vertex);
|
// vertices.push_back(vertex);
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
return vertices;
|
// return vertices;
|
||||||
}
|
// }
|
||||||
|
|
||||||
private:
|
// private:
|
||||||
/// Serialization of properties is done by saving the property store buffer
|
// /// 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 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
|
// /// 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; }
|
// inline std::string SerializeProperties(const auto &&properties) { return properties; }
|
||||||
|
|
||||||
/// Serialize labels delimitied by | to string
|
// /// Serialize labels delimitied by | to string
|
||||||
std::string SerializeLabels(const auto &&labels) {
|
// std::string SerializeLabels(const auto &&labels) {
|
||||||
if (labels.HasError() || (*labels).empty()) {
|
// if (labels.HasError() || (*labels).empty()) {
|
||||||
return "";
|
// return "";
|
||||||
}
|
// }
|
||||||
std::string result = std::to_string((*labels)[0].AsUint());
|
// std::string result = std::to_string((*labels)[0].AsUint());
|
||||||
std::string ser_labels = std::accumulate(
|
// std::string ser_labels = std::accumulate(
|
||||||
std::next((*labels).begin()), (*labels).end(), result,
|
// std::next((*labels).begin()), (*labels).end(), result,
|
||||||
[](const std::string &join, const auto &label_id) { return join + "," + std::to_string(label_id.AsUint()); });
|
// [](const std::string &join, const auto &label_id) { return join + "," + std::to_string(label_id.AsUint());
|
||||||
return ser_labels;
|
// });
|
||||||
}
|
// return ser_labels;
|
||||||
|
// }
|
||||||
|
|
||||||
/// Serializes id type to string
|
// /// Serializes id type to string
|
||||||
inline std::string SerializeIdType(const auto &id) { return std::to_string(id.AsUint()); }
|
// inline std::string SerializeIdType(const auto &id) { return std::to_string(id.AsUint()); }
|
||||||
|
|
||||||
/// Serialize vertex to string
|
// /// Serialize vertex to string
|
||||||
/// The format: | label1,label2,label3 | gid
|
// /// The format: | label1,label2,label3 | gid
|
||||||
std::string SerializeVertex(const query::VertexAccessor &vertex_acc) {
|
// std::string SerializeVertex(const query::VertexAccessor &vertex_acc) {
|
||||||
std::string result = SerializeLabels(vertex_acc.Labels(storage::View::OLD)) + "|";
|
// std::string result = SerializeLabels(vertex_acc.Labels(storage::View::OLD)) + "|";
|
||||||
result += SerializeIdType(vertex_acc.Gid());
|
// result += SerializeIdType(vertex_acc.Gid());
|
||||||
return result;
|
// return result;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Deserialize vertex from string
|
// /// Deserialize vertex from string
|
||||||
/// Properties are read from value and set to the vertex later
|
// /// 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::VertexAccessor DeserializeVertex(const std::string_view key, const std::string_view value,
|
||||||
query::DbAccessor &dba) {
|
// query::DbAccessor &dba) {
|
||||||
/// Create vertex
|
// /// Create vertex
|
||||||
auto impl = dba.InsertVertex();
|
// auto impl = dba.InsertVertex();
|
||||||
spdlog::info("Key to deserialize: {}", key);
|
// spdlog::info("Key to deserialize: {}", key);
|
||||||
const auto vertex_parts = utils::Split(key, "|");
|
// const auto vertex_parts = utils::Split(key, "|");
|
||||||
// Deserialize labels
|
// // Deserialize labels
|
||||||
if (!vertex_parts[0].empty()) {
|
// if (!vertex_parts[0].empty()) {
|
||||||
const auto labels = utils::Split(vertex_parts[0], ",");
|
// const auto labels = utils::Split(vertex_parts[0], ",");
|
||||||
for (const auto &label : labels) {
|
// for (const auto &label : labels) {
|
||||||
const storage::LabelId label_id = storage::LabelId::FromUint(std::stoull(label));
|
// const storage::LabelId label_id = storage::LabelId::FromUint(std::stoull(label));
|
||||||
auto maybe_error = impl.AddLabel(label_id);
|
// auto maybe_error = impl.AddLabel(label_id);
|
||||||
if (maybe_error.HasError()) {
|
// if (maybe_error.HasError()) {
|
||||||
switch (maybe_error.GetError()) {
|
// switch (maybe_error.GetError()) {
|
||||||
case storage::Error::SERIALIZATION_ERROR:
|
// case storage::Error::SERIALIZATION_ERROR:
|
||||||
throw utils::BasicException("Serialization");
|
// throw utils::BasicException("Serialization");
|
||||||
case storage::Error::DELETED_OBJECT:
|
// case storage::Error::DELETED_OBJECT:
|
||||||
throw utils::BasicException("Trying to set a label on a deleted node.");
|
// throw utils::BasicException("Trying to set a label on a deleted node.");
|
||||||
case storage::Error::VERTEX_HAS_EDGES:
|
// case storage::Error::VERTEX_HAS_EDGES:
|
||||||
case storage::Error::PROPERTIES_DISABLED:
|
// case storage::Error::PROPERTIES_DISABLED:
|
||||||
case storage::Error::NONEXISTENT_OBJECT:
|
// case storage::Error::NONEXISTENT_OBJECT:
|
||||||
throw utils::BasicException("Unexpected error when setting a label.");
|
// throw utils::BasicException("Unexpected error when setting a label.");
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
impl.SetGid(storage::Gid::FromUint(std::stoull(vertex_parts[1])));
|
// impl.SetGid(storage::Gid::FromUint(std::stoull(vertex_parts[1])));
|
||||||
impl.SetPropertyStore(value);
|
// impl.SetPropertyStore(value);
|
||||||
return impl;
|
// return impl;
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Serializes edge accessor to obtain a key for the key-value store.
|
// /// 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.
|
// /// @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
|
// // | from_gid | to_gid | direction | edge_type | edge_gid
|
||||||
std::pair<std::string, std::string> SerializeEdge(const query::EdgeAccessor &edge_acc) {
|
// std::pair<std::string, std::string> SerializeEdge(const query::EdgeAccessor &edge_acc) {
|
||||||
// Serialized objects
|
// // Serialized objects
|
||||||
auto from_gid = SerializeIdType(edge_acc.From().Gid());
|
// auto from_gid = SerializeIdType(edge_acc.From().Gid());
|
||||||
auto to_gid = SerializeIdType(edge_acc.To().Gid());
|
// auto to_gid = SerializeIdType(edge_acc.To().Gid());
|
||||||
auto edge_type = SerializeIdType(edge_acc.EdgeType());
|
// auto edge_type = SerializeIdType(edge_acc.EdgeType());
|
||||||
auto edge_gid = SerializeIdType(edge_acc.Gid());
|
// auto edge_gid = SerializeIdType(edge_acc.Gid());
|
||||||
// source->destination key
|
// // source->destination key
|
||||||
std::string src_dest_key = from_gid + "|";
|
// std::string src_dest_key = from_gid + "|";
|
||||||
src_dest_key += to_gid + "|";
|
// src_dest_key += to_gid + "|";
|
||||||
src_dest_key += outEdgeDirection;
|
// src_dest_key += outEdgeDirection;
|
||||||
src_dest_key += "|" + edge_type + "|";
|
// src_dest_key += "|" + edge_type + "|";
|
||||||
src_dest_key += edge_gid;
|
// src_dest_key += edge_gid;
|
||||||
// destination->source key
|
// // destination->source key
|
||||||
std::string dest_src_key = to_gid + "|";
|
// std::string dest_src_key = to_gid + "|";
|
||||||
dest_src_key += from_gid + "|";
|
// dest_src_key += from_gid + "|";
|
||||||
dest_src_key += inEdgeDirection;
|
// dest_src_key += inEdgeDirection;
|
||||||
dest_src_key += "|" + edge_type + "|";
|
// dest_src_key += "|" + edge_type + "|";
|
||||||
dest_src_key += edge_gid;
|
// dest_src_key += edge_gid;
|
||||||
return {src_dest_key, dest_src_key};
|
// return {src_dest_key, dest_src_key};
|
||||||
}
|
// }
|
||||||
|
|
||||||
/// Deserialize edge from the given key-value.
|
// /// Deserialize edge from the given key-value.
|
||||||
/// Properties are read from value and set to the edge later.
|
// /// 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::EdgeAccessor DeserializeEdge(const std::string_view key, const std::string_view value,
|
||||||
query::DbAccessor &dba) {
|
// query::DbAccessor &dba) {
|
||||||
const auto edge_parts = utils::Split(key, "|");
|
// const auto edge_parts = utils::Split(key, "|");
|
||||||
auto [from_gid, to_gid] = std::invoke(
|
// auto [from_gid, to_gid] = std::invoke(
|
||||||
[&](const auto &edge_parts) {
|
// [&](const auto &edge_parts) {
|
||||||
if (edge_parts[2] == "0") { // out edge
|
// if (edge_parts[2] == "0") { // out edge
|
||||||
return std::make_pair(edge_parts[0], edge_parts[1]);
|
// return std::make_pair(edge_parts[0], edge_parts[1]);
|
||||||
}
|
// }
|
||||||
// in edge
|
// // in edge
|
||||||
return std::make_pair(edge_parts[1], edge_parts[0]);
|
// return std::make_pair(edge_parts[1], edge_parts[0]);
|
||||||
},
|
// },
|
||||||
edge_parts);
|
// edge_parts);
|
||||||
// load vertex accessors
|
// // load vertex accessors
|
||||||
auto from_acc = FindVertex(from_gid, dba);
|
// auto from_acc = FindVertex(from_gid, dba);
|
||||||
auto to_acc = FindVertex(to_gid, dba);
|
// auto to_acc = FindVertex(to_gid, dba);
|
||||||
if (!from_acc.has_value() || !to_acc.has_value()) {
|
// if (!from_acc.has_value() || !to_acc.has_value()) {
|
||||||
throw utils::BasicException("Non-existing vertices during edge deserialization");
|
// throw utils::BasicException("Non-existing vertices during edge deserialization");
|
||||||
}
|
// }
|
||||||
const auto edge_type_id = storage::EdgeTypeId::FromUint(std::stoull(edge_parts[3]));
|
// 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);
|
// auto maybe_edge = dba.InsertEdge(&*from_acc, &*to_acc, edge_type_id);
|
||||||
MG_ASSERT(maybe_edge.HasValue());
|
// MG_ASSERT(maybe_edge.HasValue());
|
||||||
// in the new storage API, setting gid must be done atomically
|
// // in the new storage API, setting gid must be done atomically
|
||||||
maybe_edge->SetGid(storage::Gid::FromUint(std::stoull(edge_parts[4])));
|
// maybe_edge->SetGid(storage::Gid::FromUint(std::stoull(edge_parts[4])));
|
||||||
maybe_edge->SetPropertyStore(value);
|
// maybe_edge->SetPropertyStore(value);
|
||||||
return *maybe_edge;
|
// return *maybe_edge;
|
||||||
}
|
// }
|
||||||
|
|
||||||
rocksdb::Options options_;
|
rocksdb::Options options_;
|
||||||
rocksdb::DB *db_;
|
rocksdb::DB *db_;
|
||||||
|
621
src/storage/v2/disk/vertex_accessor.cpp
Normal file
621
src/storage/v2/disk/vertex_accessor.cpp
Normal 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, °ree](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, °ree](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
|
117
src/storage/v2/disk/vertex_accessor.hpp
Normal file
117
src/storage/v2/disk/vertex_accessor.hpp
Normal 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
|
@ -10,253 +10,16 @@
|
|||||||
// licenses/APL.txt.
|
// licenses/APL.txt.
|
||||||
|
|
||||||
#include "storage/v2/edge_accessor.hpp"
|
#include "storage/v2/edge_accessor.hpp"
|
||||||
|
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <tuple>
|
|
||||||
|
|
||||||
#include "storage/v2/mvcc.hpp"
|
#include "storage/v2/inmemory/edge_accessor.hpp"
|
||||||
#include "storage/v2/property_value.hpp"
|
|
||||||
#include "storage/v2/vertex_accessor.hpp"
|
|
||||||
#include "utils/memory_tracker.hpp"
|
|
||||||
|
|
||||||
namespace memgraph::storage {
|
namespace memgraph::storage {
|
||||||
|
|
||||||
bool EdgeAccessor::IsVisible(const View view) const {
|
std::unique_ptr<EdgeAccessor> EdgeAccessor::Create(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex,
|
||||||
bool exists = true;
|
Vertex *to_vertex, Transaction *transaction, Indices *indices,
|
||||||
bool deleted = true;
|
Constraints *constraints, Config::Items config, bool for_deleted) {
|
||||||
// When edges don't have properties, their isolation level is still dictated by MVCC ->
|
return std::make_unique<InMemoryEdgeAccessor>(edge, edge_type, from_vertex, to_vertex, transaction, indices,
|
||||||
// iterate over the deltas of the from_vertex_ and see which deltas can be applied on edges.
|
constraints, config, for_deleted);
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
return std::move(properties);
|
|
||||||
/*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);
|
|
||||||
*/
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string EdgeAccessor::PropertyStore() const { return edge_.ptr->properties.StringBuffer(); }
|
|
||||||
|
|
||||||
void EdgeAccessor::SetPropertyStore(std::string_view buffer) const { edge_.ptr->properties.SetBuffer(buffer); }
|
|
||||||
|
|
||||||
} // namespace memgraph::storage
|
} // namespace memgraph::storage
|
||||||
|
@ -11,103 +11,73 @@
|
|||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <memory>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
|
|
||||||
#include "storage/v2/edge.hpp"
|
#include "storage/v2/edge.hpp"
|
||||||
|
#include "storage/v2/edge_accessor.hpp"
|
||||||
#include "storage/v2/edge_ref.hpp"
|
#include "storage/v2/edge_ref.hpp"
|
||||||
|
|
||||||
#include "storage/v2/config.hpp"
|
#include "storage/v2/config.hpp"
|
||||||
#include "storage/v2/result.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 {
|
namespace memgraph::storage {
|
||||||
|
|
||||||
struct Vertex;
|
struct Transaction;
|
||||||
class VertexAccessor;
|
|
||||||
struct Indices;
|
|
||||||
struct Constraints;
|
|
||||||
|
|
||||||
class EdgeAccessor final {
|
class EdgeAccessor {
|
||||||
private:
|
private:
|
||||||
friend class Storage;
|
friend class Storage;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
|
EdgeAccessor(EdgeTypeId edge_type, Transaction *transaction, Config::Items config, bool for_deleted = false)
|
||||||
Indices *indices, Constraints *constraints, Config::Items config, bool for_deleted = false)
|
: edge_type_(edge_type), transaction_(transaction), config_(config), for_deleted_(for_deleted) {}
|
||||||
: edge_(edge),
|
|
||||||
edge_type_(edge_type),
|
virtual ~EdgeAccessor() {}
|
||||||
from_vertex_(from_vertex),
|
|
||||||
to_vertex_(to_vertex),
|
static std::unique_ptr<EdgeAccessor> Create(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex,
|
||||||
transaction_(transaction),
|
Vertex *to_vertex, Transaction *transaction, Indices *indices,
|
||||||
indices_(indices),
|
Constraints *constraints, Config::Items config, bool for_deleted = false);
|
||||||
constraints_(constraints),
|
|
||||||
config_(config),
|
|
||||||
for_deleted_(for_deleted) {}
|
|
||||||
|
|
||||||
/// @return true if the object is visible from the current transaction
|
/// @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_; }
|
EdgeTypeId EdgeType() const { return edge_type_; }
|
||||||
|
|
||||||
/// Set a property value and return the old value.
|
/// Set a property value and return the old value.
|
||||||
/// @throw std::bad_alloc
|
/// @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,
|
/// Set property values only if property store is empty. Returns `true` if successully set all values,
|
||||||
/// `false` otherwise.
|
/// `false` otherwise.
|
||||||
/// @throw std::bad_alloc
|
/// @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.
|
/// Remove all properties and return old values for each removed property.
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
|
virtual Result<std::map<PropertyId, PropertyValue>> ClearProperties() = 0;
|
||||||
|
|
||||||
/// @throw std::bad_alloc
|
/// @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
|
/// @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;
|
virtual storage::Gid Gid() const noexcept = 0;
|
||||||
|
|
||||||
void SetPropertyStore(std::string_view buffer) const;
|
virtual bool IsCycle() const = 0;
|
||||||
|
|
||||||
Gid Gid() const noexcept {
|
virtual bool operator==(const EdgeAccessor &other) const noexcept = 0;
|
||||||
if (config_.properties_on_edges) {
|
|
||||||
return edge_.ptr->gid;
|
|
||||||
} else {
|
|
||||||
return edge_.gid;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void SetGid(storage::Gid gid_) {
|
|
||||||
if (config_.properties_on_edges) {
|
|
||||||
edge_.ptr->gid = storage::Gid::FromUint(gid_.AsUint());
|
|
||||||
} else {
|
|
||||||
edge_.gid = storage::Gid::FromUint(gid_.AsUint());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bool IsCycle() const { return from_vertex_ == to_vertex_; }
|
|
||||||
|
|
||||||
bool operator==(const EdgeAccessor &other) const noexcept {
|
|
||||||
return edge_ == other.edge_ && transaction_ == other.transaction_;
|
|
||||||
}
|
|
||||||
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
|
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
|
||||||
|
|
||||||
private:
|
protected:
|
||||||
EdgeRef edge_;
|
|
||||||
EdgeTypeId edge_type_;
|
EdgeTypeId edge_type_;
|
||||||
Vertex *from_vertex_;
|
|
||||||
Vertex *to_vertex_;
|
|
||||||
Transaction *transaction_;
|
Transaction *transaction_;
|
||||||
Indices *indices_;
|
|
||||||
Constraints *constraints_;
|
|
||||||
Config::Items config_;
|
Config::Items config_;
|
||||||
|
|
||||||
// if the accessor was created for a deleted edge.
|
// if the accessor was created for a deleted edge.
|
||||||
|
258
src/storage/v2/inmemory/edge_accessor.cpp
Normal file
258
src/storage/v2/inmemory/edge_accessor.cpp
Normal 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
|
102
src/storage/v2/inmemory/edge_accessor.hpp
Normal file
102
src/storage/v2/inmemory/edge_accessor.hpp
Normal 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
|
1986
src/storage/v2/inmemory/storage.cpp
Normal file
1986
src/storage/v2/inmemory/storage.cpp
Normal file
File diff suppressed because it is too large
Load Diff
526
src/storage/v2/inmemory/storage.hpp
Normal file
526
src/storage/v2/inmemory/storage.hpp
Normal 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
|
604
src/storage/v2/inmemory/vertex_accessor.cpp
Normal file
604
src/storage/v2/inmemory/vertex_accessor.cpp
Normal 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, °ree](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, °ree](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
|
117
src/storage/v2/inmemory/vertex_accessor.hpp
Normal file
117
src/storage/v2/inmemory/vertex_accessor.hpp
Normal 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
|
File diff suppressed because it is too large
Load Diff
@ -11,49 +11,27 @@
|
|||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <atomic>
|
#include <set>
|
||||||
#include <cstdint>
|
|
||||||
#include <filesystem>
|
|
||||||
#include <optional>
|
|
||||||
#include <shared_mutex>
|
|
||||||
#include <span>
|
#include <span>
|
||||||
#include <variant>
|
|
||||||
|
|
||||||
#include "io/network/endpoint.hpp"
|
#include "io/network/endpoint.hpp"
|
||||||
#include "kvstore/kvstore.hpp"
|
|
||||||
#include "storage/v2/commit_log.hpp"
|
|
||||||
#include "storage/v2/config.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/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/result.hpp"
|
||||||
#include "storage/v2/transaction.hpp"
|
#include "storage/v2/storage_error.hpp"
|
||||||
#include "storage/v2/vertex.hpp"
|
|
||||||
#include "storage/v2/vertex_accessor.hpp"
|
#include "storage/v2/vertex_accessor.hpp"
|
||||||
#include "utils/file_locker.hpp"
|
#include "storage/v2/view.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/config.hpp"
|
||||||
#include "storage/v2/replication/enums.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 {
|
namespace memgraph::storage {
|
||||||
|
|
||||||
|
struct Transaction;
|
||||||
|
class EdgeAccessor;
|
||||||
|
|
||||||
|
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
|
||||||
|
|
||||||
// The storage is based on this paper:
|
// The storage is based on this paper:
|
||||||
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
|
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
|
||||||
// The paper implements a fully serializable storage, in our implementation we
|
// The paper implements a fully serializable storage, in our implementation we
|
||||||
@ -70,7 +48,7 @@ class AllVerticesIterable final {
|
|||||||
Indices *indices_;
|
Indices *indices_;
|
||||||
Constraints *constraints_;
|
Constraints *constraints_;
|
||||||
Config::Items config_;
|
Config::Items config_;
|
||||||
std::optional<VertexAccessor> vertex_;
|
std::unique_ptr<VertexAccessor> vertex_;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
class Iterator final {
|
class Iterator final {
|
||||||
@ -80,7 +58,7 @@ class AllVerticesIterable final {
|
|||||||
public:
|
public:
|
||||||
Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it);
|
Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it);
|
||||||
|
|
||||||
VertexAccessor operator*() const;
|
VertexAccessor *operator*() const;
|
||||||
|
|
||||||
Iterator &operator++();
|
Iterator &operator++();
|
||||||
|
|
||||||
@ -152,7 +130,7 @@ class VerticesIterable final {
|
|||||||
|
|
||||||
~Iterator();
|
~Iterator();
|
||||||
|
|
||||||
VertexAccessor operator*() const;
|
VertexAccessor *operator*() const;
|
||||||
|
|
||||||
Iterator &operator++();
|
Iterator &operator++();
|
||||||
|
|
||||||
@ -186,23 +164,11 @@ struct StorageInfo {
|
|||||||
uint64_t disk_usage;
|
uint64_t disk_usage;
|
||||||
};
|
};
|
||||||
|
|
||||||
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
|
class Storage {
|
||||||
|
|
||||||
class Storage final {
|
|
||||||
public:
|
public:
|
||||||
/// @throw std::system_error
|
class Accessor {
|
||||||
/// @throw std::bad_alloc
|
|
||||||
explicit Storage(Config config = Config());
|
|
||||||
|
|
||||||
~Storage();
|
|
||||||
|
|
||||||
class Accessor final {
|
|
||||||
private:
|
|
||||||
friend class Storage;
|
|
||||||
|
|
||||||
explicit Accessor(Storage *storage, IsolationLevel isolation_level);
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
|
Accessor() {}
|
||||||
Accessor(const Accessor &) = delete;
|
Accessor(const Accessor &) = delete;
|
||||||
Accessor &operator=(const Accessor &) = delete;
|
Accessor &operator=(const Accessor &) = delete;
|
||||||
Accessor &operator=(Accessor &&other) = 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.
|
// and iterators) are *invalid*. You have to get all derived objects again.
|
||||||
Accessor(Accessor &&other) noexcept;
|
Accessor(Accessor &&other) noexcept;
|
||||||
|
|
||||||
~Accessor();
|
virtual ~Accessor() {}
|
||||||
|
|
||||||
/// @throw std::bad_alloc
|
/// @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) {
|
virtual VerticesIterable Vertices(View view) = 0;
|
||||||
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
|
|
||||||
&storage_->indices_, &storage_->constraints_,
|
|
||||||
storage_->config_.items));
|
|
||||||
}
|
|
||||||
|
|
||||||
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,
|
virtual VerticesIterable Vertices(LabelId label, PropertyId property,
|
||||||
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
|
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
|
||||||
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
|
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) = 0;
|
||||||
|
|
||||||
/// Return approximate number of all vertices in the database.
|
/// Return approximate number of all vertices in the database.
|
||||||
/// Note that this is always an over-estimate and never an under-estimate.
|
/// 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.
|
/// Return approximate number of vertices with the given label.
|
||||||
/// Note that this is always an over-estimate and never an under-estimate.
|
/// Note that this is always an over-estimate and never an under-estimate.
|
||||||
int64_t ApproximateVertexCount(LabelId label) const {
|
virtual int64_t ApproximateVertexCount(LabelId label) const = 0;
|
||||||
return storage_->indices_.label_index.ApproximateVertexCount(label);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Return approximate number of vertices with the given label and property.
|
/// Return approximate number of vertices with the given label and property.
|
||||||
/// Note that this is always an over-estimate and never an under-estimate.
|
/// Note that this is always an over-estimate and never an under-estimate.
|
||||||
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
|
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property) const = 0;
|
||||||
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Return approximate number of vertices with the given label and the given
|
/// 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
|
/// value for the given property. Note that this is always an over-estimate
|
||||||
/// and never an under-estimate.
|
/// and never an under-estimate.
|
||||||
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
|
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const = 0;
|
||||||
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Return approximate number of vertices with the given label and value for
|
/// Return approximate number of vertices with the given label and value for
|
||||||
/// the given property in the range defined by provided upper and lower
|
/// the given property in the range defined by provided upper and lower
|
||||||
/// bounds.
|
/// bounds.
|
||||||
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
|
virtual int64_t ApproximateVertexCount(LabelId label, PropertyId property,
|
||||||
const std::optional<utils::Bound<PropertyValue>> &lower,
|
const std::optional<utils::Bound<PropertyValue>> &lower,
|
||||||
const std::optional<utils::Bound<PropertyValue>> &upper) const {
|
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
|
||||||
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
|
virtual std::optional<storage::IndexStats> GetIndexStats(const storage::LabelId &label,
|
||||||
const storage::PropertyId &property) const {
|
const storage::PropertyId &property) const = 0;
|
||||||
return storage_->indices_.label_property_index.GetIndexStats(label, property);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() {
|
virtual std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats() = 0;
|
||||||
return storage_->indices_.label_property_index.ClearIndexStats();
|
|
||||||
}
|
|
||||||
|
|
||||||
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(const std::span<std::string> labels) {
|
virtual std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStatsForLabels(
|
||||||
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
|
const std::span<std::string> labels) = 0;
|
||||||
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) {
|
virtual void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
|
||||||
storage_->indices_.label_property_index.SetIndexStats(label, property, stats);
|
const IndexStats &stats) = 0;
|
||||||
}
|
|
||||||
|
|
||||||
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
|
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
|
||||||
/// @throw std::bad_alloc
|
/// @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
|
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
|
virtual Result<
|
||||||
VertexAccessor *vertex);
|
std::optional<std::pair<std::unique_ptr<VertexAccessor>, std::vector<std::unique_ptr<EdgeAccessor>>>>>
|
||||||
|
DetachDeleteVertex(VertexAccessor *vertex) = 0;
|
||||||
|
|
||||||
/// @throw std::bad_alloc
|
/// @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
|
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
|
||||||
/// @throw std::bad_alloc
|
/// @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;
|
virtual const std::string &LabelToName(LabelId label) const = 0;
|
||||||
const std::string &PropertyToName(PropertyId property) const;
|
virtual const std::string &PropertyToName(PropertyId property) const = 0;
|
||||||
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
|
virtual const std::string &EdgeTypeToName(EdgeTypeId edge_type) const = 0;
|
||||||
|
|
||||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
/// @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
|
/// @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
|
/// @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 {
|
virtual bool LabelPropertyIndexExists(LabelId label, PropertyId property) const = 0;
|
||||||
return storage_->indices_.label_property_index.IndexExists(label, property);
|
|
||||||
}
|
|
||||||
|
|
||||||
IndicesInfo ListAllIndices() const {
|
virtual IndicesInfo ListAllIndices() const = 0;
|
||||||
return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
|
|
||||||
}
|
|
||||||
|
|
||||||
ConstraintsInfo ListAllConstraints() const {
|
virtual ConstraintsInfo ListAllConstraints() const = 0;
|
||||||
return {ListExistenceConstraints(storage_->constraints_),
|
|
||||||
storage_->constraints_.unique_constraints.ListConstraints()};
|
|
||||||
}
|
|
||||||
|
|
||||||
void AdvanceCommand();
|
virtual void AdvanceCommand() = 0;
|
||||||
|
|
||||||
/// Returns void if the transaction has been committed.
|
/// Returns void if the transaction has been committed.
|
||||||
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
|
/// 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
|
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
|
||||||
/// case the transaction is automatically aborted.
|
/// case the transaction is automatically aborted.
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
utils::BasicResult<StorageDataManipulationError, void> Commit(
|
virtual utils::BasicResult<StorageDataManipulationError, void> Commit(
|
||||||
std::optional<uint64_t> desired_commit_timestamp = {});
|
std::optional<uint64_t> desired_commit_timestamp = {}) = 0;
|
||||||
|
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
void Abort();
|
virtual void Abort() = 0;
|
||||||
|
|
||||||
void FinalizeTransaction();
|
virtual void FinalizeTransaction() = 0;
|
||||||
|
|
||||||
std::optional<uint64_t> GetTransactionId() const;
|
virtual std::optional<uint64_t> GetTransactionId() const = 0;
|
||||||
|
|
||||||
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_;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
|
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) = 0;
|
||||||
return Accessor{this, override_isolation_level.value_or(isolation_level_)};
|
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.
|
/// Create an index.
|
||||||
/// Returns void if the index has been created.
|
/// Returns void if the index has been created.
|
||||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||||
/// * `IndexDefinitionError`: the index already exists.
|
/// * `IndexDefinitionError`: the index already exists.
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
|
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
|
||||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
|
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.
|
/// Drop an existing index.
|
||||||
/// Returns void if the index has been dropped.
|
/// Returns void if the index has been dropped.
|
||||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||||
/// * `IndexDefinitionError`: the index does not exist.
|
/// * `IndexDefinitionError`: the index does not exist.
|
||||||
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
||||||
LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
|
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.
|
/// Drop an existing index.
|
||||||
/// Returns void if the index has been dropped.
|
/// Returns void if the index has been dropped.
|
||||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||||
/// * `IndexDefinitionError`: the index does not exist.
|
/// * `IndexDefinitionError`: the index does not exist.
|
||||||
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
||||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
|
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 void if the existence constraint has been created.
|
||||||
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
|
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
|
||||||
@ -427,16 +364,26 @@ class Storage final {
|
|||||||
/// * `ConstraintDefinitionError`: the constraint already exists.
|
/// * `ConstraintDefinitionError`: the constraint already exists.
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
/// @throw std::length_error
|
/// @throw std::length_error
|
||||||
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
|
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
|
||||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
|
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.
|
/// Drop an existing existence constraint.
|
||||||
/// Returns void if the existence constraint has been dropped.
|
/// Returns void if the existence constraint has been dropped.
|
||||||
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
|
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
|
||||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||||
/// * `ConstraintDefinitionError`: the constraint did not exists.
|
/// * `ConstraintDefinitionError`: the constraint did not exists.
|
||||||
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
|
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
|
||||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
|
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.
|
/// Create an unique constraint.
|
||||||
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
|
/// 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
|
/// * `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.
|
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
|
||||||
/// @throw std::bad_alloc
|
/// @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(
|
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.
|
/// Removes an existing unique constraint.
|
||||||
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
|
/// 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,
|
/// * `NOT_FOUND` if the specified constraint was not found,
|
||||||
/// * `EMPTY_PROPERTIES` if the property set is empty, or
|
/// * `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.
|
/// * `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(
|
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();
|
virtual bool LockPath() = 0;
|
||||||
bool UnlockPath();
|
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 {
|
enum class RegisterReplicaError : uint8_t {
|
||||||
NAME_EXISTS,
|
NAME_EXISTS,
|
||||||
@ -482,15 +445,22 @@ class Storage final {
|
|||||||
|
|
||||||
/// @pre The instance should have a MAIN role
|
/// @pre The instance should have a MAIN role
|
||||||
/// @pre Timeout can only be set for SYNC replication
|
/// @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,
|
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
|
/// @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 {
|
struct TimestampInfo {
|
||||||
uint64_t current_timestamp_of_replica;
|
uint64_t current_timestamp_of_replica;
|
||||||
@ -505,167 +475,15 @@ class Storage final {
|
|||||||
TimestampInfo timestamp_info;
|
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 };
|
enum class CreateSnapshotError : uint8_t { DisabledForReplica };
|
||||||
|
|
||||||
utils::BasicResult<CreateSnapshotError> CreateSnapshot();
|
virtual utils::BasicResult<CreateSnapshotError> CreateSnapshot() = 0;
|
||||||
|
|
||||||
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
|
} // namespace memgraph::storage
|
||||||
|
@ -11,590 +11,32 @@
|
|||||||
|
|
||||||
#include "storage/v2/vertex_accessor.hpp"
|
#include "storage/v2/vertex_accessor.hpp"
|
||||||
|
|
||||||
#include <memory>
|
#include "storage/v2/inmemory/edge_accessor.hpp"
|
||||||
|
#include "storage/v2/inmemory/vertex_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/logging.hpp"
|
|
||||||
#include "utils/memory_tracker.hpp"
|
|
||||||
|
|
||||||
namespace memgraph::storage {
|
namespace memgraph::storage {
|
||||||
|
|
||||||
namespace detail {
|
std::unique_ptr<VertexAccessor> VertexAccessor::Create(Vertex *vertex, Transaction *transaction, Indices *indices,
|
||||||
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) {
|
Constraints *constraints, Config::Items config, View view) {
|
||||||
if (const auto [exists, deleted] = detail::IsVisible(vertex, transaction, view); !exists || deleted) {
|
return InMemoryVertexAccessor::Create(vertex, transaction, indices, constraints, config, view);
|
||||||
return std::nullopt;
|
|
||||||
}
|
|
||||||
|
|
||||||
return VertexAccessor{vertex, transaction, indices, constraints, config};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
bool VertexAccessor::IsVisible(View view) const {
|
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::InEdges(
|
||||||
const auto [exists, deleted] = detail::IsVisible(vertex_, transaction_, view);
|
View view, const std::vector<EdgeTypeId> &edge_types) const {
|
||||||
return exists && (for_deleted_ || !deleted);
|
return InEdges(view, edge_types, nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
Result<bool> VertexAccessor::AddLabel(LabelId label) {
|
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::InEdges(View view) const {
|
||||||
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
|
return InEdges(view, {}, nullptr);
|
||||||
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> VertexAccessor::RemoveLabel(LabelId label) {
|
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::OutEdges(
|
||||||
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
|
View view, const std::vector<EdgeTypeId> &edge_types) const {
|
||||||
|
return OutEdges(view, edge_types, nullptr);
|
||||||
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> VertexAccessor::HasLabel(LabelId label, View view) const {
|
Result<std::vector<std::unique_ptr<EdgeAccessor>>> VertexAccessor::OutEdges(View view) const {
|
||||||
bool exists = true;
|
return OutEdges(view, {}, nullptr);
|
||||||
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);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string VertexAccessor::PropertyStore() const { return vertex_->properties.StringBuffer(); }
|
|
||||||
|
|
||||||
void VertexAccessor::SetPropertyStore(const std::string_view buffer) const { vertex_->properties.SetBuffer(buffer); }
|
|
||||||
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
// return std::move(properties);
|
|
||||||
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, °ree](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, °ree](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
|
} // namespace memgraph::storage
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
#include "storage/v2/vertex.hpp"
|
#include "storage/v2/vertex.hpp"
|
||||||
|
|
||||||
#include "storage/v2/config.hpp"
|
#include "storage/v2/config.hpp"
|
||||||
|
#include "storage/v2/constraints.hpp"
|
||||||
#include "storage/v2/result.hpp"
|
#include "storage/v2/result.hpp"
|
||||||
#include "storage/v2/transaction.hpp"
|
#include "storage/v2/transaction.hpp"
|
||||||
#include "storage/v2/view.hpp"
|
#include "storage/v2/view.hpp"
|
||||||
@ -26,65 +27,59 @@
|
|||||||
namespace memgraph::storage {
|
namespace memgraph::storage {
|
||||||
|
|
||||||
class EdgeAccessor;
|
class EdgeAccessor;
|
||||||
class Storage;
|
|
||||||
struct Indices;
|
struct Indices;
|
||||||
struct Constraints;
|
|
||||||
|
|
||||||
class VertexAccessor final {
|
class VertexAccessor {
|
||||||
private:
|
private:
|
||||||
friend class Storage;
|
friend class Storage;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
VertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
|
VertexAccessor(Transaction *transaction, Config::Items config, bool for_deleted = false)
|
||||||
Config::Items config, bool for_deleted = false)
|
: transaction_(transaction), config_(config), for_deleted_(for_deleted) {}
|
||||||
: vertex_(vertex),
|
|
||||||
transaction_(transaction),
|
|
||||||
indices_(indices),
|
|
||||||
constraints_(constraints),
|
|
||||||
config_(config),
|
|
||||||
for_deleted_(for_deleted) {}
|
|
||||||
|
|
||||||
static std::optional<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
|
virtual ~VertexAccessor() {}
|
||||||
|
|
||||||
|
static std::unique_ptr<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
|
||||||
Constraints *constraints, Config::Items config, View view);
|
Constraints *constraints, Config::Items config, View view);
|
||||||
|
|
||||||
/// @return true if the object is visible from the current transaction
|
/// @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.
|
/// Add a label and return `true` if insertion took place.
|
||||||
/// `false` is returned if the label already existed.
|
/// `false` is returned if the label already existed.
|
||||||
/// @throw std::bad_alloc
|
/// @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.
|
/// Remove a label and return `true` if deletion took place.
|
||||||
/// `false` is returned if the vertex did not have a label already.
|
/// `false` is returned if the vertex did not have a label already.
|
||||||
/// @throw std::bad_alloc
|
/// @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::bad_alloc
|
||||||
/// @throw std::length_error if the resulting vector exceeds
|
/// @throw std::length_error if the resulting vector exceeds
|
||||||
/// std::vector::max_size().
|
/// 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.
|
/// Set a property value and return the old value.
|
||||||
/// @throw std::bad_alloc
|
/// @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,
|
/// Set property values only if property store is empty. Returns `true` if successully set all values,
|
||||||
/// `false` otherwise.
|
/// `false` otherwise.
|
||||||
/// @throw std::bad_alloc
|
/// @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.
|
/// Remove all properties and return the values of the removed properties.
|
||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
|
virtual Result<std::map<PropertyId, PropertyValue>> ClearProperties() = 0;
|
||||||
|
|
||||||
/// @throw std::bad_alloc
|
/// @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
|
/// @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;
|
std::string PropertyStore() const;
|
||||||
|
|
||||||
@ -93,33 +88,37 @@ class VertexAccessor final {
|
|||||||
/// @throw std::bad_alloc
|
/// @throw std::bad_alloc
|
||||||
/// @throw std::length_error if the resulting vector exceeds
|
/// @throw std::length_error if the resulting vector exceeds
|
||||||
/// std::vector::max_size().
|
/// std::vector::max_size().
|
||||||
Result<std::vector<EdgeAccessor>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
|
virtual Result<std::vector<std::unique_ptr<EdgeAccessor>>> InEdges(View view,
|
||||||
const VertexAccessor *destination = nullptr) const;
|
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::bad_alloc
|
||||||
/// @throw std::length_error if the resulting vector exceeds
|
/// @throw std::length_error if the resulting vector exceeds
|
||||||
/// std::vector::max_size().
|
/// std::vector::max_size().
|
||||||
Result<std::vector<EdgeAccessor>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
|
virtual Result<std::vector<std::unique_ptr<EdgeAccessor>>> OutEdges(View view,
|
||||||
const VertexAccessor *destination = nullptr) const;
|
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;
|
||||||
|
|
||||||
void SetGid(storage::Gid gid_) { vertex_->gid = storage::Gid::FromUint(gid_.AsUint()); }
|
virtual Gid Gid() const noexcept = 0;
|
||||||
|
|
||||||
bool operator==(const VertexAccessor &other) const noexcept {
|
virtual bool operator==(const VertexAccessor &other) const noexcept = 0;
|
||||||
return vertex_ == other.vertex_ && transaction_ == other.transaction_;
|
|
||||||
}
|
|
||||||
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
|
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
|
||||||
|
|
||||||
private:
|
protected:
|
||||||
Vertex *vertex_;
|
|
||||||
Transaction *transaction_;
|
Transaction *transaction_;
|
||||||
Indices *indices_;
|
|
||||||
Constraints *constraints_;
|
|
||||||
Config::Items config_;
|
Config::Items config_;
|
||||||
|
|
||||||
// if the accessor was created for a deleted vertex.
|
// if the accessor was created for a deleted vertex.
|
||||||
|
Loading…
Reference in New Issue
Block a user