Reference vertices by id in edges (#532)
This commit is contained in:
parent
f910cb770c
commit
8e1f83acc9
src
query/v2
storage/v3
tests/unit
@ -168,7 +168,8 @@ class VertexAccessor final {
|
||||
auto InEdges(storage::v3::View view, const std::vector<storage::v3::EdgeTypeId> &edge_types,
|
||||
const VertexAccessor &dest) const
|
||||
-> storage::v3::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_.InEdges(view)))> {
|
||||
auto maybe_edges = impl_.InEdges(view, edge_types, &dest.impl_);
|
||||
const auto dest_id = dest.impl_.Id(view).GetValue();
|
||||
auto maybe_edges = impl_.InEdges(view, edge_types, &dest_id);
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
|
||||
}
|
||||
@ -185,7 +186,8 @@ class VertexAccessor final {
|
||||
auto OutEdges(storage::v3::View view, const std::vector<storage::v3::EdgeTypeId> &edge_types,
|
||||
const VertexAccessor &dest) const
|
||||
-> storage::v3::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_.OutEdges(view)))> {
|
||||
auto maybe_edges = impl_.OutEdges(view, edge_types, &dest.impl_);
|
||||
const auto dest_id = dest.impl_.Id(view).GetValue();
|
||||
auto maybe_edges = impl_.OutEdges(view, edge_types, &dest_id);
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
|
||||
}
|
||||
@ -205,9 +207,14 @@ class VertexAccessor final {
|
||||
bool operator!=(const VertexAccessor &v) const noexcept { return !(*this == v); }
|
||||
};
|
||||
|
||||
inline VertexAccessor EdgeAccessor::To() const { return VertexAccessor(impl_.ToVertex()); }
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wnull-dereference"
|
||||
// NOLINTNEXTLINE(readability-convert-member-functions-to-static,clang-analyzer-core.NonNullParamChecker)
|
||||
inline VertexAccessor EdgeAccessor::To() const { return *static_cast<VertexAccessor *>(nullptr); }
|
||||
|
||||
inline VertexAccessor EdgeAccessor::From() const { return VertexAccessor(impl_.FromVertex()); }
|
||||
// NOLINTNEXTLINE(readability-convert-member-functions-to-static,clang-analyzer-core.NonNullParamChecker)
|
||||
inline VertexAccessor EdgeAccessor::From() const { return *static_cast<VertexAccessor *>(nullptr); }
|
||||
#pragma clang diagnostic pop
|
||||
|
||||
inline bool EdgeAccessor::IsCycle() const { return To() == From(); }
|
||||
|
||||
@ -291,13 +298,15 @@ class DbAccessor final {
|
||||
|
||||
storage::v3::Result<EdgeAccessor> InsertEdge(VertexAccessor *from, VertexAccessor *to,
|
||||
const storage::v3::EdgeTypeId &edge_type) {
|
||||
auto maybe_edge = accessor_->CreateEdge(&from->impl_, &to->impl_, edge_type);
|
||||
static constexpr auto kDummyGid = storage::v3::Gid::FromUint(0);
|
||||
auto maybe_edge = accessor_->CreateEdge(from->impl_.Id(storage::v3::View::NEW).GetValue(),
|
||||
to->impl_.Id(storage::v3::View::NEW).GetValue(), edge_type, kDummyGid);
|
||||
if (maybe_edge.HasError()) return storage::v3::Result<EdgeAccessor>(maybe_edge.GetError());
|
||||
return EdgeAccessor(*maybe_edge);
|
||||
}
|
||||
|
||||
storage::v3::Result<std::optional<EdgeAccessor>> RemoveEdge(EdgeAccessor *edge) {
|
||||
auto res = accessor_->DeleteEdge(&edge->impl_);
|
||||
auto res = accessor_->DeleteEdge(edge->impl_.FromVertex(), edge->impl_.ToVertex(), edge->impl_.Gid());
|
||||
if (res.HasError()) {
|
||||
return res.GetError();
|
||||
}
|
||||
|
@ -474,9 +474,10 @@ struct mgp_edge {
|
||||
/// the allocator which was used to allocate `this`.
|
||||
using allocator_type = memgraph::utils::Allocator<mgp_edge>;
|
||||
|
||||
// TODO(antaljanosbenjamin): Handle this static assert failure when we will support procedures again
|
||||
// Hopefully EdgeAccessor copy constructor remains noexcept, so that we can
|
||||
// have everything noexcept here.
|
||||
static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::v2::EdgeAccessor>);
|
||||
// static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::v2::EdgeAccessor>);
|
||||
|
||||
static mgp_edge *Copy(const mgp_edge &edge, mgp_memory &memory);
|
||||
|
||||
|
@ -166,8 +166,10 @@ const char *TriggerEventTypeToString(TriggerEventType event_type);
|
||||
|
||||
static_assert(std::is_trivially_copy_constructible_v<VertexAccessor>,
|
||||
"VertexAccessor is not trivially copy constructible, move it where possible and remove this assert");
|
||||
static_assert(std::is_trivially_copy_constructible_v<EdgeAccessor>,
|
||||
"EdgeAccessor is not trivially copy constructible, move it where possible and remove this asssert");
|
||||
// TODO(antaljanosbenjamin): Either satisfy this static_assert or move the edge accessors where it is possible when we
|
||||
// will support triggers.
|
||||
// static_assert(std::is_trivially_copy_constructible_v<EdgeAccessor>,
|
||||
// "EdgeAccessor is not trivially copy constructible, move it where possible and remove this asssert");
|
||||
|
||||
// Holds the information necessary for triggers
|
||||
class TriggerContext {
|
||||
|
@ -12,10 +12,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
|
||||
#include "storage/v3/edge_ref.hpp"
|
||||
#include "storage/v3/id_types.hpp"
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/vertex_id.hpp"
|
||||
#include "utils/logging.hpp"
|
||||
|
||||
namespace memgraph::storage::v3 {
|
||||
@ -173,33 +175,33 @@ struct Delta {
|
||||
uint64_t command_id)
|
||||
: action(Action::SET_PROPERTY), timestamp(timestamp), command_id(command_id), property({key, value}) {}
|
||||
|
||||
Delta(AddInEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
|
||||
uint64_t command_id)
|
||||
Delta(AddInEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
|
||||
std::atomic<uint64_t> *timestamp, uint64_t command_id)
|
||||
: action(Action::ADD_IN_EDGE),
|
||||
timestamp(timestamp),
|
||||
command_id(command_id),
|
||||
vertex_edge({edge_type, vertex, edge}) {}
|
||||
vertex_edge({edge_type, std::move(vertex_id), edge}) {}
|
||||
|
||||
Delta(AddOutEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
|
||||
uint64_t command_id)
|
||||
Delta(AddOutEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
|
||||
std::atomic<uint64_t> *timestamp, uint64_t command_id)
|
||||
: action(Action::ADD_OUT_EDGE),
|
||||
timestamp(timestamp),
|
||||
command_id(command_id),
|
||||
vertex_edge({edge_type, vertex, edge}) {}
|
||||
vertex_edge({edge_type, std::move(vertex_id), edge}) {}
|
||||
|
||||
Delta(RemoveInEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge,
|
||||
Delta(RemoveInEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
|
||||
std::atomic<uint64_t> *timestamp, uint64_t command_id)
|
||||
: action(Action::REMOVE_IN_EDGE),
|
||||
timestamp(timestamp),
|
||||
command_id(command_id),
|
||||
vertex_edge({edge_type, vertex, edge}) {}
|
||||
vertex_edge({edge_type, std::move(vertex_id), edge}) {}
|
||||
|
||||
Delta(RemoveOutEdgeTag /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge,
|
||||
Delta(RemoveOutEdgeTag /*unused*/, EdgeTypeId edge_type, VertexId vertex_id, EdgeRef edge,
|
||||
std::atomic<uint64_t> *timestamp, uint64_t command_id)
|
||||
: action(Action::REMOVE_OUT_EDGE),
|
||||
timestamp(timestamp),
|
||||
command_id(command_id),
|
||||
vertex_edge({edge_type, vertex, edge}) {}
|
||||
vertex_edge({edge_type, std::move(vertex_id), edge}) {}
|
||||
|
||||
Delta(const Delta &) = delete;
|
||||
Delta(Delta &&) = delete;
|
||||
@ -212,11 +214,12 @@ struct Delta {
|
||||
case Action::RECREATE_OBJECT:
|
||||
case Action::ADD_LABEL:
|
||||
case Action::REMOVE_LABEL:
|
||||
break;
|
||||
case Action::ADD_IN_EDGE:
|
||||
case Action::ADD_OUT_EDGE:
|
||||
case Action::REMOVE_IN_EDGE:
|
||||
case Action::REMOVE_OUT_EDGE:
|
||||
break;
|
||||
std::destroy_at(&vertex_edge.vertex_id);
|
||||
case Action::SET_PROPERTY:
|
||||
property.value.~PropertyValue();
|
||||
break;
|
||||
@ -239,7 +242,7 @@ struct Delta {
|
||||
} property;
|
||||
struct {
|
||||
EdgeTypeId edge_type;
|
||||
Vertex *vertex;
|
||||
VertexId vertex_id;
|
||||
EdgeRef edge;
|
||||
} vertex_edge;
|
||||
};
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include "storage/v3/schema_validator.hpp"
|
||||
#include "storage/v3/schemas.hpp"
|
||||
#include "storage/v3/vertex_accessor.hpp"
|
||||
#include "storage/v3/vertex_id.hpp"
|
||||
#include "storage/v3/vertices_skip_list.hpp"
|
||||
#include "utils/file_locker.hpp"
|
||||
#include "utils/logging.hpp"
|
||||
@ -93,6 +94,10 @@ namespace memgraph::storage::v3::durability {
|
||||
// IMPORTANT: When changing snapshot encoding/decoding bump the snapshot/WAL
|
||||
// version in `version.hpp`.
|
||||
|
||||
namespace {
|
||||
constexpr auto kDummyLabelId = LabelId::FromUint(0);
|
||||
} // namespace
|
||||
|
||||
// Function used to read information about the snapshot file.
|
||||
SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
|
||||
// Check magic and version.
|
||||
@ -444,7 +449,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, VerticesSkipLi
|
||||
// TODO Fix Gid
|
||||
SPDLOG_TRACE("Recovered inbound edge {} with label \"{}\" from vertex {}.", *edge_gid,
|
||||
name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), 1);
|
||||
vertex.in_edges.emplace_back(get_edge_type_from_id(*edge_type), &from_vertex->vertex, edge_ref);
|
||||
vertex.in_edges.emplace_back(get_edge_type_from_id(*edge_type), VertexId{kDummyLabelId, {}}, edge_ref);
|
||||
}
|
||||
}
|
||||
|
||||
@ -482,7 +487,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, VerticesSkipLi
|
||||
// TODO Fix Gid
|
||||
SPDLOG_TRACE("Recovered outbound edge {} with label \"{}\" to vertex {}.", *edge_gid,
|
||||
name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), 1);
|
||||
vertex.out_edges.emplace_back(get_edge_type_from_id(*edge_type), &to_vertex->vertex, edge_ref);
|
||||
vertex.out_edges.emplace_back(get_edge_type_from_id(*edge_type), VertexId{kDummyLabelId, {}}, edge_ref);
|
||||
}
|
||||
// Increment edge count. We only increment the count here because the
|
||||
// information is duplicated in in_edges.
|
||||
@ -718,8 +723,14 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
|
||||
// but that isn't an issue because we won't use that part of the API
|
||||
// here.
|
||||
// TODO(jbajic) Fix snapshot with new schema rules
|
||||
auto ea = EdgeAccessor{edge_ref, EdgeTypeId::FromUint(0UL), nullptr, nullptr, transaction, indices, constraints,
|
||||
items, vertex_validator};
|
||||
auto ea = EdgeAccessor{edge_ref,
|
||||
EdgeTypeId::FromUint(0UL),
|
||||
VertexId{kDummyLabelId, {}},
|
||||
VertexId{kDummyLabelId, {}},
|
||||
transaction,
|
||||
indices,
|
||||
constraints,
|
||||
items};
|
||||
|
||||
// Get edge data.
|
||||
auto maybe_props = ea.Properties(View::OLD);
|
||||
|
@ -50,13 +50,9 @@ bool EdgeAccessor::IsVisible(const View view) const {
|
||||
return exists && (for_deleted_ || !deleted);
|
||||
}
|
||||
|
||||
VertexAccessor EdgeAccessor::FromVertex() const {
|
||||
return {from_vertex_, transaction_, indices_, constraints_, config_, *vertex_validator_};
|
||||
}
|
||||
const VertexId &EdgeAccessor::FromVertex() const { return from_vertex_; }
|
||||
|
||||
VertexAccessor EdgeAccessor::ToVertex() const {
|
||||
return {to_vertex_, transaction_, indices_, constraints_, config_, *vertex_validator_};
|
||||
}
|
||||
const VertexId &EdgeAccessor::ToVertex() const { return to_vertex_; }
|
||||
|
||||
Result<PropertyValue> EdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
|
||||
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
|
||||
|
@ -35,26 +35,24 @@ class EdgeAccessor final {
|
||||
friend class Shard;
|
||||
|
||||
public:
|
||||
EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
|
||||
Indices *indices, Constraints *constraints, Config::Items config,
|
||||
const VertexValidator &vertex_validator, bool for_deleted = false)
|
||||
EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, VertexId from_vertex, VertexId to_vertex, Transaction *transaction,
|
||||
Indices *indices, Constraints *constraints, Config::Items config, bool for_deleted = false)
|
||||
: edge_(edge),
|
||||
edge_type_(edge_type),
|
||||
from_vertex_(from_vertex),
|
||||
to_vertex_(to_vertex),
|
||||
from_vertex_(std::move(from_vertex)),
|
||||
to_vertex_(std::move(to_vertex)),
|
||||
transaction_(transaction),
|
||||
indices_(indices),
|
||||
constraints_(constraints),
|
||||
config_(config),
|
||||
vertex_validator_{&vertex_validator},
|
||||
for_deleted_(for_deleted) {}
|
||||
|
||||
/// @return true if the object is visible from the current transaction
|
||||
bool IsVisible(View view) const;
|
||||
|
||||
VertexAccessor FromVertex() const;
|
||||
const VertexId &FromVertex() const;
|
||||
|
||||
VertexAccessor ToVertex() const;
|
||||
const VertexId &ToVertex() const;
|
||||
|
||||
EdgeTypeId EdgeType() const { return edge_type_; }
|
||||
|
||||
@ -89,13 +87,12 @@ class EdgeAccessor final {
|
||||
private:
|
||||
EdgeRef edge_;
|
||||
EdgeTypeId edge_type_;
|
||||
Vertex *from_vertex_;
|
||||
Vertex *to_vertex_;
|
||||
VertexId from_vertex_;
|
||||
VertexId to_vertex_;
|
||||
Transaction *transaction_;
|
||||
Indices *indices_;
|
||||
Constraints *constraints_;
|
||||
Config::Items config_;
|
||||
const VertexValidator *vertex_validator_;
|
||||
|
||||
// if the accessor was created for a deleted edge.
|
||||
// Accessor behaves differently for some methods based on this
|
||||
|
@ -11,6 +11,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <bit>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <type_traits>
|
||||
@ -20,30 +21,36 @@
|
||||
namespace memgraph::storage::v3 {
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define STORAGE_DEFINE_ID_TYPE(name) \
|
||||
class name final { \
|
||||
private: \
|
||||
explicit name(uint64_t id) : id_(id) {} \
|
||||
\
|
||||
public: \
|
||||
/* Default constructor to allow serialization or preallocation. */ \
|
||||
name() = default; \
|
||||
\
|
||||
static name FromUint(uint64_t id) { return (name){id}; } \
|
||||
static name FromInt(int64_t id) { return (name){utils::MemcpyCast<uint64_t>(id)}; } \
|
||||
uint64_t AsUint() const { return id_; } \
|
||||
int64_t AsInt() const { return utils::MemcpyCast<int64_t>(id_); } \
|
||||
\
|
||||
private: \
|
||||
uint64_t id_; \
|
||||
}; \
|
||||
static_assert(std::is_trivially_copyable<name>::value, "storage::" #name " must be trivially copyable!"); \
|
||||
inline bool operator==(const name &first, const name &second) { return first.AsUint() == second.AsUint(); } \
|
||||
inline bool operator!=(const name &first, const name &second) { return first.AsUint() != second.AsUint(); } \
|
||||
inline bool operator<(const name &first, const name &second) { return first.AsUint() < second.AsUint(); } \
|
||||
inline bool operator>(const name &first, const name &second) { return first.AsUint() > second.AsUint(); } \
|
||||
inline bool operator<=(const name &first, const name &second) { return first.AsUint() <= second.AsUint(); } \
|
||||
inline bool operator>=(const name &first, const name &second) { return first.AsUint() >= second.AsUint(); }
|
||||
#define STORAGE_DEFINE_ID_TYPE(name) \
|
||||
class name final { \
|
||||
private: \
|
||||
constexpr explicit name(uint64_t id) : id_(id) {} \
|
||||
\
|
||||
public: \
|
||||
/* Default constructor to allow serialization or preallocation. */ \
|
||||
constexpr name() = default; \
|
||||
\
|
||||
constexpr static name FromUint(uint64_t id) { return (name){id}; } \
|
||||
constexpr static name FromInt(int64_t id) { return (name){std::bit_cast<uint64_t>(id)}; } \
|
||||
constexpr uint64_t AsUint() const { return id_; } \
|
||||
constexpr int64_t AsInt() const { return std::bit_cast<int64_t>(id_); } \
|
||||
\
|
||||
private: \
|
||||
uint64_t id_; \
|
||||
}; \
|
||||
static_assert(std::is_trivially_copyable<name>::value, "storage::" #name " must be trivially copyable!"); \
|
||||
constexpr inline bool operator==(const name &first, const name &second) { \
|
||||
return first.AsUint() == second.AsUint(); \
|
||||
} \
|
||||
constexpr inline bool operator!=(const name &first, const name &second) { \
|
||||
return first.AsUint() != second.AsUint(); \
|
||||
} \
|
||||
constexpr inline bool operator<(const name &first, const name &second) { return first.AsUint() < second.AsUint(); } \
|
||||
constexpr inline bool operator>(const name &first, const name &second) { return first.AsUint() > second.AsUint(); } \
|
||||
constexpr inline bool operator<=(const name &first, const name &second) { \
|
||||
return first.AsUint() <= second.AsUint(); \
|
||||
} \
|
||||
constexpr inline bool operator>=(const name &first, const name &second) { return first.AsUint() >= second.AsUint(); }
|
||||
|
||||
STORAGE_DEFINE_ID_TYPE(Gid);
|
||||
STORAGE_DEFINE_ID_TYPE(LabelId);
|
||||
|
@ -217,13 +217,13 @@ class LabelPropertyIndex {
|
||||
};
|
||||
|
||||
Iterable Vertices(LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower_bound,
|
||||
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction,
|
||||
const VertexValidator &vertex_validator) {
|
||||
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view,
|
||||
Transaction *transaction) {
|
||||
auto it = index_.find({label, property});
|
||||
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(),
|
||||
property.AsUint());
|
||||
return {it->second.access(), label, property, lower_bound, upper_bound, view,
|
||||
transaction, indices_, constraints_, config_, vertex_validator};
|
||||
return {it->second.access(), label, property, lower_bound, upper_bound, view,
|
||||
transaction, indices_, constraints_, config_, *vertex_validator_};
|
||||
}
|
||||
|
||||
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
|
||||
|
@ -114,6 +114,9 @@ inline void CreateAndLinkDelta(Transaction *transaction, TObj *object, Args &&..
|
||||
// concurrently (as well as other execution threads).
|
||||
|
||||
// 1. We need to set the next delta of the new delta to the existing delta.
|
||||
// TODO(antaljanosbenjamin): clang-tidy detects (in my opinion a false positive) issue in
|
||||
// `Shard::Accessor::CreateEdge`.
|
||||
// NOLINTNEXTLINE(clang-analyzer-core.NullDereference)
|
||||
delta->next.store(object->delta, std::memory_order_release);
|
||||
// 2. We need to set the previous delta of the new delta to the object.
|
||||
delta->prev.Set(object);
|
||||
|
@ -179,7 +179,6 @@ void Shard::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Bui
|
||||
shard_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
|
||||
shard_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id);
|
||||
const auto &recovery_info = recovered_snapshot.recovery_info;
|
||||
shard_->edge_id_ = recovery_info.next_edge_id;
|
||||
shard_->timestamp_ = std::max(shard_->timestamp_, recovery_info.next_timestamp);
|
||||
|
||||
// durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &shard_->indices_,
|
||||
|
@ -370,7 +370,6 @@ Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key,
|
||||
// &edges_, &edge_count_, &name_id_mapper_, &indices_, &constraints_, config_.items,
|
||||
// &wal_seq_num_);
|
||||
if (info) {
|
||||
edge_id_ = info->next_edge_id;
|
||||
timestamp_ = std::max(timestamp_, info->next_timestamp);
|
||||
if (info->last_commit_timestamp) {
|
||||
last_commit_timestamp_ = *info->last_commit_timestamp;
|
||||
@ -563,8 +562,8 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
|
||||
"accessor when deleting a vertex!");
|
||||
auto *vertex_ptr = vertex->vertex_;
|
||||
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
|
||||
std::vector<Vertex::EdgeLink> in_edges;
|
||||
std::vector<Vertex::EdgeLink> out_edges;
|
||||
|
||||
{
|
||||
if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR;
|
||||
@ -576,11 +575,12 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
|
||||
}
|
||||
|
||||
std::vector<EdgeAccessor> deleted_edges;
|
||||
const VertexId vertex_id{shard_->primary_label_, vertex_ptr->keys.Keys()};
|
||||
for (const auto &item : in_edges) {
|
||||
auto [edge_type, from_vertex, edge] = item;
|
||||
EdgeAccessor e(edge, edge_type, from_vertex, vertex_ptr, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_, shard_->vertex_validator_);
|
||||
auto ret = DeleteEdge(&e);
|
||||
EdgeAccessor e(edge, edge_type, from_vertex, vertex_id, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_);
|
||||
auto ret = DeleteEdge(e.FromVertex(), e.ToVertex(), e.Gid());
|
||||
if (ret.HasError()) {
|
||||
MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
|
||||
return ret.GetError();
|
||||
@ -592,9 +592,9 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
|
||||
}
|
||||
for (const auto &item : out_edges) {
|
||||
auto [edge_type, to_vertex, edge] = item;
|
||||
EdgeAccessor e(edge, edge_type, vertex_ptr, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_, shard_->vertex_validator_);
|
||||
auto ret = DeleteEdge(&e);
|
||||
EdgeAccessor e(edge, edge_type, vertex_id, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_);
|
||||
auto ret = DeleteEdge(e.FromVertex(), e.ToVertex(), e.Gid());
|
||||
if (ret.HasError()) {
|
||||
MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!");
|
||||
return ret.GetError();
|
||||
@ -621,80 +621,39 @@ Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shar
|
||||
std::move(deleted_edges));
|
||||
}
|
||||
|
||||
Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) {
|
||||
Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexId from_vertex_id, VertexId to_vertex_id,
|
||||
const EdgeTypeId edge_type, const Gid gid) {
|
||||
OOMExceptionEnabler oom_exception;
|
||||
MG_ASSERT(from->transaction_ == to->transaction_,
|
||||
"VertexAccessors must be from the same transaction when creating "
|
||||
"an edge!");
|
||||
MG_ASSERT(from->transaction_ == &transaction_,
|
||||
"VertexAccessors must be from the same transaction in when "
|
||||
"creating an edge!");
|
||||
Vertex *from_vertex{nullptr};
|
||||
Vertex *to_vertex{nullptr};
|
||||
|
||||
auto *from_vertex = from->vertex_;
|
||||
auto *to_vertex = to->vertex_;
|
||||
auto acc = shard_->vertices_.access();
|
||||
|
||||
if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (from_vertex->deleted) return Error::DELETED_OBJECT;
|
||||
const auto from_is_local = shard_->IsVertexBelongToShard(from_vertex_id);
|
||||
const auto to_is_local = shard_->IsVertexBelongToShard(to_vertex_id);
|
||||
MG_ASSERT(from_is_local || to_is_local, "Trying to create an edge without having a local vertex");
|
||||
|
||||
if (to_vertex != from_vertex) {
|
||||
if (from_is_local) {
|
||||
auto it = acc.find(from_vertex_id.primary_key);
|
||||
MG_ASSERT(it != acc.end(), "Cannot find local vertex");
|
||||
from_vertex = &it->vertex;
|
||||
}
|
||||
|
||||
if (to_is_local) {
|
||||
auto it = acc.find(to_vertex_id.primary_key);
|
||||
MG_ASSERT(it != acc.end(), "Cannot find local vertex");
|
||||
to_vertex = &it->vertex;
|
||||
}
|
||||
|
||||
if (from_is_local) {
|
||||
if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (from_vertex->deleted) return Error::DELETED_OBJECT;
|
||||
}
|
||||
if (to_is_local && to_vertex != from_vertex) {
|
||||
if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (to_vertex->deleted) return Error::DELETED_OBJECT;
|
||||
}
|
||||
|
||||
auto gid = Gid::FromUint(shard_->edge_id_++);
|
||||
EdgeRef edge(gid);
|
||||
if (config_.properties_on_edges) {
|
||||
auto acc = shard_->edges_.access();
|
||||
auto *delta = CreateDeleteObjectDelta(&transaction_);
|
||||
auto [it, inserted] = acc.insert(Edge(gid, delta));
|
||||
MG_ASSERT(inserted, "The edge must be inserted here!");
|
||||
MG_ASSERT(it != acc.end(), "Invalid Edge accessor!");
|
||||
edge = EdgeRef(&*it);
|
||||
delta->prev.Set(&*it);
|
||||
}
|
||||
|
||||
CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
|
||||
from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
|
||||
|
||||
CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
|
||||
to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
|
||||
|
||||
// Increment edge count.
|
||||
++shard_->edge_count_;
|
||||
|
||||
return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_, shard_->vertex_validator_);
|
||||
}
|
||||
|
||||
Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type,
|
||||
Gid gid) {
|
||||
OOMExceptionEnabler oom_exception;
|
||||
MG_ASSERT(from->transaction_ == to->transaction_,
|
||||
"VertexAccessors must be from the same transaction when creating "
|
||||
"an edge!");
|
||||
MG_ASSERT(from->transaction_ == &transaction_,
|
||||
"VertexAccessors must be from the same transaction in when "
|
||||
"creating an edge!");
|
||||
|
||||
auto *from_vertex = from->vertex_;
|
||||
auto *to_vertex = to->vertex_;
|
||||
|
||||
if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (from_vertex->deleted) return Error::DELETED_OBJECT;
|
||||
|
||||
if (to_vertex != from_vertex) {
|
||||
if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (to_vertex->deleted) return Error::DELETED_OBJECT;
|
||||
}
|
||||
|
||||
// NOTE: When we update the next `edge_id_` here we perform a RMW
|
||||
// (read-modify-write) operation that ISN'T atomic! But, that isn't an issue
|
||||
// because this function is only called from the replication delta applier
|
||||
// that runs single-threadedly and while this instance is set-up to apply
|
||||
// threads (it is the replica), it is guaranteed that no other writes are
|
||||
// possible.
|
||||
shard_->edge_id_ = std::max(shard_->edge_id_, gid.AsUint() + 1);
|
||||
|
||||
EdgeRef edge(gid);
|
||||
if (config_.properties_on_edges) {
|
||||
auto acc = shard_->edges_.access();
|
||||
@ -706,69 +665,99 @@ Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAcc
|
||||
delta->prev.Set(&*it);
|
||||
}
|
||||
|
||||
CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex, edge);
|
||||
from_vertex->out_edges.emplace_back(edge_type, to_vertex, edge);
|
||||
|
||||
CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex, edge);
|
||||
to_vertex->in_edges.emplace_back(edge_type, from_vertex, edge);
|
||||
|
||||
if (from_is_local) {
|
||||
CreateAndLinkDelta(&transaction_, from_vertex, Delta::RemoveOutEdgeTag(), edge_type, to_vertex_id, edge);
|
||||
from_vertex->out_edges.emplace_back(edge_type, to_vertex_id, edge);
|
||||
}
|
||||
if (to_is_local) {
|
||||
CreateAndLinkDelta(&transaction_, to_vertex, Delta::RemoveInEdgeTag(), edge_type, from_vertex_id, edge);
|
||||
to_vertex->in_edges.emplace_back(edge_type, from_vertex_id, edge);
|
||||
}
|
||||
// Increment edge count.
|
||||
++shard_->edge_count_;
|
||||
|
||||
return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_, &shard_->constraints_,
|
||||
config_, shard_->vertex_validator_);
|
||||
return EdgeAccessor(edge, edge_type, std::move(from_vertex_id), std::move(to_vertex_id), &transaction_,
|
||||
&shard_->indices_, &shard_->constraints_, config_);
|
||||
}
|
||||
|
||||
Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(EdgeAccessor *edge) {
|
||||
MG_ASSERT(edge->transaction_ == &transaction_,
|
||||
"EdgeAccessor must be from the same transaction as the storage "
|
||||
"accessor when deleting an edge!");
|
||||
auto edge_ref = edge->edge_;
|
||||
auto edge_type = edge->edge_type_;
|
||||
Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(VertexId from_vertex_id, VertexId to_vertex_id,
|
||||
const Gid edge_id) {
|
||||
Vertex *from_vertex{nullptr};
|
||||
Vertex *to_vertex{nullptr};
|
||||
|
||||
if (config_.properties_on_edges) {
|
||||
auto *edge_ptr = edge_ref.ptr;
|
||||
auto acc = shard_->vertices_.access();
|
||||
|
||||
if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR;
|
||||
const auto from_is_local = shard_->IsVertexBelongToShard(from_vertex_id);
|
||||
const auto to_is_local = shard_->IsVertexBelongToShard(to_vertex_id);
|
||||
|
||||
if (edge_ptr->deleted) return std::optional<EdgeAccessor>{};
|
||||
if (from_is_local) {
|
||||
auto it = acc.find(from_vertex_id.primary_key);
|
||||
MG_ASSERT(it != acc.end(), "Cannot find local vertex");
|
||||
from_vertex = &it->vertex;
|
||||
}
|
||||
|
||||
auto *from_vertex = edge->from_vertex_;
|
||||
auto *to_vertex = edge->to_vertex_;
|
||||
if (to_is_local) {
|
||||
auto it = acc.find(to_vertex_id.primary_key);
|
||||
MG_ASSERT(it != acc.end(), "Cannot find local vertex");
|
||||
to_vertex = &it->vertex;
|
||||
}
|
||||
|
||||
if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
|
||||
MG_ASSERT(from_is_local || to_is_local, "Trying to delete an edge without having a local vertex");
|
||||
|
||||
if (to_vertex != from_vertex) {
|
||||
if (!PrepareForWrite(&transaction_, to_vertex)) return Error::SERIALIZATION_ERROR;
|
||||
if (from_is_local) {
|
||||
if (!PrepareForWrite(&transaction_, from_vertex)) {
|
||||
return Error::SERIALIZATION_ERROR;
|
||||
}
|
||||
MG_ASSERT(!from_vertex->deleted, "Invalid database state!");
|
||||
}
|
||||
if (to_is_local && to_vertex != from_vertex) {
|
||||
if (!PrepareForWrite(&transaction_, to_vertex)) {
|
||||
return Error::SERIALIZATION_ERROR;
|
||||
}
|
||||
MG_ASSERT(!to_vertex->deleted, "Invalid database state!");
|
||||
}
|
||||
|
||||
auto delete_edge_from_storage = [&edge_type, &edge_ref, this](auto *vertex, auto *edges) {
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link(edge_type, vertex, edge_ref);
|
||||
auto it = std::find(edges->begin(), edges->end(), link);
|
||||
const auto edge_ref = std::invoke([edge_id, this]() -> EdgeRef {
|
||||
if (!config_.properties_on_edges) {
|
||||
return EdgeRef(edge_id);
|
||||
}
|
||||
auto edge_acc = shard_->edges_.access();
|
||||
auto res = edge_acc.find(edge_id);
|
||||
MG_ASSERT(res != edge_acc.end(), "Cannot find edge");
|
||||
return EdgeRef(&*res);
|
||||
});
|
||||
|
||||
std::optional<EdgeTypeId> edge_type{};
|
||||
auto delete_edge_from_storage = [&edge_type, &edge_ref, this](std::vector<Vertex::EdgeLink> &edges) mutable {
|
||||
auto it = std::find_if(edges.begin(), edges.end(),
|
||||
[&edge_ref](const Vertex::EdgeLink &link) { return std::get<2>(link) == edge_ref; });
|
||||
if (config_.properties_on_edges) {
|
||||
MG_ASSERT(it != edges->end(), "Invalid database state!");
|
||||
} else if (it == edges->end()) {
|
||||
MG_ASSERT(it != edges.end(), "Invalid database state!");
|
||||
} else if (it == edges.end()) {
|
||||
return false;
|
||||
}
|
||||
std::swap(*it, *edges->rbegin());
|
||||
edges->pop_back();
|
||||
edge_type = std::get<0>(*it);
|
||||
std::swap(*it, *edges.rbegin());
|
||||
edges.pop_back();
|
||||
return true;
|
||||
};
|
||||
|
||||
auto op1 = delete_edge_from_storage(to_vertex, &from_vertex->out_edges);
|
||||
auto op2 = delete_edge_from_storage(from_vertex, &to_vertex->in_edges);
|
||||
// NOLINTNEXTLINE(clang-analyzer-core.NonNullParamChecker)
|
||||
auto success_on_to = to_is_local ? delete_edge_from_storage(to_vertex->in_edges) : false;
|
||||
auto success_on_from = from_is_local ? delete_edge_from_storage(from_vertex->out_edges) : false;
|
||||
|
||||
if (config_.properties_on_edges) {
|
||||
MG_ASSERT((op1 && op2), "Invalid database state!");
|
||||
// Because of the check above, we are sure that the vertex exists.
|
||||
// One vertex is always local to the shard, so at least one of the operation should always succeed
|
||||
MG_ASSERT((to_is_local == success_on_to) && (from_is_local == success_on_from), "Invalid database state!");
|
||||
} else {
|
||||
MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!");
|
||||
if (!op1 && !op2) {
|
||||
// We might get here with self-edges, because without the edge object we cannot detect already deleted edges, thus
|
||||
// it is possible that both of the operation fails
|
||||
if (!success_on_to && !success_on_from) {
|
||||
// The edge is already deleted.
|
||||
return std::optional<EdgeAccessor>{};
|
||||
}
|
||||
|
||||
MG_ASSERT((!to_is_local || !from_is_local) || (success_on_to == success_on_from), "Invalid database state!");
|
||||
}
|
||||
|
||||
if (config_.properties_on_edges) {
|
||||
@ -777,14 +766,20 @@ Result<std::optional<EdgeAccessor>> Shard::Accessor::DeleteEdge(EdgeAccessor *ed
|
||||
edge_ptr->deleted = true;
|
||||
}
|
||||
|
||||
CreateAndLinkDelta(&transaction_, from_vertex, Delta::AddOutEdgeTag(), edge_type, to_vertex, edge_ref);
|
||||
CreateAndLinkDelta(&transaction_, to_vertex, Delta::AddInEdgeTag(), edge_type, from_vertex, edge_ref);
|
||||
MG_ASSERT(edge_type.has_value(), "Edge type is not determined");
|
||||
|
||||
if (from_is_local) {
|
||||
CreateAndLinkDelta(&transaction_, from_vertex, Delta::AddOutEdgeTag(), *edge_type, to_vertex_id, edge_ref);
|
||||
}
|
||||
if (to_is_local) {
|
||||
CreateAndLinkDelta(&transaction_, to_vertex, Delta::AddInEdgeTag(), *edge_type, from_vertex_id, edge_ref);
|
||||
}
|
||||
|
||||
// Decrement edge count.
|
||||
--shard_->edge_count_;
|
||||
|
||||
return std::make_optional<EdgeAccessor>(edge_ref, edge_type, from_vertex, to_vertex, &transaction_, &shard_->indices_,
|
||||
&shard_->constraints_, config_, shard_->vertex_validator_, true);
|
||||
return std::make_optional<EdgeAccessor>(edge_ref, *edge_type, std::move(from_vertex_id), std::move(to_vertex_id),
|
||||
&transaction_, &shard_->indices_, &shard_->constraints_, config_, true);
|
||||
}
|
||||
|
||||
const std::string &Shard::Accessor::LabelToName(LabelId label) const { return shard_->LabelToName(label); }
|
||||
@ -936,16 +931,16 @@ void Shard::Accessor::Abort() {
|
||||
break;
|
||||
}
|
||||
case Delta::Action::ADD_IN_EDGE: {
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
|
||||
current->vertex_edge.vertex, current->vertex_edge.edge};
|
||||
Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
|
||||
current->vertex_edge.edge};
|
||||
auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
|
||||
MG_ASSERT(it == vertex->in_edges.end(), "Invalid database state!");
|
||||
vertex->in_edges.push_back(link);
|
||||
break;
|
||||
}
|
||||
case Delta::Action::ADD_OUT_EDGE: {
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
|
||||
current->vertex_edge.vertex, current->vertex_edge.edge};
|
||||
Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
|
||||
current->vertex_edge.edge};
|
||||
auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
|
||||
MG_ASSERT(it == vertex->out_edges.end(), "Invalid database state!");
|
||||
vertex->out_edges.push_back(link);
|
||||
@ -957,8 +952,8 @@ void Shard::Accessor::Abort() {
|
||||
break;
|
||||
}
|
||||
case Delta::Action::REMOVE_IN_EDGE: {
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
|
||||
current->vertex_edge.vertex, current->vertex_edge.edge};
|
||||
Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
|
||||
current->vertex_edge.edge};
|
||||
auto it = std::find(vertex->in_edges.begin(), vertex->in_edges.end(), link);
|
||||
MG_ASSERT(it != vertex->in_edges.end(), "Invalid database state!");
|
||||
std::swap(*it, *vertex->in_edges.rbegin());
|
||||
@ -966,8 +961,8 @@ void Shard::Accessor::Abort() {
|
||||
break;
|
||||
}
|
||||
case Delta::Action::REMOVE_OUT_EDGE: {
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{current->vertex_edge.edge_type,
|
||||
current->vertex_edge.vertex, current->vertex_edge.edge};
|
||||
Vertex::EdgeLink link{current->vertex_edge.edge_type, current->vertex_edge.vertex_id,
|
||||
current->vertex_edge.edge};
|
||||
auto it = std::find(vertex->out_edges.begin(), vertex->out_edges.end(), link);
|
||||
MG_ASSERT(it != vertex->out_edges.end(), "Invalid database state!");
|
||||
std::swap(*it, *vertex->out_edges.rbegin());
|
||||
@ -1208,27 +1203,23 @@ VerticesIterable Shard::Accessor::Vertices(LabelId label, View view) {
|
||||
}
|
||||
|
||||
VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property, View view) {
|
||||
return VerticesIterable(shard_->indices_.label_property_index.Vertices(
|
||||
label, property, std::nullopt, std::nullopt, view, &transaction_, shard_->vertex_validator_));
|
||||
return VerticesIterable(
|
||||
shard_->indices_.label_property_index.Vertices(label, property, std::nullopt, std::nullopt, view, &transaction_));
|
||||
}
|
||||
|
||||
VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) {
|
||||
return VerticesIterable(shard_->indices_.label_property_index.Vertices(
|
||||
label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_,
|
||||
shard_->vertex_validator_));
|
||||
label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_));
|
||||
}
|
||||
|
||||
VerticesIterable Shard::Accessor::Vertices(LabelId label, PropertyId property,
|
||||
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
|
||||
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) {
|
||||
return VerticesIterable(shard_->indices_.label_property_index.Vertices(
|
||||
label, property, lower_bound, upper_bound, view, &transaction_, shard_->vertex_validator_));
|
||||
return VerticesIterable(
|
||||
shard_->indices_.label_property_index.Vertices(label, property, lower_bound, upper_bound, view, &transaction_));
|
||||
}
|
||||
|
||||
Transaction Shard::CreateTransaction(IsolationLevel isolation_level) {
|
||||
// We acquire the transaction engine lock here because we access (and
|
||||
// modify) the transaction engine variables (`transaction_id` and
|
||||
// `timestamp`) below.
|
||||
uint64_t transaction_id{0};
|
||||
uint64_t start_timestamp{0};
|
||||
|
||||
@ -1742,6 +1733,11 @@ uint64_t Shard::CommitTimestamp(const std::optional<uint64_t> desired_commit_tim
|
||||
return *desired_commit_timestamp;
|
||||
}
|
||||
|
||||
bool Shard::IsVertexBelongToShard(const VertexId &vertex_id) const {
|
||||
return vertex_id.primary_label == primary_label_ && vertex_id.primary_key >= min_primary_key_ &&
|
||||
(!max_primary_key_.has_value() || vertex_id.primary_key < *max_primary_key_);
|
||||
}
|
||||
|
||||
bool Shard::SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) {
|
||||
// We don't want to restart the server if we're already a REPLICA
|
||||
if (replication_role_ == ReplicationRole::REPLICA) {
|
||||
|
@ -44,6 +44,7 @@
|
||||
#include "storage/v3/transaction.hpp"
|
||||
#include "storage/v3/vertex.hpp"
|
||||
#include "storage/v3/vertex_accessor.hpp"
|
||||
#include "storage/v3/vertex_id.hpp"
|
||||
#include "storage/v3/vertices_skip_list.hpp"
|
||||
#include "utils/exceptions.hpp"
|
||||
#include "utils/file_locker.hpp"
|
||||
@ -301,11 +302,11 @@ class Shard final {
|
||||
VertexAccessor *vertex);
|
||||
|
||||
/// @throw std::bad_alloc
|
||||
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
|
||||
Result<EdgeAccessor> CreateEdge(VertexId from_vertex_id, VertexId to_vertex_id, EdgeTypeId edge_type, Gid gid);
|
||||
|
||||
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
|
||||
/// @throw std::bad_alloc
|
||||
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
|
||||
Result<std::optional<EdgeAccessor>> DeleteEdge(VertexId from_vertex_id, VertexId to_vertex_id, Gid edge_id);
|
||||
|
||||
const std::string &LabelToName(LabelId label) const;
|
||||
const std::string &PropertyToName(PropertyId property) const;
|
||||
@ -347,9 +348,6 @@ class Shard final {
|
||||
/// @throw std::bad_alloc
|
||||
VertexAccessor CreateVertex(Gid gid, LabelId primary_label);
|
||||
|
||||
/// @throw std::bad_alloc
|
||||
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, Gid gid);
|
||||
|
||||
Shard *shard_;
|
||||
Transaction transaction_;
|
||||
std::optional<uint64_t> commit_timestamp_;
|
||||
@ -505,14 +503,16 @@ class Shard final {
|
||||
|
||||
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
|
||||
|
||||
[[nodiscard]] bool IsVertexBelongToShard(const VertexId &vertex_id) const;
|
||||
|
||||
// Main object storage
|
||||
NameIdMapper name_id_mapper_;
|
||||
LabelId primary_label_;
|
||||
// The shard's range is [min, max)
|
||||
PrimaryKey min_primary_key_;
|
||||
std::optional<PrimaryKey> max_primary_key_;
|
||||
VerticesSkipList vertices_;
|
||||
utils::SkipList<Edge> edges_;
|
||||
uint64_t edge_id_{0};
|
||||
// Even though the edge count is already kept in the `edges_` SkipList, the
|
||||
// list is used only when properties are enabled for edges. Because of that we
|
||||
// keep a separate count of edges that is always updated.
|
||||
|
@ -22,23 +22,28 @@
|
||||
#include "storage/v3/key_store.hpp"
|
||||
#include "storage/v3/property_store.hpp"
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/vertex_id.hpp"
|
||||
#include "utils/algorithm.hpp"
|
||||
#include "utils/spin_lock.hpp"
|
||||
|
||||
namespace memgraph::storage::v3 {
|
||||
|
||||
struct Vertex {
|
||||
using EdgeLink = std::tuple<EdgeTypeId, VertexId, EdgeRef>;
|
||||
|
||||
Vertex(Delta *delta, const std::vector<PropertyValue> &primary_properties) : keys{primary_properties}, delta{delta} {
|
||||
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT,
|
||||
"Vertex must be created with an initial DELETE_OBJECT delta!");
|
||||
}
|
||||
|
||||
friend bool operator==(const Vertex &vertex, const PrimaryKey &primary_key) { return vertex.keys == primary_key; }
|
||||
|
||||
KeyStore keys;
|
||||
|
||||
std::vector<LabelId> labels;
|
||||
PropertyStore properties;
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
|
||||
std::vector<EdgeLink> in_edges;
|
||||
std::vector<EdgeLink> out_edges;
|
||||
|
||||
bool deleted{false};
|
||||
// uint8_t PAD;
|
||||
|
@ -198,75 +198,27 @@ Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
|
||||
}
|
||||
|
||||
Result<LabelId> VertexAccessor::PrimaryLabel(const View view) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
Delta *delta = nullptr;
|
||||
{
|
||||
deleted = vertex_->deleted;
|
||||
delta = vertex_->delta;
|
||||
if (const auto result = CheckVertexExistence(view); result.HasError()) {
|
||||
return result.GetError();
|
||||
}
|
||||
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
|
||||
switch (delta.action) {
|
||||
case Delta::Action::DELETE_OBJECT: {
|
||||
exists = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::RECREATE_OBJECT: {
|
||||
deleted = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::ADD_LABEL:
|
||||
case Delta::Action::REMOVE_LABEL:
|
||||
case Delta::Action::SET_PROPERTY:
|
||||
case Delta::Action::ADD_IN_EDGE:
|
||||
case Delta::Action::ADD_OUT_EDGE:
|
||||
case Delta::Action::REMOVE_IN_EDGE:
|
||||
case Delta::Action::REMOVE_OUT_EDGE:
|
||||
break;
|
||||
}
|
||||
});
|
||||
if (!exists) return Error::NONEXISTENT_OBJECT;
|
||||
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
|
||||
|
||||
return vertex_validator_->primary_label_;
|
||||
}
|
||||
|
||||
Result<PrimaryKey> VertexAccessor::PrimaryKey(const View view) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
Delta *delta = nullptr;
|
||||
{
|
||||
deleted = vertex_->deleted;
|
||||
delta = vertex_->delta;
|
||||
}
|
||||
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
|
||||
switch (delta.action) {
|
||||
case Delta::Action::DELETE_OBJECT: {
|
||||
exists = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::RECREATE_OBJECT: {
|
||||
deleted = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::ADD_LABEL:
|
||||
case Delta::Action::REMOVE_LABEL:
|
||||
case Delta::Action::SET_PROPERTY:
|
||||
case Delta::Action::ADD_IN_EDGE:
|
||||
case Delta::Action::ADD_OUT_EDGE:
|
||||
case Delta::Action::REMOVE_IN_EDGE:
|
||||
case Delta::Action::REMOVE_OUT_EDGE:
|
||||
break;
|
||||
}
|
||||
});
|
||||
if (!exists) {
|
||||
return Error::NONEXISTENT_OBJECT;
|
||||
}
|
||||
if (!for_deleted_ && deleted) {
|
||||
return Error::DELETED_OBJECT;
|
||||
if (const auto result = CheckVertexExistence(view); result.HasError()) {
|
||||
return result.GetError();
|
||||
}
|
||||
return vertex_->keys.Keys();
|
||||
}
|
||||
|
||||
Result<VertexId> VertexAccessor::Id(View view) const {
|
||||
if (const auto result = CheckVertexExistence(view); result.HasError()) {
|
||||
return result.GetError();
|
||||
}
|
||||
return VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
|
||||
};
|
||||
|
||||
Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
@ -337,6 +289,43 @@ Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const Pro
|
||||
return std::move(current_value);
|
||||
}
|
||||
|
||||
Result<void> VertexAccessor::CheckVertexExistence(View view) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
Delta *delta = nullptr;
|
||||
{
|
||||
deleted = vertex_->deleted;
|
||||
delta = vertex_->delta;
|
||||
}
|
||||
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted](const Delta &delta) {
|
||||
switch (delta.action) {
|
||||
case Delta::Action::DELETE_OBJECT: {
|
||||
exists = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::RECREATE_OBJECT: {
|
||||
deleted = false;
|
||||
break;
|
||||
}
|
||||
case Delta::Action::ADD_LABEL:
|
||||
case Delta::Action::REMOVE_LABEL:
|
||||
case Delta::Action::SET_PROPERTY:
|
||||
case Delta::Action::ADD_IN_EDGE:
|
||||
case Delta::Action::ADD_OUT_EDGE:
|
||||
case Delta::Action::REMOVE_IN_EDGE:
|
||||
case Delta::Action::REMOVE_OUT_EDGE:
|
||||
break;
|
||||
}
|
||||
});
|
||||
if (!exists) {
|
||||
return Error::NONEXISTENT_OBJECT;
|
||||
}
|
||||
if (!for_deleted_ && deleted) {
|
||||
return Error::DELETED_OBJECT;
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
ResultSchema<PropertyValue> VertexAccessor::SetPropertyAndValidate(PropertyId property, const PropertyValue &value) {
|
||||
if (auto maybe_violation_error = vertex_validator_->ValidatePropertyUpdate(property); maybe_violation_error) {
|
||||
return {*maybe_violation_error};
|
||||
@ -472,20 +461,21 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view
|
||||
}
|
||||
|
||||
Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::vector<EdgeTypeId> &edge_types,
|
||||
const VertexAccessor *destination) const {
|
||||
MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
|
||||
const VertexId *destination_id) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
|
||||
std::vector<Vertex::EdgeLink> in_edges;
|
||||
Delta *delta = nullptr;
|
||||
{
|
||||
deleted = vertex_->deleted;
|
||||
if (edge_types.empty() && !destination) {
|
||||
if (edge_types.empty() && nullptr == destination_id) {
|
||||
in_edges = vertex_->in_edges;
|
||||
} else {
|
||||
for (const auto &item : vertex_->in_edges) {
|
||||
const auto &[edge_type, from_vertex, edge] = item;
|
||||
if (destination && from_vertex != destination->vertex_) continue;
|
||||
if (nullptr != destination_id && from_vertex != *destination_id) {
|
||||
continue;
|
||||
};
|
||||
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
|
||||
continue;
|
||||
in_edges.push_back(item);
|
||||
@ -494,29 +484,27 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::
|
||||
delta = vertex_->delta;
|
||||
}
|
||||
ApplyDeltasForRead(
|
||||
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destination](const Delta &delta) {
|
||||
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, destination_id](const Delta &delta) {
|
||||
switch (delta.action) {
|
||||
case Delta::Action::ADD_IN_EDGE: {
|
||||
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
|
||||
if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
|
||||
if (!edge_types.empty() &&
|
||||
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
|
||||
break;
|
||||
// Add the edge because we don't see the removal.
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
|
||||
delta.vertex_edge.edge};
|
||||
Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
|
||||
auto it = std::find(in_edges.begin(), in_edges.end(), link);
|
||||
MG_ASSERT(it == in_edges.end(), "Invalid database state!");
|
||||
in_edges.push_back(link);
|
||||
break;
|
||||
}
|
||||
case Delta::Action::REMOVE_IN_EDGE: {
|
||||
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
|
||||
if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
|
||||
if (!edge_types.empty() &&
|
||||
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
|
||||
break;
|
||||
// Remove the label because we don't see the addition.
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
|
||||
delta.vertex_edge.edge};
|
||||
Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
|
||||
auto it = std::find(in_edges.begin(), in_edges.end(), link);
|
||||
MG_ASSERT(it != in_edges.end(), "Invalid database state!");
|
||||
std::swap(*it, *in_edges.rbegin());
|
||||
@ -542,30 +530,32 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::
|
||||
if (!exists) return Error::NONEXISTENT_OBJECT;
|
||||
if (deleted) return Error::DELETED_OBJECT;
|
||||
std::vector<EdgeAccessor> ret;
|
||||
if (in_edges.empty()) {
|
||||
return ret;
|
||||
}
|
||||
ret.reserve(in_edges.size());
|
||||
const auto id = VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
|
||||
for (const auto &item : in_edges) {
|
||||
const auto &[edge_type, from_vertex, edge] = item;
|
||||
ret.emplace_back(edge, edge_type, from_vertex, vertex_, transaction_, indices_, constraints_, config_,
|
||||
*vertex_validator_);
|
||||
ret.emplace_back(edge, edge_type, from_vertex, id, transaction_, indices_, constraints_, config_);
|
||||
}
|
||||
return std::move(ret);
|
||||
return ret;
|
||||
}
|
||||
|
||||
Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std::vector<EdgeTypeId> &edge_types,
|
||||
const VertexAccessor *destination) const {
|
||||
MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
|
||||
const VertexId *destination_id) const {
|
||||
bool exists = true;
|
||||
bool deleted = false;
|
||||
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
|
||||
std::vector<Vertex::EdgeLink> out_edges;
|
||||
Delta *delta = nullptr;
|
||||
{
|
||||
deleted = vertex_->deleted;
|
||||
if (edge_types.empty() && !destination) {
|
||||
if (edge_types.empty() && nullptr == destination_id) {
|
||||
out_edges = vertex_->out_edges;
|
||||
} else {
|
||||
for (const auto &item : vertex_->out_edges) {
|
||||
const auto &[edge_type, to_vertex, edge] = item;
|
||||
if (destination && to_vertex != destination->vertex_) continue;
|
||||
if (nullptr != destination_id && to_vertex != *destination_id) continue;
|
||||
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
|
||||
continue;
|
||||
out_edges.push_back(item);
|
||||
@ -574,29 +564,27 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std:
|
||||
delta = vertex_->delta;
|
||||
}
|
||||
ApplyDeltasForRead(
|
||||
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destination](const Delta &delta) {
|
||||
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, destination_id](const Delta &delta) {
|
||||
switch (delta.action) {
|
||||
case Delta::Action::ADD_OUT_EDGE: {
|
||||
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
|
||||
if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
|
||||
if (!edge_types.empty() &&
|
||||
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
|
||||
break;
|
||||
// Add the edge because we don't see the removal.
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
|
||||
delta.vertex_edge.edge};
|
||||
Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
|
||||
auto it = std::find(out_edges.begin(), out_edges.end(), link);
|
||||
MG_ASSERT(it == out_edges.end(), "Invalid database state!");
|
||||
out_edges.push_back(link);
|
||||
break;
|
||||
}
|
||||
case Delta::Action::REMOVE_OUT_EDGE: {
|
||||
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
|
||||
if (nullptr != destination_id && delta.vertex_edge.vertex_id != *destination_id) break;
|
||||
if (!edge_types.empty() &&
|
||||
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
|
||||
break;
|
||||
// Remove the label because we don't see the addition.
|
||||
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
|
||||
delta.vertex_edge.edge};
|
||||
Vertex::EdgeLink link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex_id, delta.vertex_edge.edge};
|
||||
auto it = std::find(out_edges.begin(), out_edges.end(), link);
|
||||
MG_ASSERT(it != out_edges.end(), "Invalid database state!");
|
||||
std::swap(*it, *out_edges.rbegin());
|
||||
@ -622,13 +610,16 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std:
|
||||
if (!exists) return Error::NONEXISTENT_OBJECT;
|
||||
if (deleted) return Error::DELETED_OBJECT;
|
||||
std::vector<EdgeAccessor> ret;
|
||||
if (out_edges.empty()) {
|
||||
return ret;
|
||||
}
|
||||
ret.reserve(out_edges.size());
|
||||
const auto id = VertexId{vertex_validator_->primary_label_, vertex_->keys.Keys()};
|
||||
for (const auto &item : out_edges) {
|
||||
const auto &[edge_type, to_vertex, edge] = item;
|
||||
ret.emplace_back(edge, edge_type, vertex_, to_vertex, transaction_, indices_, constraints_, config_,
|
||||
*vertex_validator_);
|
||||
ret.emplace_back(edge, edge_type, id, to_vertex, transaction_, indices_, constraints_, config_);
|
||||
}
|
||||
return std::move(ret);
|
||||
return ret;
|
||||
}
|
||||
|
||||
Result<size_t> VertexAccessor::InDegree(View view) const {
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include "storage/v3/schema_validator.hpp"
|
||||
#include "storage/v3/transaction.hpp"
|
||||
#include "storage/v3/vertex.hpp"
|
||||
#include "storage/v3/vertex_id.hpp"
|
||||
#include "storage/v3/view.hpp"
|
||||
|
||||
namespace memgraph::storage::v3 {
|
||||
@ -76,6 +77,8 @@ class VertexAccessor final {
|
||||
|
||||
Result<PrimaryKey> PrimaryKey(View view) const;
|
||||
|
||||
Result<VertexId> Id(View view) const;
|
||||
|
||||
/// Set a property value and return the old value or error.
|
||||
/// @throw std::bad_alloc
|
||||
ResultSchema<PropertyValue> SetPropertyAndValidate(PropertyId property, const PropertyValue &value);
|
||||
@ -94,13 +97,13 @@ class VertexAccessor final {
|
||||
/// @throw std::length_error if the resulting vector exceeds
|
||||
/// std::vector::max_size().
|
||||
Result<std::vector<EdgeAccessor>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
|
||||
const VertexAccessor *destination = nullptr) const;
|
||||
const VertexId *destination_id = nullptr) const;
|
||||
|
||||
/// @throw std::bad_alloc
|
||||
/// @throw std::length_error if the resulting vector exceeds
|
||||
/// std::vector::max_size().
|
||||
Result<std::vector<EdgeAccessor>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
|
||||
const VertexAccessor *destination = nullptr) const;
|
||||
const VertexId *destination_id = nullptr) const;
|
||||
|
||||
Result<size_t> InDegree(View view) const;
|
||||
|
||||
@ -128,6 +131,8 @@ class VertexAccessor final {
|
||||
/// @throw std::bad_alloc
|
||||
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
|
||||
|
||||
Result<void> CheckVertexExistence(View view) const;
|
||||
|
||||
Vertex *vertex_;
|
||||
Transaction *transaction_;
|
||||
Indices *indices_;
|
||||
|
32
src/storage/v3/vertex_id.hpp
Normal file
32
src/storage/v3/vertex_id.hpp
Normal file
@ -0,0 +1,32 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "storage/v3/id_types.hpp"
|
||||
#include "storage/v3/key_store.hpp"
|
||||
|
||||
namespace memgraph::storage::v3 {
|
||||
|
||||
// TODO(antaljanosbenjamin): It is possible to use a union of the current primary key and a vertex pointer: for local
|
||||
// vertices we can spare some space by eliminating copying the primary label and key, however it might introduce some
|
||||
// overhead for "remove vertices", because of the extra enum that is necessary for this optimization.
|
||||
struct VertexId {
|
||||
VertexId(const LabelId primary_label, PrimaryKey primary_key)
|
||||
: primary_label{primary_label}, primary_key{std::move(primary_key)} {}
|
||||
LabelId primary_label;
|
||||
PrimaryKey primary_key;
|
||||
};
|
||||
|
||||
inline bool operator==(const VertexId &lhs, const VertexId &rhs) {
|
||||
return lhs.primary_label == rhs.primary_label && lhs.primary_key == rhs.primary_key;
|
||||
}
|
||||
} // namespace memgraph::storage::v3
|
@ -340,6 +340,9 @@ target_link_libraries(${test_prefix}storage_v3_indices mg-storage-v3)
|
||||
add_unit_test(storage_v3_vertex_accessors.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v3_vertex_accessors mg-storage-v3)
|
||||
|
||||
add_unit_test(storage_v3_edge.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v3_edge mg-storage-v3)
|
||||
|
||||
# Test mg-query-v2
|
||||
add_unit_test(query_v2_interpreter.cpp ${CMAKE_SOURCE_DIR}/src/glue/v2/communication.cpp)
|
||||
target_link_libraries(${test_prefix}query_v2_interpreter mg-storage-v3 mg-query-v2 mg-communication)
|
||||
|
@ -2195,7 +2195,9 @@ TEST_F(StorageV3, VertexNonexistentLabelPropertyEdgeAPI) {
|
||||
// Modify vertex.
|
||||
ASSERT_TRUE(vertex.AddLabelAndValidate(label1).HasValue());
|
||||
ASSERT_TRUE(vertex.SetPropertyAndValidate(property1, PropertyValue("value")).HasValue());
|
||||
ASSERT_TRUE(acc.CreateEdge(&vertex, &vertex, NameToEdgeTypeId("edge")).HasValue());
|
||||
ASSERT_TRUE(acc.CreateEdge(vertex.Id(View::NEW).GetValue(), vertex.Id(View::NEW).GetValue(), NameToEdgeTypeId("edge"),
|
||||
Gid::FromUint(1))
|
||||
.HasValue());
|
||||
|
||||
// Check state after (OLD view).
|
||||
ASSERT_EQ(vertex.Labels(View::OLD).GetError(), Error::NONEXISTENT_OBJECT);
|
||||
|
5274
tests/unit/storage_v3_edge.cpp
Normal file
5274
tests/unit/storage_v3_edge.cpp
Normal file
File diff suppressed because it is too large
Load Diff
Loading…
Reference in New Issue
Block a user