Fix build of mg-query
This commit is contained in:
parent
e0da73780a
commit
ed76943dff
@ -1,4 +1,4 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
// Copyright 2023 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
@ -112,16 +112,16 @@ query::Graph *SubgraphDbAccessor::getGraph() { return graph_; }
|
||||
VertexAccessor SubgraphVertexAccessor::GetVertexAccessor() const { return impl_; }
|
||||
|
||||
auto SubgraphVertexAccessor::OutEdges(storage::View view) const -> decltype(impl_.OutEdges(view)) {
|
||||
auto maybe_edges = impl_.impl_.OutEdges(view, {});
|
||||
auto maybe_edges = impl_.impl_->OutEdges(view, {});
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
auto edges = std::move(*maybe_edges);
|
||||
const auto &graph_edges = graph_->edges();
|
||||
|
||||
std::vector<storage::EdgeAccessor> filteredOutEdges;
|
||||
std::vector<std::unique_ptr<storage::EdgeAccessor>> filteredOutEdges;
|
||||
for (auto &edge : edges) {
|
||||
auto edge_q = EdgeAccessor(edge);
|
||||
if (graph_edges.contains(edge_q)) {
|
||||
filteredOutEdges.push_back(edge);
|
||||
filteredOutEdges.push_back(std::move(edge));
|
||||
}
|
||||
}
|
||||
|
||||
@ -129,16 +129,16 @@ auto SubgraphVertexAccessor::OutEdges(storage::View view) const -> decltype(impl
|
||||
}
|
||||
|
||||
auto SubgraphVertexAccessor::InEdges(storage::View view) const -> decltype(impl_.InEdges(view)) {
|
||||
auto maybe_edges = impl_.impl_.InEdges(view, {});
|
||||
auto maybe_edges = impl_.impl_->InEdges(view, {});
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
auto edges = std::move(*maybe_edges);
|
||||
const auto &graph_edges = graph_->edges();
|
||||
|
||||
std::vector<storage::EdgeAccessor> filteredOutEdges;
|
||||
std::vector<std::unique_ptr<storage::EdgeAccessor>> filteredOutEdges;
|
||||
for (auto &edge : edges) {
|
||||
auto edge_q = EdgeAccessor(edge);
|
||||
if (graph_edges.contains(edge_q)) {
|
||||
filteredOutEdges.push_back(edge);
|
||||
filteredOutEdges.push_back(std::move(edge));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -57,6 +57,7 @@ class EdgeAccessor final {
|
||||
storage::EdgeAccessor *impl_;
|
||||
|
||||
public:
|
||||
explicit EdgeAccessor(std::unique_ptr<storage::EdgeAccessor> &impl) : impl_(impl.get()) {}
|
||||
explicit EdgeAccessor(storage::EdgeAccessor *impl) : impl_(impl) {}
|
||||
|
||||
bool IsVisible(storage::View view) const { return impl_->IsVisible(view); }
|
||||
@ -104,9 +105,7 @@ class VertexAccessor final {
|
||||
public:
|
||||
storage::VertexAccessor *impl_;
|
||||
|
||||
static EdgeAccessor MakeEdgeAccessor(const std::unique_ptr<storage::EdgeAccessor> &impl) {
|
||||
return EdgeAccessor(impl.get());
|
||||
}
|
||||
static EdgeAccessor MakeEdgeAccessor(std::unique_ptr<storage::EdgeAccessor> &impl) { return EdgeAccessor(impl); }
|
||||
|
||||
public:
|
||||
explicit VertexAccessor(storage::VertexAccessor *impl) : impl_(impl) {}
|
||||
@ -146,7 +145,7 @@ class VertexAccessor final {
|
||||
}
|
||||
|
||||
auto InEdges(storage::View view, const std::vector<storage::EdgeTypeId> &edge_types) const
|
||||
-> storage::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_->InEdges(view)))> {
|
||||
-> storage::Result<decltype(iter::imap(MakeEdgeAccessor, *(impl_->InEdges(view))))> {
|
||||
auto maybe_edges = impl_->InEdges(view, edge_types);
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
|
||||
@ -172,7 +171,7 @@ class VertexAccessor final {
|
||||
|
||||
auto OutEdges(storage::View view, const std::vector<storage::EdgeTypeId> &edge_types,
|
||||
const VertexAccessor &dest) const
|
||||
-> storage::Result<decltype(iter::imap(MakeEdgeAccessor, *impl_->OutEdges(view)))> {
|
||||
-> storage::Result<decltype(iter::imap(MakeEdgeAccessor, *(impl_->OutEdges(view))))> {
|
||||
auto maybe_edges = impl_->OutEdges(view, edge_types, dest.impl_);
|
||||
if (maybe_edges.HasError()) return maybe_edges.GetError();
|
||||
return iter::imap(MakeEdgeAccessor, std::move(*maybe_edges));
|
||||
|
@ -1147,8 +1147,7 @@ PreparedQuery Interpreter::PrepareTransactionQuery(std::string_view query_upper)
|
||||
in_explicit_transaction_ = true;
|
||||
expect_rollback_ = false;
|
||||
|
||||
db_accessor_ =
|
||||
std::make_unique<storage::Storage::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
|
||||
db_accessor_ = interpreter_context_->db->Access(GetIsolationLevelOverride());
|
||||
execution_db_accessor_.emplace(db_accessor_.get());
|
||||
transaction_status_.store(TransactionStatus::ACTIVE, std::memory_order_release);
|
||||
|
||||
@ -2639,8 +2638,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
|
||||
utils::Downcast<ProfileQuery>(parsed_query.query) || utils::Downcast<DumpQuery>(parsed_query.query) ||
|
||||
utils::Downcast<TriggerQuery>(parsed_query.query) || utils::Downcast<AnalyzeGraphQuery>(parsed_query.query) ||
|
||||
utils::Downcast<TransactionQueueQuery>(parsed_query.query))) {
|
||||
db_accessor_ =
|
||||
std::make_unique<storage::Storage::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
|
||||
db_accessor_ = interpreter_context_->db->Access(GetIsolationLevelOverride());
|
||||
execution_db_accessor_.emplace(db_accessor_.get());
|
||||
transaction_status_.store(TransactionStatus::ACTIVE, std::memory_order_release);
|
||||
|
||||
@ -2784,7 +2782,7 @@ void RunTriggersIndividually(const utils::SkipList<Trigger> &triggers, Interpret
|
||||
|
||||
// create a new transaction for each trigger
|
||||
auto storage_acc = interpreter_context->db->Access();
|
||||
DbAccessor db_accessor{&storage_acc};
|
||||
DbAccessor db_accessor{storage_acc.get()};
|
||||
|
||||
trigger_context.AdaptForAccessor(&db_accessor);
|
||||
try {
|
||||
|
@ -53,6 +53,7 @@
|
||||
#include "utils/likely.hpp"
|
||||
#include "utils/logging.hpp"
|
||||
#include "utils/memory.hpp"
|
||||
#include "utils/message.hpp"
|
||||
#include "utils/pmr/list.hpp"
|
||||
#include "utils/pmr/unordered_map.hpp"
|
||||
#include "utils/pmr/unordered_set.hpp"
|
||||
|
@ -496,7 +496,7 @@ Streams::StreamsMap::iterator Streams::CreateConsumer(StreamsMap &map, const std
|
||||
[interpreter_context, interpreter]() { interpreter_context->interpreters->erase(interpreter.get()); }};
|
||||
|
||||
EventCounter::IncrementCounter(EventCounter::MessagesConsumed, messages.size());
|
||||
CallCustomTransformation(transformation_name, messages, result, accessor, *memory_resource, stream_name);
|
||||
CallCustomTransformation(transformation_name, messages, result, *accessor, *memory_resource, stream_name);
|
||||
|
||||
DiscardValueResultStream stream;
|
||||
|
||||
@ -743,7 +743,7 @@ TransformationResult Streams::Check(const std::string &stream_name, std::optiona
|
||||
&transformation_name = transformation_name, &result,
|
||||
&test_result]<typename T>(const std::vector<T> &messages) mutable {
|
||||
auto accessor = interpreter_context->db->Access();
|
||||
CallCustomTransformation(transformation_name, messages, result, accessor, *memory_resource, stream_name);
|
||||
CallCustomTransformation(transformation_name, messages, result, *accessor, *memory_resource, stream_name);
|
||||
|
||||
auto result_row = std::vector<TypedValue>();
|
||||
result_row.reserve(kCheckStreamResultSize);
|
||||
|
@ -1,4 +1,4 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
// Copyright 2023 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
@ -269,7 +269,7 @@ class TypedValue {
|
||||
|
||||
explicit TypedValue(const EdgeAccessor &edge, utils::MemoryResource *memory = utils::NewDeleteResource())
|
||||
: memory_(memory), type_(Type::Edge) {
|
||||
new (&edge_v) EdgeAccessor(edge);
|
||||
new (&edge_v) EdgeAccessor(std::move(edge));
|
||||
}
|
||||
|
||||
explicit TypedValue(const Path &path, utils::MemoryResource *memory = utils::NewDeleteResource())
|
||||
|
@ -55,23 +55,12 @@
|
||||
|
||||
namespace memgraph::storage {
|
||||
|
||||
/// Structure used to return information about the storage.
|
||||
struct StorageInfo {
|
||||
uint64_t vertex_count;
|
||||
uint64_t edge_count;
|
||||
double average_degree;
|
||||
uint64_t memory_usage;
|
||||
uint64_t disk_usage;
|
||||
};
|
||||
|
||||
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
|
||||
|
||||
// The storage is based on this paper:
|
||||
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
|
||||
// The paper implements a fully serializable storage, in our implementation we
|
||||
// only implement snapshot isolation for transactions.
|
||||
|
||||
class InMemoryStorage final {
|
||||
class InMemoryStorage final : public Storage {
|
||||
public:
|
||||
/// @throw std::system_error
|
||||
/// @throw std::bad_alloc
|
||||
@ -254,22 +243,23 @@ class InMemoryStorage final {
|
||||
Config::Items config_;
|
||||
};
|
||||
|
||||
InMemoryAccessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
|
||||
return InMemoryAccessor{this, override_isolation_level.value_or(isolation_level_)};
|
||||
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;
|
||||
const std::string &PropertyToName(PropertyId property) const;
|
||||
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
|
||||
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);
|
||||
LabelId NameToLabel(std::string_view name) override;
|
||||
|
||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
||||
PropertyId NameToProperty(std::string_view name);
|
||||
PropertyId NameToProperty(std::string_view name) override;
|
||||
|
||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
||||
EdgeTypeId NameToEdgeType(std::string_view name);
|
||||
EdgeTypeId NameToEdgeType(std::string_view name) override;
|
||||
|
||||
/// Create an index.
|
||||
/// Returns void if the index has been created.
|
||||
@ -278,7 +268,7 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
|
||||
|
||||
/// Create an index.
|
||||
/// Returns void if the index has been created.
|
||||
@ -287,7 +277,7 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
|
||||
|
||||
/// Drop an existing index.
|
||||
/// Returns void if the index has been dropped.
|
||||
@ -295,7 +285,7 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
|
||||
|
||||
/// Drop an existing index.
|
||||
/// Returns void if the index has been dropped.
|
||||
@ -303,9 +293,9 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
|
||||
|
||||
IndicesInfo ListAllIndices() const;
|
||||
IndicesInfo ListAllIndices() const override;
|
||||
|
||||
/// Returns void if the existence constraint has been created.
|
||||
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
|
||||
@ -315,7 +305,7 @@ class InMemoryStorage final {
|
||||
/// @throw std::bad_alloc
|
||||
/// @throw std::length_error
|
||||
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) override;
|
||||
|
||||
/// Drop an existing existence constraint.
|
||||
/// Returns void if the existence constraint has been dropped.
|
||||
@ -323,7 +313,7 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
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:
|
||||
@ -336,7 +326,7 @@ class InMemoryStorage final {
|
||||
/// * `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 = {});
|
||||
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:
|
||||
@ -347,60 +337,38 @@ class InMemoryStorage final {
|
||||
/// * `EMPTY_PROPERTIES` if the property set is empty, or
|
||||
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
|
||||
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, std::optional<uint64_t> desired_commit_timestamp) override;
|
||||
|
||||
ConstraintsInfo ListAllConstraints() const;
|
||||
ConstraintsInfo ListAllConstraints() const override;
|
||||
|
||||
StorageInfo GetInfo() const;
|
||||
StorageInfo GetInfo() const override;
|
||||
|
||||
bool LockPath();
|
||||
bool UnlockPath();
|
||||
bool LockPath() override;
|
||||
bool UnlockPath() override;
|
||||
|
||||
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
|
||||
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) override;
|
||||
|
||||
bool SetMainReplicationRole();
|
||||
|
||||
enum class RegisterReplicaError : uint8_t {
|
||||
NAME_EXISTS,
|
||||
END_POINT_EXISTS,
|
||||
CONNECTION_FAILED,
|
||||
COULD_NOT_BE_PERSISTED
|
||||
};
|
||||
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 = {});
|
||||
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config) override;
|
||||
/// @pre The instance should have a MAIN role
|
||||
bool UnregisterReplica(const std::string &name);
|
||||
bool UnregisterReplica(const std::string &name) override;
|
||||
|
||||
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
|
||||
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name) override;
|
||||
|
||||
ReplicationRole GetReplicationRole() const;
|
||||
ReplicationRole GetReplicationRole() const override;
|
||||
|
||||
struct TimestampInfo {
|
||||
uint64_t current_timestamp_of_replica;
|
||||
uint64_t current_number_of_timestamp_behind_master;
|
||||
};
|
||||
std::vector<ReplicaInfo> ReplicasInfo() override;
|
||||
|
||||
struct ReplicaInfo {
|
||||
std::string name;
|
||||
replication::ReplicationMode mode;
|
||||
io::network::Endpoint endpoint;
|
||||
replication::ReplicaState state;
|
||||
TimestampInfo timestamp_info;
|
||||
};
|
||||
void FreeMemory() override;
|
||||
|
||||
std::vector<ReplicaInfo> ReplicasInfo();
|
||||
void SetIsolationLevel(IsolationLevel isolation_level) override;
|
||||
|
||||
void FreeMemory();
|
||||
|
||||
void SetIsolationLevel(IsolationLevel isolation_level);
|
||||
|
||||
enum class CreateSnapshotError : uint8_t { DisabledForReplica };
|
||||
|
||||
utils::BasicResult<CreateSnapshotError> CreateSnapshot();
|
||||
utils::BasicResult<CreateSnapshotError> CreateSnapshot() override;
|
||||
|
||||
private:
|
||||
Transaction CreateTransaction(IsolationLevel isolation_level);
|
||||
|
@ -307,14 +307,16 @@ uint64_t InMemoryStorage::ReplicationServer::ReadAndApplyDelta(durability::BaseD
|
||||
auto edge_acc = storage_->edges_.access();
|
||||
auto vertex_acc = storage_->vertices_.access();
|
||||
|
||||
std::optional<std::pair<uint64_t, storage::InMemoryStorage::InMemoryAccessor>> commit_timestamp_and_accessor;
|
||||
std::optional<std::pair<uint64_t, storage::InMemoryStorage::InMemoryAccessor *>> commit_timestamp_and_accessor;
|
||||
auto get_transaction = [this, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
|
||||
if (!commit_timestamp_and_accessor) {
|
||||
commit_timestamp_and_accessor.emplace(commit_timestamp, storage_->Access());
|
||||
commit_timestamp_and_accessor.emplace(commit_timestamp,
|
||||
dynamic_cast<storage::InMemoryStorage::InMemoryAccessor *>(
|
||||
storage_->Access(std::optional<IsolationLevel>{}).get()));
|
||||
} else if (commit_timestamp_and_accessor->first != commit_timestamp) {
|
||||
throw utils::BasicException("Received more than one transaction!");
|
||||
}
|
||||
return &commit_timestamp_and_accessor->second;
|
||||
return commit_timestamp_and_accessor->second;
|
||||
};
|
||||
|
||||
uint64_t applied_deltas = 0;
|
||||
@ -486,7 +488,7 @@ uint64_t InMemoryStorage::ReplicationServer::ReadAndApplyDelta(durability::BaseD
|
||||
spdlog::trace(" Transaction end");
|
||||
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
|
||||
throw utils::BasicException("Invalid data!");
|
||||
auto ret = commit_timestamp_and_accessor->second.Commit(commit_timestamp_and_accessor->first);
|
||||
auto ret = commit_timestamp_and_accessor->second->Commit(commit_timestamp_and_accessor->first);
|
||||
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
|
||||
commit_timestamp_and_accessor = std::nullopt;
|
||||
break;
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <set>
|
||||
#include <span>
|
||||
|
||||
#include "io/network/endpoint.hpp"
|
||||
#include "storage/v2/config.hpp"
|
||||
#include "storage/v2/indices.hpp"
|
||||
#include "storage/v2/result.hpp"
|
||||
@ -21,11 +22,16 @@
|
||||
#include "storage/v2/vertex_accessor.hpp"
|
||||
#include "storage/v2/view.hpp"
|
||||
|
||||
#include "storage/v2/replication/config.hpp"
|
||||
#include "storage/v2/replication/enums.hpp"
|
||||
|
||||
namespace memgraph::storage {
|
||||
|
||||
struct Transaction;
|
||||
class EdgeAccessor;
|
||||
|
||||
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
|
||||
|
||||
// The storage is based on this paper:
|
||||
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
|
||||
// The paper implements a fully serializable storage, in our implementation we
|
||||
@ -149,6 +155,15 @@ struct ConstraintsInfo {
|
||||
std::vector<std::pair<LabelId, std::set<PropertyId>>> unique;
|
||||
};
|
||||
|
||||
/// Structure used to return information about the storage.
|
||||
struct StorageInfo {
|
||||
uint64_t vertex_count;
|
||||
uint64_t edge_count;
|
||||
double average_degree;
|
||||
uint64_t memory_usage;
|
||||
uint64_t disk_usage;
|
||||
};
|
||||
|
||||
class Storage {
|
||||
public:
|
||||
class Accessor {
|
||||
@ -273,6 +288,202 @@ class Storage {
|
||||
|
||||
virtual std::optional<uint64_t> GetTransactionId() const = 0;
|
||||
};
|
||||
|
||||
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) = 0;
|
||||
std::unique_ptr<Accessor> Access() { return Access(std::optional<IsolationLevel>{}); }
|
||||
|
||||
virtual const std::string &LabelToName(LabelId label) const = 0;
|
||||
virtual const std::string &PropertyToName(PropertyId property) const = 0;
|
||||
virtual const std::string &EdgeTypeToName(EdgeTypeId edge_type) const = 0;
|
||||
|
||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
||||
virtual LabelId NameToLabel(std::string_view name) = 0;
|
||||
|
||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
||||
virtual PropertyId NameToProperty(std::string_view name) = 0;
|
||||
|
||||
/// @throw std::bad_alloc if unable to insert a new mapping
|
||||
virtual EdgeTypeId NameToEdgeType(std::string_view name) = 0;
|
||||
|
||||
/// Create an index.
|
||||
/// Returns void if the index has been created.
|
||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||
/// * `IndexDefinitionError`: the index already exists.
|
||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||
/// @throw std::bad_alloc
|
||||
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
|
||||
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) {
|
||||
return CreateIndex(label, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// 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
|
||||
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) {
|
||||
return CreateIndex(label, property, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// Drop an existing index.
|
||||
/// Returns void if the index has been dropped.
|
||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||
/// * `IndexDefinitionError`: the index does not exist.
|
||||
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
||||
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) {
|
||||
return DropIndex(label, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// Drop an existing index.
|
||||
/// Returns void if the index has been dropped.
|
||||
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
|
||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||
/// * `IndexDefinitionError`: the index does not exist.
|
||||
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) {
|
||||
return DropIndex(label, property, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
virtual IndicesInfo ListAllIndices() const;
|
||||
|
||||
/// Returns void if the existence constraint has been created.
|
||||
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
|
||||
/// * `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
|
||||
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(LabelId label,
|
||||
PropertyId property) {
|
||||
return CreateExistenceConstraint(label, property, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// Drop an existing existence constraint.
|
||||
/// Returns void if the existence constraint has been dropped.
|
||||
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
|
||||
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
|
||||
/// * `ConstraintDefinitionError`: the constraint did not exists.
|
||||
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
|
||||
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(LabelId label,
|
||||
PropertyId property) {
|
||||
return DropExistenceConstraint(label, property, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// Create an unique constraint.
|
||||
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
|
||||
/// * `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
|
||||
virtual utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
|
||||
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
|
||||
std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
|
||||
LabelId label, const std::set<PropertyId> &properties) {
|
||||
return CreateUniqueConstraint(label, properties, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
/// 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.
|
||||
virtual utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
|
||||
DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
|
||||
std::optional<uint64_t> desired_commit_timestamp) = 0;
|
||||
|
||||
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
|
||||
LabelId label, const std::set<PropertyId> &properties) {
|
||||
return DropUniqueConstraint(label, properties, std::optional<uint64_t>{});
|
||||
}
|
||||
|
||||
virtual ConstraintsInfo ListAllConstraints() const = 0;
|
||||
|
||||
virtual StorageInfo GetInfo() const = 0;
|
||||
|
||||
virtual bool LockPath() = 0;
|
||||
virtual bool UnlockPath() = 0;
|
||||
|
||||
virtual bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) = 0;
|
||||
|
||||
bool SetReplicaRole(io::network::Endpoint endpoint) {
|
||||
return SetReplicaRole(endpoint, replication::ReplicationServerConfig{});
|
||||
}
|
||||
|
||||
virtual bool SetMainReplicationRole() = 0;
|
||||
|
||||
enum class RegisterReplicaError : uint8_t {
|
||||
NAME_EXISTS,
|
||||
END_POINT_EXISTS,
|
||||
CONNECTION_FAILED,
|
||||
COULD_NOT_BE_PERSISTED
|
||||
};
|
||||
|
||||
/// @pre The instance should have a MAIN role
|
||||
/// @pre Timeout can only be set for SYNC replication
|
||||
virtual utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
|
||||
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
|
||||
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config) = 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
|
||||
virtual bool UnregisterReplica(const std::string &name) = 0;
|
||||
|
||||
virtual std::optional<replication::ReplicaState> GetReplicaState(std::string_view name) = 0;
|
||||
|
||||
virtual ReplicationRole GetReplicationRole() const = 0;
|
||||
|
||||
struct TimestampInfo {
|
||||
uint64_t current_timestamp_of_replica;
|
||||
uint64_t current_number_of_timestamp_behind_master;
|
||||
};
|
||||
|
||||
struct ReplicaInfo {
|
||||
std::string name;
|
||||
replication::ReplicationMode mode;
|
||||
io::network::Endpoint endpoint;
|
||||
replication::ReplicaState state;
|
||||
TimestampInfo timestamp_info;
|
||||
};
|
||||
|
||||
virtual std::vector<ReplicaInfo> ReplicasInfo() = 0;
|
||||
|
||||
virtual void FreeMemory() = 0;
|
||||
|
||||
virtual void SetIsolationLevel(IsolationLevel isolation_level) = 0;
|
||||
|
||||
enum class CreateSnapshotError : uint8_t { DisabledForReplica };
|
||||
|
||||
virtual utils::BasicResult<CreateSnapshotError> CreateSnapshot() = 0;
|
||||
};
|
||||
|
||||
} // namespace memgraph::storage
|
||||
|
Loading…
Reference in New Issue
Block a user