From 7e84744d0754e9b4658c383f897a091dc3807cc4 Mon Sep 17 00:00:00 2001 From: Jure Bajic <jure.bajic@memgraph.com> Date: Thu, 1 Sep 2022 09:10:40 +0200 Subject: [PATCH] Split storage and shards (#519) - Rename storage to shard - Add primary label and range for shard - Remove id_mapper functionality from shard - Adapt tests --- src/glue/v2/communication.cpp | 14 +- src/glue/v2/communication.hpp | 16 +- src/query/v2/db_accessor.hpp | 19 +- src/query/v2/interpreter.cpp | 37 +- src/query/v2/interpreter.hpp | 23 +- src/query/v2/stream/streams.cpp | 2 +- src/storage/v3/CMakeLists.txt | 1 + src/storage/v3/edge_accessor.hpp | 2 +- .../v3/replication/replication_client.cpp | 122 +- .../v3/replication/replication_client.hpp | 8 +- .../v3/replication/replication_server.cpp | 186 +- .../v3/replication/replication_server.hpp | 8 +- src/storage/v3/shard.cpp | 1849 +++++++++++++++++ src/storage/v3/shard.hpp | 617 ++++++ src/storage/v3/storage.cpp | 1847 +--------------- src/storage/v3/storage.hpp | 601 +----- src/storage/v3/vertex_accessor.hpp | 4 +- tests/unit/CMakeLists.txt | 6 +- tests/unit/query_v2_interpreter.cpp | 50 +- tests/unit/result_stream_faker.hpp | 4 +- tests/unit/storage_v3.cpp | 180 +- tests/unit/storage_v3_test_utils.cpp | 2 +- tests/unit/storage_v3_test_utils.hpp | 2 +- tests/unit/storage_v3_vertex_accessors.cpp | 44 +- 24 files changed, 2869 insertions(+), 2775 deletions(-) create mode 100644 src/storage/v3/shard.cpp create mode 100644 src/storage/v3/shard.hpp diff --git a/src/glue/v2/communication.cpp b/src/glue/v2/communication.cpp index 55dfd8838..d05b521fe 100644 --- a/src/glue/v2/communication.cpp +++ b/src/glue/v2/communication.cpp @@ -16,7 +16,7 @@ #include <vector> #include "storage/v3/edge_accessor.hpp" -#include "storage/v3/storage.hpp" +#include "storage/v3/shard.hpp" #include "storage/v3/vertex_accessor.hpp" #include "utils/temporal.hpp" @@ -64,16 +64,16 @@ query::v2::TypedValue ToTypedValue(const Value &value) { } storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const query::v2::VertexAccessor &vertex, - const storage::v3::Storage &db, storage::v3::View view) { + const storage::v3::Shard &db, storage::v3::View view) { return ToBoltVertex(vertex.impl_, db, view); } storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const query::v2::EdgeAccessor &edge, - const storage::v3::Storage &db, storage::v3::View view) { + const storage::v3::Shard &db, storage::v3::View view) { return ToBoltEdge(edge.impl_, db, view); } -storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const storage::v3::Storage &db, +storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const storage::v3::Shard &db, storage::v3::View view) { switch (value.type()) { case query::v2::TypedValue::Type::Null: @@ -132,7 +132,7 @@ storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const } storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3::VertexAccessor &vertex, - const storage::v3::Storage &db, storage::v3::View view) { + const storage::v3::Shard &db, storage::v3::View view) { // TODO(jbajic) Fix bolt communication auto id = communication::bolt::Id::FromUint(0); auto maybe_labels = vertex.Labels(view); @@ -152,7 +152,7 @@ storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3: } storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::EdgeAccessor &edge, - const storage::v3::Storage &db, storage::v3::View view) { + const storage::v3::Shard &db, storage::v3::View view) { // TODO(jbajic) Fix bolt communication auto id = communication::bolt::Id::FromUint(0); auto from = communication::bolt::Id::FromUint(0); @@ -167,7 +167,7 @@ storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::Edg return communication::bolt::Edge{id, from, to, type, properties}; } -storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Storage &db, +storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Shard &db, storage::v3::View view) { std::vector<communication::bolt::Vertex> vertices; vertices.reserve(path.vertices().size()); diff --git a/src/glue/v2/communication.hpp b/src/glue/v2/communication.hpp index 13bf96fca..794912724 100644 --- a/src/glue/v2/communication.hpp +++ b/src/glue/v2/communication.hpp @@ -28,36 +28,36 @@ namespace memgraph::glue::v2 { /// @param storage::v3::VertexAccessor for converting to /// communication::bolt::Vertex. -/// @param storage::v3::Storage for getting label and property names. +/// @param storage::v3::Shard for getting label and property names. /// @param storage::v3::View for deciding which vertex attributes are visible. /// /// @throw std::bad_alloc storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3::VertexAccessor &vertex, - const storage::v3::Storage &db, storage::v3::View view); + const storage::v3::Shard &db, storage::v3::View view); /// @param storage::v3::EdgeAccessor for converting to communication::bolt::Edge. -/// @param storage::v3::Storage for getting edge type and property names. +/// @param storage::v3::Shard for getting edge type and property names. /// @param storage::v3::View for deciding which edge attributes are visible. /// /// @throw std::bad_alloc storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::EdgeAccessor &edge, - const storage::v3::Storage &db, storage::v3::View view); + const storage::v3::Shard &db, storage::v3::View view); /// @param query::v2::Path for converting to communication::bolt::Path. -/// @param storage::v3::Storage for ToBoltVertex and ToBoltEdge. +/// @param storage::v3::Shard for ToBoltVertex and ToBoltEdge. /// @param storage::v3::View for ToBoltVertex and ToBoltEdge. /// /// @throw std::bad_alloc -storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Storage &db, +storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::Path &path, const storage::v3::Shard &db, storage::v3::View view); /// @param query::v2::TypedValue for converting to communication::bolt::Value. -/// @param storage::v3::Storage for ToBoltVertex and ToBoltEdge. +/// @param storage::v3::Shard for ToBoltVertex and ToBoltEdge. /// @param storage::v3::View for ToBoltVertex and ToBoltEdge. /// /// @throw std::bad_alloc storage::v3::Result<communication::bolt::Value> ToBoltValue(const query::v2::TypedValue &value, - const storage::v3::Storage &db, storage::v3::View view); + const storage::v3::Shard &db, storage::v3::View view); query::v2::TypedValue ToTypedValue(const communication::bolt::Value &value); diff --git a/src/query/v2/db_accessor.hpp b/src/query/v2/db_accessor.hpp index beacf2138..a652ca656 100644 --- a/src/query/v2/db_accessor.hpp +++ b/src/query/v2/db_accessor.hpp @@ -212,7 +212,7 @@ inline VertexAccessor EdgeAccessor::From() const { return VertexAccessor(impl_.F inline bool EdgeAccessor::IsCycle() const { return To() == From(); } class DbAccessor final { - storage::v3::Storage::Accessor *accessor_; + storage::v3::Shard::Accessor *accessor_; class VerticesIterable final { storage::v3::VerticesIterable iterable_; @@ -244,7 +244,7 @@ class DbAccessor final { }; public: - explicit DbAccessor(storage::v3::Storage::Accessor *accessor) : accessor_(accessor) {} + explicit DbAccessor(storage::v3::Shard::Accessor *accessor) : accessor_(accessor) {} // TODO(jbajic) Fix Remove Gid // NOLINTNEXTLINE(readability-convert-member-functions-to-static) @@ -348,11 +348,20 @@ class DbAccessor final { return {std::make_optional<VertexAccessor>(*value)}; } - storage::v3::PropertyId NameToProperty(const std::string_view name) { return accessor_->NameToProperty(name); } + // TODO(jbajic) Query engine should have a map of labels, properties and edge + // types + // NOLINTNEXTLINE(readability-convert-member-functions-to-static) + storage::v3::PropertyId NameToProperty(const std::string_view /*name*/) { + return storage::v3::PropertyId::FromUint(0); + } - storage::v3::LabelId NameToLabel(const std::string_view name) { return accessor_->NameToLabel(name); } + // NOLINTNEXTLINE(readability-convert-member-functions-to-static) + storage::v3::LabelId NameToLabel(const std::string_view /*name*/) { return storage::v3::LabelId::FromUint(0); } - storage::v3::EdgeTypeId NameToEdgeType(const std::string_view name) { return accessor_->NameToEdgeType(name); } + // NOLINTNEXTLINE(readability-convert-member-functions-to-static) + storage::v3::EdgeTypeId NameToEdgeType(const std::string_view /*name*/) { + return storage::v3::EdgeTypeId::FromUint(0); + } const std::string &PropertyToName(storage::v3::PropertyId prop) const { return accessor_->PropertyToName(prop); } diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp index c65b5e971..86825dd5e 100644 --- a/src/query/v2/interpreter.cpp +++ b/src/query/v2/interpreter.cpp @@ -43,6 +43,7 @@ #include "query/v2/trigger.hpp" #include "query/v2/typed_value.hpp" #include "storage/v3/property_value.hpp" +#include "storage/v3/shard.hpp" #include "storage/v3/storage.hpp" #include "utils/algorithm.hpp" #include "utils/csv_parsing.hpp" @@ -127,7 +128,7 @@ std::optional<std::string> GetOptionalStringValue(query::v2::Expression *express class ReplQueryHandler final : public query::v2::ReplicationQueryHandler { public: - explicit ReplQueryHandler(storage::v3::Storage *db) : db_(db) {} + explicit ReplQueryHandler(storage::v3::Shard *db) : db_(db) {} /// @throw QueryRuntimeException if an error ocurred. void SetReplicationRole(ReplicationQuery::ReplicationRole replication_role, std::optional<int64_t> port) override { @@ -255,7 +256,7 @@ class ReplQueryHandler final : public query::v2::ReplicationQueryHandler { } private: - storage::v3::Storage *db_; + storage::v3::Shard *db_; }; /// returns false if the replication role can't be set /// @throw QueryRuntimeException if an error ocurred. @@ -913,7 +914,7 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp callback.header = {"property_name", "property_type"}; callback.fn = [interpreter_context, primary_label = schema_query->label_]() { auto *db = interpreter_context->db; - const auto label = db->NameToLabel(primary_label.name); + const auto label = interpreter_context->NameToLabelId(primary_label.name); const auto *schema = db->GetSchema(label); std::vector<std::vector<TypedValue>> results; if (schema) { @@ -938,11 +939,11 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp callback.fn = [interpreter_context, primary_label = schema_query->label_, schema_type_map = std::move(schema_type_map)]() { auto *db = interpreter_context->db; - const auto label = db->NameToLabel(primary_label.name); + const auto label = interpreter_context->NameToLabelId(primary_label.name); std::vector<storage::v3::SchemaProperty> schemas_types; schemas_types.reserve(schema_type_map.size()); for (const auto &schema_type : schema_type_map) { - auto property_id = db->NameToProperty(schema_type.first.name); + auto property_id = interpreter_context->NameToPropertyId(schema_type.first.name); schemas_types.push_back({property_id, schema_type.second}); } if (!db->CreateSchema(label, schemas_types)) { @@ -957,7 +958,7 @@ Callback HandleSchemaQuery(SchemaQuery *schema_query, InterpreterContext *interp case SchemaQuery::Action::DROP_SCHEMA: { callback.fn = [interpreter_context, primary_label = schema_query->label_]() { auto *db = interpreter_context->db; - const auto label = db->NameToLabel(primary_label.name); + const auto label = interpreter_context->NameToLabelId(primary_label.name); if (!db->DropSchema(label)) { throw QueryException(fmt::format("Schema on label :{} does not exist!", primary_label.name)); @@ -1138,7 +1139,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *strea using RWType = plan::ReadWriteTypeChecker::RWType; } // namespace -InterpreterContext::InterpreterContext(storage::v3::Storage *db, const InterpreterConfig config, +InterpreterContext::InterpreterContext(storage::v3::Shard *db, const InterpreterConfig config, const std::filesystem::path &data_directory) : db(db), trigger_store(data_directory / "triggers"), config(config), streams{this, data_directory / "streams"} {} @@ -1157,8 +1158,8 @@ PreparedQuery Interpreter::PrepareTransactionQuery(std::string_view query_upper) in_explicit_transaction_ = true; expect_rollback_ = false; - db_accessor_ = std::make_unique<storage::v3::Storage::Accessor>( - interpreter_context_->db->Access(GetIsolationLevelOverride())); + db_accessor_ = + std::make_unique<storage::v3::Shard::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride())); execution_db_accessor_.emplace(db_accessor_.get()); if (interpreter_context_->trigger_store.HasTriggers()) { @@ -1427,14 +1428,14 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans } }; - auto label = interpreter_context->db->NameToLabel(index_query->label_.name); + auto label = interpreter_context->NameToLabelId(index_query->label_.name); std::vector<storage::v3::PropertyId> properties; std::vector<std::string> properties_string; properties.reserve(index_query->properties_.size()); properties_string.reserve(index_query->properties_.size()); for (const auto &prop : index_query->properties_) { - properties.push_back(interpreter_context->db->NameToProperty(prop.name)); + properties.push_back(interpreter_context->NameToPropertyId(prop.name)); properties_string.push_back(prop.name); } auto properties_stringified = utils::Join(properties_string, ", "); @@ -1842,7 +1843,7 @@ PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_expli [interpreter_context](AnyStream *stream, std::optional<int> n) -> std::optional<QueryHandlerResult> { if (auto maybe_error = interpreter_context->db->CreateSnapshot(); maybe_error.HasError()) { switch (maybe_error.GetError()) { - case storage::v3::Storage::CreateSnapshotError::DisabledForReplica: + case storage::v3::Shard::CreateSnapshotError::DisabledForReplica: throw utils::BasicException( "Failed to create a snapshot. Replica instances are not allowed to create them."); } @@ -1897,8 +1898,8 @@ PreparedQuery PrepareVersionQuery(ParsedQuery parsed_query, const bool in_explic } PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transaction, - std::map<std::string, TypedValue> *summary, InterpreterContext *interpreter_context, - storage::v3::Storage *db, utils::MemoryResource *execution_memory) { + std::map<std::string, TypedValue> * /*summary*/, InterpreterContext *interpreter_context, + storage::v3::Shard *db, utils::MemoryResource * /*execution_memory*/) { if (in_explicit_transaction) { throw InfoInMulticommandTxException(); } @@ -1994,13 +1995,13 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_ auto *constraint_query = utils::Downcast<ConstraintQuery>(parsed_query.query); std::function<void(Notification &)> handler; - auto label = interpreter_context->db->NameToLabel(constraint_query->constraint_.label.name); + auto label = interpreter_context->NameToLabelId(constraint_query->constraint_.label.name); std::vector<storage::v3::PropertyId> properties; std::vector<std::string> properties_string; properties.reserve(constraint_query->constraint_.properties.size()); properties_string.reserve(constraint_query->constraint_.properties.size()); for (const auto &prop : constraint_query->constraint_.properties) { - properties.push_back(interpreter_context->db->NameToProperty(prop.name)); + properties.push_back(interpreter_context->NameToPropertyId(prop.name)); properties_string.push_back(prop.name); } auto properties_stringified = utils::Join(properties_string, ", "); @@ -2259,8 +2260,8 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string, (utils::Downcast<CypherQuery>(parsed_query.query) || utils::Downcast<ExplainQuery>(parsed_query.query) || utils::Downcast<ProfileQuery>(parsed_query.query) || utils::Downcast<DumpQuery>(parsed_query.query) || utils::Downcast<TriggerQuery>(parsed_query.query))) { - db_accessor_ = std::make_unique<storage::v3::Storage::Accessor>( - interpreter_context_->db->Access(GetIsolationLevelOverride())); + db_accessor_ = + std::make_unique<storage::v3::Shard::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride())); execution_db_accessor_.emplace(db_accessor_.get()); if (utils::Downcast<CypherQuery>(parsed_query.query) && interpreter_context_->trigger_store.HasTriggers()) { diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp index d3da9c724..75267ae88 100644 --- a/src/query/v2/interpreter.hpp +++ b/src/query/v2/interpreter.hpp @@ -31,6 +31,7 @@ #include "query/v2/trigger.hpp" #include "query/v2/typed_value.hpp" #include "storage/v3/isolation_level.hpp" +#include "storage/v3/name_id_mapper.hpp" #include "utils/event_counter.hpp" #include "utils/logging.hpp" #include "utils/memory.hpp" @@ -166,10 +167,10 @@ struct PreparedQuery { * been passed to an `Interpreter` instance. */ struct InterpreterContext { - explicit InterpreterContext(storage::v3::Storage *db, InterpreterConfig config, + explicit InterpreterContext(storage::v3::Shard *db, InterpreterConfig config, const std::filesystem::path &data_directory); - storage::v3::Storage *db; + storage::v3::Shard *db; std::optional<double> tsc_frequency{utils::GetTSCFrequency()}; std::atomic<bool> is_shutting_down{false}; @@ -186,6 +187,22 @@ struct InterpreterContext { const InterpreterConfig config; query::v2::stream::Streams streams; + + storage::v3::LabelId NameToLabelId(std::string_view label_name) { + return storage::v3::LabelId::FromUint(query_id_mapper.NameToId(label_name)); + } + + storage::v3::PropertyId NameToPropertyId(std::string_view property_name) { + return storage::v3::PropertyId::FromUint(query_id_mapper.NameToId(property_name)); + } + + storage::v3::EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) { + return storage::v3::EdgeTypeId::FromUint(query_id_mapper.NameToId(edge_type_name)); + } + + private: + // TODO Replace with local map of labels, properties and edge type ids + storage::v3::NameIdMapper query_id_mapper; }; /// Function that is used to tell all active interpreters that they should stop @@ -316,7 +333,7 @@ class Interpreter final { // This cannot be std::optional because we need to move this accessor later on into a lambda capture // which is assigned to std::function. std::function requires every object to be copyable, so we // move this unique_ptr into a shrared_ptr. - std::unique_ptr<storage::v3::Storage::Accessor> db_accessor_; + std::unique_ptr<storage::v3::Shard::Accessor> db_accessor_; std::optional<DbAccessor> execution_db_accessor_; std::optional<TriggerContextCollector> trigger_context_collector_; bool in_explicit_transaction_{false}; diff --git a/src/query/v2/stream/streams.cpp b/src/query/v2/stream/streams.cpp index 563e1401f..026976127 100644 --- a/src/query/v2/stream/streams.cpp +++ b/src/query/v2/stream/streams.cpp @@ -83,7 +83,7 @@ std::pair<TypedValue /*query*/, TypedValue /*parameters*/> ExtractTransformation template <typename TMessage> void CallCustomTransformation(const std::string &transformation_name, const std::vector<TMessage> &messages, - mgp_result &result, storage::v3::Storage::Accessor &storage_accessor, + mgp_result &result, storage::v3::Shard::Accessor &storage_accessor, utils::MemoryResource &memory_resource, const std::string &stream_name) { DbAccessor db_accessor{&storage_accessor}; { diff --git a/src/storage/v3/CMakeLists.txt b/src/storage/v3/CMakeLists.txt index 272d2cd03..bd93ba607 100644 --- a/src/storage/v3/CMakeLists.txt +++ b/src/storage/v3/CMakeLists.txt @@ -14,6 +14,7 @@ set(storage_v3_src_files vertex_accessor.cpp schemas.cpp schema_validator.cpp + shard.cpp storage.cpp) # #### Replication ##### diff --git a/src/storage/v3/edge_accessor.hpp b/src/storage/v3/edge_accessor.hpp index cf8b658d8..8a17b163c 100644 --- a/src/storage/v3/edge_accessor.hpp +++ b/src/storage/v3/edge_accessor.hpp @@ -31,7 +31,7 @@ struct Constraints; class EdgeAccessor final { private: - friend class Storage; + friend class Shard; public: EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction, diff --git a/src/storage/v3/replication/replication_client.cpp b/src/storage/v3/replication/replication_client.cpp index ecc3ffbf4..9a5594c5b 100644 --- a/src/storage/v3/replication/replication_client.cpp +++ b/src/storage/v3/replication/replication_client.cpp @@ -30,10 +30,10 @@ template <typename> } // namespace ////// ReplicationClient ////// -Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint, - const replication::ReplicationMode mode, - const replication::ReplicationClientConfig &config) - : name_(std::move(name)), storage_(storage), mode_(mode) { +Shard::ReplicationClient::ReplicationClient(std::string name, Shard *shard, const io::network::Endpoint &endpoint, + const replication::ReplicationMode mode, + const replication::ReplicationClientConfig &config) + : name_(std::move(name)), shard_(shard), mode_(mode) { if (config.ssl) { rpc_context_.emplace(config.ssl->key_file, config.ssl->cert_file); } else { @@ -54,14 +54,14 @@ Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage } } -void Storage::ReplicationClient::TryInitializeClientAsync() { +void Shard::ReplicationClient::TryInitializeClientAsync() { thread_pool_.AddTask([this] { rpc_client_->Abort(); this->TryInitializeClientSync(); }); } -void Storage::ReplicationClient::FrequentCheck() { +void Shard::ReplicationClient::FrequentCheck() { const auto is_success = std::invoke([this]() { try { auto stream{rpc_client_->Stream<replication::FrequentHeartbeatRpc>()}; @@ -87,15 +87,15 @@ void Storage::ReplicationClient::FrequentCheck() { } /// @throws rpc::RpcFailedException -void Storage::ReplicationClient::InitializeClient() { +void Shard::ReplicationClient::InitializeClient() { uint64_t current_commit_timestamp{kTimestampInitialId}; - auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(storage_->last_commit_timestamp_, storage_->epoch_id_)}; + auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(shard_->last_commit_timestamp_, shard_->epoch_id_)}; const auto response = stream.AwaitResponse(); std::optional<uint64_t> branching_point; - if (response.epoch_id != storage_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) { - const auto &epoch_history = storage_->epoch_history_; + if (response.epoch_id != shard_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) { + const auto &epoch_history = shard_->epoch_history_; const auto epoch_info_iter = std::find_if(epoch_history.crbegin(), epoch_history.crend(), [&](const auto &epoch_info) { return epoch_info.first == response.epoch_id; }); @@ -115,8 +115,8 @@ void Storage::ReplicationClient::InitializeClient() { current_commit_timestamp = response.current_commit_timestamp; spdlog::trace("Current timestamp on replica: {}", current_commit_timestamp); - spdlog::trace("Current timestamp on main: {}", storage_->last_commit_timestamp_); - if (current_commit_timestamp == storage_->last_commit_timestamp_) { + spdlog::trace("Current timestamp on main: {}", shard_->last_commit_timestamp_); + if (current_commit_timestamp == shard_->last_commit_timestamp_) { spdlog::debug("Replica '{}' up to date", name_); std::unique_lock client_guard{client_lock_}; replica_state_.store(replication::ReplicaState::READY); @@ -130,7 +130,7 @@ void Storage::ReplicationClient::InitializeClient() { } } -void Storage::ReplicationClient::TryInitializeClientSync() { +void Shard::ReplicationClient::TryInitializeClientSync() { try { InitializeClient(); } catch (const rpc::RpcFailedException &) { @@ -141,19 +141,19 @@ void Storage::ReplicationClient::TryInitializeClientSync() { } } -void Storage::ReplicationClient::HandleRpcFailure() { +void Shard::ReplicationClient::HandleRpcFailure() { spdlog::error(utils::MessageWithLink("Couldn't replicate data to {}.", name_, "https://memgr.ph/replication")); TryInitializeClientAsync(); } -replication::SnapshotRes Storage::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) { +replication::SnapshotRes Shard::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) { auto stream{rpc_client_->Stream<replication::SnapshotRpc>()}; replication::Encoder encoder(stream.GetBuilder()); encoder.WriteFile(path); return stream.AwaitResponse(); } -replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles( +replication::WalFilesRes Shard::ReplicationClient::TransferWalFiles( const std::vector<std::filesystem::path> &wal_files) { MG_ASSERT(!wal_files.empty(), "Wal files list is empty!"); auto stream{rpc_client_->Stream<replication::WalFilesRpc>(wal_files.size())}; @@ -166,7 +166,7 @@ replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles( return stream.AwaitResponse(); } -void Storage::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) { +void Shard::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) { std::unique_lock guard(client_lock_); const auto status = replica_state_.load(); switch (status) { @@ -190,7 +190,7 @@ void Storage::ReplicationClient::StartTransactionReplication(const uint64_t curr case replication::ReplicaState::READY: MG_ASSERT(!replica_stream_); try { - replica_stream_.emplace(ReplicaStream{this, storage_->last_commit_timestamp_, current_wal_seq_num}); + replica_stream_.emplace(ReplicaStream{this, shard_->last_commit_timestamp_, current_wal_seq_num}); replica_state_.store(replication::ReplicaState::REPLICATING); } catch (const rpc::RpcFailedException &) { replica_state_.store(replication::ReplicaState::INVALID); @@ -200,7 +200,7 @@ void Storage::ReplicationClient::StartTransactionReplication(const uint64_t curr } } -void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) { +void Shard::ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) { // We can only check the state because it guarantees to be only // valid during a single transaction replication (if the assumption // that this and other transaction replication functions can only be @@ -220,7 +220,7 @@ void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void } } -void Storage::ReplicationClient::FinalizeTransactionReplication() { +void Shard::ReplicationClient::FinalizeTransactionReplication() { // We can only check the state because it guarantees to be only // valid during a single transaction replication (if the assumption // that this and other transaction replication functions can only be @@ -271,7 +271,7 @@ void Storage::ReplicationClient::FinalizeTransactionReplication() { } } -void Storage::ReplicationClient::FinalizeTransactionReplicationInternal() { +void Shard::ReplicationClient::FinalizeTransactionReplicationInternal() { MG_ASSERT(replica_stream_, "Missing stream for transaction deltas"); try { auto response = replica_stream_->Finalize(); @@ -293,9 +293,9 @@ void Storage::ReplicationClient::FinalizeTransactionReplicationInternal() { } } -void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) { +void Shard::ReplicationClient::RecoverReplica(uint64_t replica_commit) { while (true) { - auto file_locker = storage_->file_retainer_.AddLocker(); + auto file_locker = shard_->file_retainer_.AddLocker(); const auto steps = GetRecoverySteps(replica_commit, &file_locker); for (const auto &recovery_step : steps) { @@ -312,11 +312,11 @@ void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) { auto response = TransferWalFiles(arg); replica_commit = response.current_commit_timestamp; } else if constexpr (std::is_same_v<StepType, RecoveryCurrentWal>) { - if (storage_->wal_file_ && storage_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) { - storage_->wal_file_->DisableFlushing(); + if (shard_->wal_file_ && shard_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) { + shard_->wal_file_->DisableFlushing(); spdlog::debug("Sending current wal file"); replica_commit = ReplicateCurrentWal(); - storage_->wal_file_->EnableFlushing(); + shard_->wal_file_->EnableFlushing(); } } else { static_assert(always_false_v<T>, "Missing type from variant visitor"); @@ -345,20 +345,20 @@ void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) { // By adding this lock, we can avoid that, and go to RECOVERY immediately. std::unique_lock client_guard{client_lock_}; SPDLOG_INFO("Replica timestamp: {}", replica_commit); - SPDLOG_INFO("Last commit: {}", storage_->last_commit_timestamp_); - if (storage_->last_commit_timestamp_ == replica_commit) { + SPDLOG_INFO("Last commit: {}", shard_->last_commit_timestamp_); + if (shard_->last_commit_timestamp_ == replica_commit) { replica_state_.store(replication::ReplicaState::READY); return; } } } -uint64_t Storage::ReplicationClient::ReplicateCurrentWal() { - const auto &wal_file = storage_->wal_file_; +uint64_t Shard::ReplicationClient::ReplicateCurrentWal() { + const auto &wal_file = shard_->wal_file_; auto stream = TransferCurrentWalFile(); stream.AppendFilename(wal_file->Path().filename()); utils::InputFile file; - MG_ASSERT(file.Open(storage_->wal_file_->Path()), "Failed to open current WAL file!"); + MG_ASSERT(file.Open(shard_->wal_file_->Path()), "Failed to open current WAL file!"); const auto [buffer, buffer_size] = wal_file->CurrentFileBuffer(); stream.AppendSize(file.GetSize() + buffer_size); stream.AppendFileData(&file); @@ -387,23 +387,23 @@ uint64_t Storage::ReplicationClient::ReplicateCurrentWal() { /// recovery steps, so we can safely send it to the replica. /// We assume that the property of preserving at least 1 WAL before the snapshot /// is satisfied as we extract the timestamp information from it. -std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient::GetRecoverySteps( +std::vector<Shard::ReplicationClient::RecoveryStep> Shard::ReplicationClient::GetRecoverySteps( const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) { // First check if we can recover using the current wal file only // otherwise save the seq_num of the current wal file // This lock is also necessary to force the missed transaction to finish. std::optional<uint64_t> current_wal_seq_num; std::optional<uint64_t> current_wal_from_timestamp; - if (storage_->wal_file_) { - current_wal_seq_num.emplace(storage_->wal_file_->SequenceNumber()); - current_wal_from_timestamp.emplace(storage_->wal_file_->FromTimestamp()); + if (shard_->wal_file_) { + current_wal_seq_num.emplace(shard_->wal_file_->SequenceNumber()); + current_wal_from_timestamp.emplace(shard_->wal_file_->FromTimestamp()); } auto locker_acc = file_locker->Access(); - auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_, current_wal_seq_num); + auto wal_files = durability::GetWalFiles(shard_->wal_directory_, shard_->uuid_, current_wal_seq_num); MG_ASSERT(wal_files, "Wal files could not be loaded"); - auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_); + auto snapshot_files = durability::GetSnapshotFiles(shard_->snapshot_directory_, shard_->uuid_); std::optional<durability::SnapshotDurabilityInfo> latest_snapshot; if (!snapshot_files.empty()) { std::sort(snapshot_files.begin(), snapshot_files.end()); @@ -529,13 +529,13 @@ std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient } ////// TimeoutDispatcher ////// -void Storage::ReplicationClient::TimeoutDispatcher::WaitForTaskToFinish() { +void Shard::ReplicationClient::TimeoutDispatcher::WaitForTaskToFinish() { // Wait for the previous timeout task to finish std::unique_lock main_guard(main_lock); main_cv.wait(main_guard, [&] { return finished; }); } -void Storage::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const double timeout) { +void Shard::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const double timeout) { timeout_pool.AddTask([timeout, this] { finished = false; using std::chrono::steady_clock; @@ -553,65 +553,65 @@ void Storage::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const doubl }); } ////// ReplicaStream ////// -Storage::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self, - const uint64_t previous_commit_timestamp, - const uint64_t current_seq_num) +Shard::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self, + const uint64_t previous_commit_timestamp, + const uint64_t current_seq_num) : self_(self), stream_(self_->rpc_client_->Stream<replication::AppendDeltasRpc>(previous_commit_timestamp, current_seq_num)) { replication::Encoder encoder{stream_.GetBuilder()}; - encoder.WriteString(self_->storage_->epoch_id_); + encoder.WriteString(self_->shard_->epoch_id_); } -void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex, - uint64_t final_commit_timestamp) { +void Shard::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex, + uint64_t final_commit_timestamp) { replication::Encoder encoder(stream_.GetBuilder()); - EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, self_->storage_->config_.items, delta, vertex, + EncodeDelta(&encoder, &self_->shard_->name_id_mapper_, self_->shard_->config_.items, delta, vertex, final_commit_timestamp); } -void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge, - uint64_t final_commit_timestamp) { +void Shard::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge, + uint64_t final_commit_timestamp) { replication::Encoder encoder(stream_.GetBuilder()); - EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, delta, edge, final_commit_timestamp); + EncodeDelta(&encoder, &self_->shard_->name_id_mapper_, delta, edge, final_commit_timestamp); } -void Storage::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) { +void Shard::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) { replication::Encoder encoder(stream_.GetBuilder()); EncodeTransactionEnd(&encoder, final_commit_timestamp); } -void Storage::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation, - LabelId label, const std::set<PropertyId> &properties, - uint64_t timestamp) { +void Shard::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation, + LabelId label, const std::set<PropertyId> &properties, + uint64_t timestamp) { replication::Encoder encoder(stream_.GetBuilder()); - EncodeOperation(&encoder, &self_->storage_->name_id_mapper_, operation, label, properties, timestamp); + EncodeOperation(&encoder, &self_->shard_->name_id_mapper_, operation, label, properties, timestamp); } -replication::AppendDeltasRes Storage::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); } +replication::AppendDeltasRes Shard::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); } ////// CurrentWalHandler ////// -Storage::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self) +Shard::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self) : self_(self), stream_(self_->rpc_client_->Stream<replication::CurrentWalRpc>()) {} -void Storage::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) { +void Shard::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) { replication::Encoder encoder(stream_.GetBuilder()); encoder.WriteString(filename); } -void Storage::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) { +void Shard::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) { replication::Encoder encoder(stream_.GetBuilder()); encoder.WriteUint(size); } -void Storage::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) { +void Shard::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) { replication::Encoder encoder(stream_.GetBuilder()); encoder.WriteFileData(file); } -void Storage::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) { +void Shard::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) { replication::Encoder encoder(stream_.GetBuilder()); encoder.WriteBuffer(buffer, buffer_size); } -replication::CurrentWalRes Storage::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); } +replication::CurrentWalRes Shard::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); } } // namespace memgraph::storage::v3 diff --git a/src/storage/v3/replication/replication_client.hpp b/src/storage/v3/replication/replication_client.hpp index f0c47f2e0..7a1731f5f 100644 --- a/src/storage/v3/replication/replication_client.hpp +++ b/src/storage/v3/replication/replication_client.hpp @@ -28,7 +28,7 @@ #include "storage/v3/replication/enums.hpp" #include "storage/v3/replication/rpc.hpp" #include "storage/v3/replication/serialization.hpp" -#include "storage/v3/storage.hpp" +#include "storage/v3/shard.hpp" #include "utils/file.hpp" #include "utils/file_locker.hpp" #include "utils/spin_lock.hpp" @@ -37,9 +37,9 @@ namespace memgraph::storage::v3 { -class Storage::ReplicationClient { +class Shard::ReplicationClient { public: - ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint, + ReplicationClient(std::string name, Shard *shard, const io::network::Endpoint &endpoint, replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {}); // Handler used for transfering the current transaction. @@ -149,7 +149,7 @@ class Storage::ReplicationClient { void HandleRpcFailure(); std::string name_; - Storage *storage_; + Shard *shard_; std::optional<communication::ClientContext> rpc_context_; std::optional<rpc::Client> rpc_client_; diff --git a/src/storage/v3/replication/replication_server.cpp b/src/storage/v3/replication/replication_server.cpp index 552214b9a..1905071ce 100644 --- a/src/storage/v3/replication/replication_server.cpp +++ b/src/storage/v3/replication/replication_server.cpp @@ -40,9 +40,9 @@ std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder }; } // namespace -Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::Endpoint endpoint, - const replication::ReplicationServerConfig &config) - : storage_(storage) { +Shard::ReplicationServer::ReplicationServer(Shard *shard, io::network::Endpoint endpoint, + const replication::ReplicationServerConfig &config) + : shard_(shard) { // Create RPC server. if (config.ssl) { rpc_server_context_.emplace(config.ssl->key_file, config.ssl->cert_file, config.ssl->ca_file, @@ -84,21 +84,21 @@ Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::End rpc_server_->Start(); } -void Storage::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::HeartbeatReq req; slk::Load(&req, req_reader); - replication::HeartbeatRes res{true, storage_->last_commit_timestamp_, storage_->epoch_id_}; + replication::HeartbeatRes res{true, shard_->last_commit_timestamp_, shard_->epoch_id_}; slk::Save(res, res_builder); } -void Storage::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::FrequentHeartbeatReq req; slk::Load(&req, req_reader); replication::FrequentHeartbeatRes res{true}; slk::Save(res, res_builder); } -void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::AppendDeltasReq req; slk::Load(&req, req_reader); @@ -107,25 +107,25 @@ void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, sl auto maybe_epoch_id = decoder.ReadString(); MG_ASSERT(maybe_epoch_id, "Invalid replication message"); - if (*maybe_epoch_id != storage_->epoch_id_) { - storage_->epoch_history_.emplace_back(std::move(storage_->epoch_id_), storage_->last_commit_timestamp_); - storage_->epoch_id_ = std::move(*maybe_epoch_id); + if (*maybe_epoch_id != shard_->epoch_id_) { + shard_->epoch_history_.emplace_back(std::move(shard_->epoch_id_), shard_->last_commit_timestamp_); + shard_->epoch_id_ = std::move(*maybe_epoch_id); } - if (storage_->wal_file_) { - if (req.seq_num > storage_->wal_file_->SequenceNumber() || *maybe_epoch_id != storage_->epoch_id_) { - storage_->wal_file_->FinalizeWal(); - storage_->wal_file_.reset(); - storage_->wal_seq_num_ = req.seq_num; + if (shard_->wal_file_) { + if (req.seq_num > shard_->wal_file_->SequenceNumber() || *maybe_epoch_id != shard_->epoch_id_) { + shard_->wal_file_->FinalizeWal(); + shard_->wal_file_.reset(); + shard_->wal_seq_num_ = req.seq_num; } else { - MG_ASSERT(storage_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file"); - storage_->wal_seq_num_ = req.seq_num + 1; + MG_ASSERT(shard_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file"); + shard_->wal_seq_num_ = req.seq_num + 1; } } else { - storage_->wal_seq_num_ = req.seq_num; + shard_->wal_seq_num_ = req.seq_num; } - if (req.previous_commit_timestamp != storage_->last_commit_timestamp_) { + if (req.previous_commit_timestamp != shard_->last_commit_timestamp_) { // Empty the stream bool transaction_complete = false; while (!transaction_complete) { @@ -134,82 +134,82 @@ void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, sl transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type); } - replication::AppendDeltasRes res{false, storage_->last_commit_timestamp_}; + replication::AppendDeltasRes res{false, shard_->last_commit_timestamp_}; slk::Save(res, res_builder); return; } ReadAndApplyDelta(&decoder); - replication::AppendDeltasRes res{true, storage_->last_commit_timestamp_}; + replication::AppendDeltasRes res{true, shard_->last_commit_timestamp_}; slk::Save(res, res_builder); } -void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::SnapshotReq req; slk::Load(&req, req_reader); replication::Decoder decoder(req_reader); - utils::EnsureDirOrDie(storage_->snapshot_directory_); + utils::EnsureDirOrDie(shard_->snapshot_directory_); - const auto maybe_snapshot_path = decoder.ReadFile(storage_->snapshot_directory_); + const auto maybe_snapshot_path = decoder.ReadFile(shard_->snapshot_directory_); MG_ASSERT(maybe_snapshot_path, "Failed to load snapshot!"); spdlog::info("Received snapshot saved to {}", *maybe_snapshot_path); // Clear the database - storage_->vertices_.clear(); - storage_->edges_.clear(); + shard_->vertices_.clear(); + shard_->edges_.clear(); - storage_->constraints_ = Constraints(); - storage_->indices_.label_index = - LabelIndex(&storage_->indices_, &storage_->constraints_, storage_->config_.items, storage_->schema_validator_); - storage_->indices_.label_property_index = LabelPropertyIndex(&storage_->indices_, &storage_->constraints_, - storage_->config_.items, storage_->schema_validator_); + shard_->constraints_ = Constraints(); + shard_->indices_.label_index = + LabelIndex(&shard_->indices_, &shard_->constraints_, shard_->config_.items, shard_->schema_validator_); + shard_->indices_.label_property_index = + LabelPropertyIndex(&shard_->indices_, &shard_->constraints_, shard_->config_.items, shard_->schema_validator_); try { spdlog::debug("Loading snapshot"); auto recovered_snapshot = durability::RecoveredSnapshot{}; - // durability::LoadSnapshot(*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_, - // &storage_->epoch_history_, - // &storage_->name_id_mapper_, &storage_->edge_count_, storage_->config_.items); + // durability::LoadSnapshot(*maybe_snapshot_path, &shard_->vertices_, &shard_->edges_, + // &shard_->epoch_history_, + // &shard_->name_id_mapper_, &shard_->edge_count_, shard_->config_.items); spdlog::debug("Snapshot loaded successfully"); // If this step is present it should always be the first step of // the recovery so we use the UUID we read from snasphost - storage_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid); - storage_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id); + 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; - storage_->edge_id_ = recovery_info.next_edge_id; - storage_->timestamp_ = std::max(storage_->timestamp_, recovery_info.next_timestamp); + shard_->edge_id_ = recovery_info.next_edge_id; + shard_->timestamp_ = std::max(shard_->timestamp_, recovery_info.next_timestamp); - // durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &storage_->indices_, - // &storage_->constraints_, &storage_->vertices_); + // durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &shard_->indices_, + // &shard_->constraints_, &shard_->vertices_); } catch (const durability::RecoveryFailure &e) { LOG_FATAL("Couldn't load the snapshot because of: {}", e.what()); } - replication::SnapshotRes res{true, storage_->last_commit_timestamp_}; + replication::SnapshotRes res{true, shard_->last_commit_timestamp_}; slk::Save(res, res_builder); // Delete other durability files - auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_); + auto snapshot_files = durability::GetSnapshotFiles(shard_->snapshot_directory_, shard_->uuid_); for (const auto &[path, uuid, _] : snapshot_files) { if (path != *maybe_snapshot_path) { - storage_->file_retainer_.DeleteFile(path); + shard_->file_retainer_.DeleteFile(path); } } - auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_); + auto wal_files = durability::GetWalFiles(shard_->wal_directory_, shard_->uuid_); if (wal_files) { for (const auto &wal_file : *wal_files) { - storage_->file_retainer_.DeleteFile(wal_file.path); + shard_->file_retainer_.DeleteFile(wal_file.path); } - storage_->wal_file_.reset(); + shard_->wal_file_.reset(); } } -void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::WalFilesReq req; slk::Load(&req, req_reader); @@ -218,31 +218,31 @@ void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::B replication::Decoder decoder(req_reader); - utils::EnsureDirOrDie(storage_->wal_directory_); + utils::EnsureDirOrDie(shard_->wal_directory_); for (auto i = 0; i < wal_file_number; ++i) { LoadWal(&decoder); } - replication::WalFilesRes res{true, storage_->last_commit_timestamp_}; + replication::WalFilesRes res{true, shard_->last_commit_timestamp_}; slk::Save(res, res_builder); } -void Storage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) { +void Shard::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) { replication::CurrentWalReq req; slk::Load(&req, req_reader); replication::Decoder decoder(req_reader); - utils::EnsureDirOrDie(storage_->wal_directory_); + utils::EnsureDirOrDie(shard_->wal_directory_); LoadWal(&decoder); - replication::CurrentWalRes res{true, storage_->last_commit_timestamp_}; + replication::CurrentWalRes res{true, shard_->last_commit_timestamp_}; slk::Save(res, res_builder); } -void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) { +void Shard::ReplicationServer::LoadWal(replication::Decoder *decoder) { const auto temp_wal_directory = std::filesystem::temp_directory_path() / "memgraph" / durability::kWalDirectory; utils::EnsureDir(temp_wal_directory); auto maybe_wal_path = decoder->ReadFile(temp_wal_directory); @@ -251,22 +251,22 @@ void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) { try { auto wal_info = durability::ReadWalInfo(*maybe_wal_path); if (wal_info.seq_num == 0) { - storage_->uuid_ = wal_info.uuid; + shard_->uuid_ = wal_info.uuid; } - if (wal_info.epoch_id != storage_->epoch_id_) { - storage_->epoch_history_.emplace_back(wal_info.epoch_id, storage_->last_commit_timestamp_); - storage_->epoch_id_ = std::move(wal_info.epoch_id); + if (wal_info.epoch_id != shard_->epoch_id_) { + shard_->epoch_history_.emplace_back(wal_info.epoch_id, shard_->last_commit_timestamp_); + shard_->epoch_id_ = std::move(wal_info.epoch_id); } - if (storage_->wal_file_) { - if (storage_->wal_file_->SequenceNumber() != wal_info.seq_num) { - storage_->wal_file_->FinalizeWal(); - storage_->wal_seq_num_ = wal_info.seq_num; - storage_->wal_file_.reset(); + if (shard_->wal_file_) { + if (shard_->wal_file_->SequenceNumber() != wal_info.seq_num) { + shard_->wal_file_->FinalizeWal(); + shard_->wal_seq_num_ = wal_info.seq_num; + shard_->wal_file_.reset(); } } else { - storage_->wal_seq_num_ = wal_info.seq_num; + shard_->wal_seq_num_ = wal_info.seq_num; } durability::Decoder wal; @@ -285,20 +285,20 @@ void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) { } } -Storage::ReplicationServer::~ReplicationServer() { +Shard::ReplicationServer::~ReplicationServer() { if (rpc_server_) { rpc_server_->Shutdown(); rpc_server_->AwaitShutdown(); } } -uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) { - auto edge_acc = storage_->edges_.access(); - // auto vertex_acc = storage_->vertices_.access(); +uint64_t Shard::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) { + auto edge_acc = shard_->edges_.access(); + // auto vertex_acc = shard_->vertices_.access(); - std::optional<std::pair<uint64_t, Storage::Accessor>> commit_timestamp_and_accessor; + std::optional<std::pair<uint64_t, Shard::Accessor>> 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, shard_->Access()); // } else if (commit_timestamp_and_accessor->first != commit_timestamp) { // throw utils::BasicException("Received more than one transaction!"); // } @@ -306,7 +306,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // }; uint64_t applied_deltas = 0; - auto max_commit_timestamp = storage_->last_commit_timestamp_; + auto max_commit_timestamp = shard_->last_commit_timestamp_; for (bool transaction_complete = false; !transaction_complete; ++applied_deltas) { const auto [timestamp, delta] = ReadDelta(decoder); @@ -316,7 +316,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type); - if (timestamp < storage_->timestamp_) { + if (timestamp < shard_->timestamp_) { continue; } @@ -406,12 +406,12 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Edge {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(), // delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value); - // if (!storage_->config_.items.properties_on_edges) + // if (!shard_->config_.items.properties_on_edges) // throw utils::BasicException( // "Can't set properties on edges because properties on edges " // "are disabled!"); - // auto *transaction = get_transaction(timestamp); + // // auto *transaction = get_transaction(timestamp); // // The following block of code effectively implements `FindEdge` and // // yields an accessor that is only valid for managing the edge's @@ -456,10 +456,10 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // nullptr, // nullptr, // &transaction->transaction_, - // &storage_->indices_, - // &storage_->constraints_, - // storage_->config_.items, - // storage_->schema_validator_}; + // &shard_->indices_, + // &shard_->constraints_, + // shard_->config_.items, + // shard_->schema_validator_}; // auto ret = ea.SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property), // delta.vertex_edge_set_property.value); @@ -481,14 +481,14 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Create label index on :{}", delta.operation_label.label); // // Need to send the timestamp // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label.label), timestamp)) + // if (!shard_->CreateIndex(shard_->NameToLabel(delta.operation_label.label), timestamp)) // throw utils::BasicException("Invalid transaction!"); // break; // } // case durability::WalDeltaData::Type::LABEL_INDEX_DROP: { // spdlog::trace(" Drop label index on :{}", delta.operation_label.label); // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label.label), timestamp)) + // if (!shard_->DropIndex(shard_->NameToLabel(delta.operation_label.label), timestamp)) // throw utils::BasicException("Invalid transaction!"); // break; // } @@ -496,8 +496,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Create label+property index on :{} ({})", delta.operation_label_property.label, // delta.operation_label_property.property); // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label_property.label), - // storage_->NameToProperty(delta.operation_label_property.property), timestamp)) + // if (!shard_->CreateIndex(shard_->NameToLabel(delta.operation_label_property.label), + // shard_->NameToProperty(delta.operation_label_property.property), timestamp)) // throw utils::BasicException("Invalid transaction!"); // break; // } @@ -505,8 +505,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Drop label+property index on :{} ({})", delta.operation_label_property.label, // delta.operation_label_property.property); // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label_property.label), - // storage_->NameToProperty(delta.operation_label_property.property), timestamp)) + // if (!shard_->DropIndex(shard_->NameToLabel(delta.operation_label_property.label), + // shard_->NameToProperty(delta.operation_label_property.property), timestamp)) // throw utils::BasicException("Invalid transaction!"); // break; // } @@ -514,9 +514,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Create existence constraint on :{} ({})", delta.operation_label_property.label, // delta.operation_label_property.property); // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // auto ret = storage_->CreateExistenceConstraint( - // storage_->NameToLabel(delta.operation_label_property.label), - // storage_->NameToProperty(delta.operation_label_property.property), timestamp); + // auto ret = shard_->CreateExistenceConstraint( + // shard_->NameToLabel(delta.operation_label_property.label), + // shard_->NameToProperty(delta.operation_label_property.property), timestamp); // if (!ret.HasValue() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!"); // break; // } @@ -524,8 +524,8 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // spdlog::trace(" Drop existence constraint on :{} ({})", delta.operation_label_property.label, // delta.operation_label_property.property); // if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); - // if (!storage_->DropExistenceConstraint(storage_->NameToLabel(delta.operation_label_property.label), - // storage_->NameToProperty(delta.operation_label_property.property), + // if (!shard_->DropExistenceConstraint(shard_->NameToLabel(delta.operation_label_property.label), + // shard_->NameToProperty(delta.operation_label_property.property), // timestamp)) // throw utils::BasicException("Invalid transaction!"); // break; @@ -537,9 +537,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // ss.str()); if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); // std::set<PropertyId> properties; // for (const auto &prop : delta.operation_label_properties.properties) { - // properties.emplace(storage_->NameToProperty(prop)); + // properties.emplace(shard_->NameToProperty(prop)); // } - // auto ret = storage_->CreateUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label), + // auto ret = shard_->CreateUniqueConstraint(shard_->NameToLabel(delta.operation_label_properties.label), // properties, timestamp); // if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) // throw utils::BasicException("Invalid transaction!"); @@ -552,9 +552,9 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * // ss.str()); if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!"); // std::set<PropertyId> properties; // for (const auto &prop : delta.operation_label_properties.properties) { - // properties.emplace(storage_->NameToProperty(prop)); + // properties.emplace(shard_->NameToProperty(prop)); // } - // auto ret = storage_->DropUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label), + // auto ret = shard_->DropUniqueConstraint(shard_->NameToLabel(delta.operation_label_properties.label), // properties, timestamp); // if (ret != UniqueConstraints::DeletionStatus::SUCCESS) throw utils::BasicException("Invalid transaction!"); // break; @@ -564,7 +564,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder * if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!"); - storage_->last_commit_timestamp_ = max_commit_timestamp; + shard_->last_commit_timestamp_ = max_commit_timestamp; return applied_deltas; } diff --git a/src/storage/v3/replication/replication_server.hpp b/src/storage/v3/replication/replication_server.hpp index 6d6cc256f..6b278a103 100644 --- a/src/storage/v3/replication/replication_server.hpp +++ b/src/storage/v3/replication/replication_server.hpp @@ -11,13 +11,13 @@ #pragma once -#include "storage/v3/storage.hpp" +#include "storage/v3/shard.hpp" namespace memgraph::storage::v3 { -class Storage::ReplicationServer { +class Shard::ReplicationServer { public: - explicit ReplicationServer(Storage *storage, io::network::Endpoint endpoint, + explicit ReplicationServer(Shard *shard, io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config); ReplicationServer(const ReplicationServer &) = delete; ReplicationServer(ReplicationServer &&) = delete; @@ -41,7 +41,7 @@ class Storage::ReplicationServer { std::optional<communication::ServerContext> rpc_server_context_; std::optional<rpc::Server> rpc_server_; - Storage *storage_; + Shard *shard_; }; } // namespace memgraph::storage::v3 diff --git a/src/storage/v3/shard.cpp b/src/storage/v3/shard.cpp new file mode 100644 index 000000000..48040a80c --- /dev/null +++ b/src/storage/v3/shard.cpp @@ -0,0 +1,1849 @@ +// 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. + +#include "storage/v3/shard.hpp" + +#include <algorithm> +#include <atomic> +#include <cstdint> +#include <iterator> +#include <memory> +#include <mutex> +#include <optional> +#include <variant> + +#include <bits/ranges_algo.h> +#include <gflags/gflags.h> +#include <spdlog/spdlog.h> + +#include "io/network/endpoint.hpp" +#include "storage/v3/constraints.hpp" +#include "storage/v3/durability/durability.hpp" +#include "storage/v3/durability/metadata.hpp" +#include "storage/v3/durability/paths.hpp" +#include "storage/v3/durability/snapshot.hpp" +#include "storage/v3/durability/wal.hpp" +#include "storage/v3/edge_accessor.hpp" +#include "storage/v3/id_types.hpp" +#include "storage/v3/indices.hpp" +#include "storage/v3/key_store.hpp" +#include "storage/v3/mvcc.hpp" +#include "storage/v3/property_value.hpp" +#include "storage/v3/replication/config.hpp" +#include "storage/v3/replication/replication_client.hpp" +#include "storage/v3/replication/replication_server.hpp" +#include "storage/v3/replication/rpc.hpp" +#include "storage/v3/schema_validator.hpp" +#include "storage/v3/transaction.hpp" +#include "storage/v3/vertex.hpp" +#include "storage/v3/vertex_accessor.hpp" +#include "storage/v3/vertices_skip_list.hpp" +#include "utils/exceptions.hpp" +#include "utils/file.hpp" +#include "utils/logging.hpp" +#include "utils/memory_tracker.hpp" +#include "utils/message.hpp" +#include "utils/result.hpp" +#include "utils/rw_lock.hpp" +#include "utils/skip_list.hpp" +#include "utils/spin_lock.hpp" +#include "utils/stat.hpp" +#include "utils/uuid.hpp" + +namespace memgraph::storage::v3 { + +using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler; + +namespace { +inline constexpr uint16_t kEpochHistoryRetention = 1000; + +void InsertVertexPKIntoList(auto &container, const PrimaryKey &primary_key) { container.push_back(primary_key); } +} // namespace + +auto AdvanceToVisibleVertex(VerticesSkipList::Iterator it, VerticesSkipList::Iterator end, + std::optional<VertexAccessor> *vertex, Transaction *tx, View view, Indices *indices, + Constraints *constraints, Config::Items config, const SchemaValidator &schema_validator) { + while (it != end) { + *vertex = VertexAccessor::Create(&it->vertex, tx, indices, constraints, config, schema_validator, view); + if (!*vertex) { + ++it; + continue; + } + break; + } + return it; +} + +AllVerticesIterable::Iterator::Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it) + : self_(self), + it_(AdvanceToVisibleVertex(it, self->vertices_accessor_.end(), &self->vertex_, self->transaction_, self->view_, + self->indices_, self_->constraints_, self->config_, *self_->schema_validator_)) {} + +VertexAccessor AllVerticesIterable::Iterator::operator*() const { return *self_->vertex_; } + +AllVerticesIterable::Iterator &AllVerticesIterable::Iterator::operator++() { + ++it_; + it_ = AdvanceToVisibleVertex(it_, self_->vertices_accessor_.end(), &self_->vertex_, self_->transaction_, self_->view_, + self_->indices_, self_->constraints_, self_->config_, *self_->schema_validator_); + return *this; +} + +VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) { + new (&all_vertices_) AllVerticesIterable(std::move(vertices)); +} + +VerticesIterable::VerticesIterable(LabelIndex::Iterable vertices) : type_(Type::BY_LABEL) { + new (&vertices_by_label_) LabelIndex::Iterable(std::move(vertices)); +} + +VerticesIterable::VerticesIterable(LabelPropertyIndex::Iterable vertices) : type_(Type::BY_LABEL_PROPERTY) { + new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(vertices)); +} + +VerticesIterable::VerticesIterable(VerticesIterable &&other) noexcept : type_(other.type_) { + switch (other.type_) { + case Type::ALL: + new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_)); + break; + case Type::BY_LABEL: + new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_)); + break; + case Type::BY_LABEL_PROPERTY: + new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_)); + break; + } +} + +VerticesIterable &VerticesIterable::operator=(VerticesIterable &&other) noexcept { + switch (type_) { + case Type::ALL: + all_vertices_.AllVerticesIterable::~AllVerticesIterable(); + break; + case Type::BY_LABEL: + vertices_by_label_.LabelIndex::Iterable::~Iterable(); + break; + case Type::BY_LABEL_PROPERTY: + vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable(); + break; + } + type_ = other.type_; + switch (other.type_) { + case Type::ALL: + new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_)); + break; + case Type::BY_LABEL: + new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_)); + break; + case Type::BY_LABEL_PROPERTY: + new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_)); + break; + } + return *this; +} + +VerticesIterable::~VerticesIterable() { + switch (type_) { + case Type::ALL: + all_vertices_.AllVerticesIterable::~AllVerticesIterable(); + break; + case Type::BY_LABEL: + vertices_by_label_.LabelIndex::Iterable::~Iterable(); + break; + case Type::BY_LABEL_PROPERTY: + vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable(); + break; + } +} + +VerticesIterable::Iterator VerticesIterable::begin() { + switch (type_) { + case Type::ALL: + return Iterator(all_vertices_.begin()); + case Type::BY_LABEL: + return Iterator(vertices_by_label_.begin()); + case Type::BY_LABEL_PROPERTY: + return Iterator(vertices_by_label_property_.begin()); + } +} + +VerticesIterable::Iterator VerticesIterable::end() { + switch (type_) { + case Type::ALL: + return Iterator(all_vertices_.end()); + case Type::BY_LABEL: + return Iterator(vertices_by_label_.end()); + case Type::BY_LABEL_PROPERTY: + return Iterator(vertices_by_label_property_.end()); + } +} + +VerticesIterable::Iterator::Iterator(AllVerticesIterable::Iterator it) : type_(Type::ALL) { + new (&all_it_) AllVerticesIterable::Iterator(it); +} + +VerticesIterable::Iterator::Iterator(LabelIndex::Iterable::Iterator it) : type_(Type::BY_LABEL) { + new (&by_label_it_) LabelIndex::Iterable::Iterator(it); +} + +VerticesIterable::Iterator::Iterator(LabelPropertyIndex::Iterable::Iterator it) : type_(Type::BY_LABEL_PROPERTY) { + new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(it); +} + +VerticesIterable::Iterator::Iterator(const VerticesIterable::Iterator &other) : type_(other.type_) { + switch (other.type_) { + case Type::ALL: + new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); + break; + case Type::BY_LABEL: + new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); + break; + case Type::BY_LABEL_PROPERTY: + new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); + break; + } +} + +VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(const VerticesIterable::Iterator &other) { + if (this == &other) { + return *this; + } + Destroy(); + type_ = other.type_; + switch (other.type_) { + case Type::ALL: + new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); + break; + case Type::BY_LABEL: + new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); + break; + case Type::BY_LABEL_PROPERTY: + new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); + break; + } + return *this; +} + +VerticesIterable::Iterator::Iterator(VerticesIterable::Iterator &&other) noexcept : type_(other.type_) { + switch (other.type_) { + case Type::ALL: + new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); + break; + case Type::BY_LABEL: + new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); + break; + case Type::BY_LABEL_PROPERTY: + new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); + break; + } +} + +VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(VerticesIterable::Iterator &&other) noexcept { + Destroy(); + type_ = other.type_; + switch (other.type_) { + case Type::ALL: + new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); + break; + case Type::BY_LABEL: + new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); + break; + case Type::BY_LABEL_PROPERTY: + new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); + break; + } + return *this; +} + +VerticesIterable::Iterator::~Iterator() { Destroy(); } + +void VerticesIterable::Iterator::Destroy() noexcept { + switch (type_) { + case Type::ALL: + all_it_.AllVerticesIterable::Iterator::~Iterator(); + break; + case Type::BY_LABEL: + by_label_it_.LabelIndex::Iterable::Iterator::~Iterator(); + break; + case Type::BY_LABEL_PROPERTY: + by_label_property_it_.LabelPropertyIndex::Iterable::Iterator::~Iterator(); + break; + } +} + +VertexAccessor VerticesIterable::Iterator::operator*() const { + switch (type_) { + case Type::ALL: + return *all_it_; + case Type::BY_LABEL: + return *by_label_it_; + case Type::BY_LABEL_PROPERTY: + return *by_label_property_it_; + } +} + +VerticesIterable::Iterator &VerticesIterable::Iterator::operator++() { + switch (type_) { + case Type::ALL: + ++all_it_; + break; + case Type::BY_LABEL: + ++by_label_it_; + break; + case Type::BY_LABEL_PROPERTY: + ++by_label_property_it_; + break; + } + return *this; +} + +bool VerticesIterable::Iterator::operator==(const Iterator &other) const { + switch (type_) { + case Type::ALL: + return all_it_ == other.all_it_; + case Type::BY_LABEL: + return by_label_it_ == other.by_label_it_; + case Type::BY_LABEL_PROPERTY: + return by_label_property_it_ == other.by_label_property_it_; + } +} + +Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key, + const std::optional<PrimaryKey> max_primary_key, Config config) + : primary_label_{primary_label}, + min_primary_key_{min_primary_key}, + max_primary_key_{max_primary_key}, + schema_validator_{schemas_}, + indices_{&constraints_, config.items, schema_validator_}, + isolation_level_{config.transaction.isolation_level}, + config_{config}, + snapshot_directory_{config_.durability.storage_directory / durability::kSnapshotDirectory}, + wal_directory_{config_.durability.storage_directory / durability::kWalDirectory}, + lock_file_path_{config_.durability.storage_directory / durability::kLockFile}, + uuid_{utils::GenerateUUID()}, + epoch_id_{utils::GenerateUUID()}, + global_locker_{file_retainer_.AddLocker()} { + if (config_.durability.snapshot_wal_mode == Config::Durability::SnapshotWalMode::DISABLED && + replication_role_ == ReplicationRole::MAIN) { + spdlog::warn( + "The instance has the MAIN replication role, but durability logs and snapshots are disabled. Please consider " + "enabling durability by using --storage-snapshot-interval-sec and --storage-wal-enabled flags because " + "without write-ahead logs this instance is not replicating any data."); + } + if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || + config_.durability.snapshot_on_exit || config_.durability.recover_on_startup) { + // Create the directory initially to crash the database in case of + // permission errors. This is done early to crash the database on startup + // instead of crashing the database for the first time during runtime (which + // could be an unpleasant surprise). + utils::EnsureDirOrDie(snapshot_directory_); + // Same reasoning as above. + utils::EnsureDirOrDie(wal_directory_); + + // Verify that the user that started the process is the same user that is + // the owner of the storage directory. + durability::VerifyStorageDirectoryOwnerAndProcessUserOrDie(config_.durability.storage_directory); + + // Create the lock file and open a handle to it. This will crash the + // database if it can't open the file for writing or if any other process is + // holding the file opened. + lock_file_handle_.Open(lock_file_path_, utils::OutputFile::Mode::OVERWRITE_EXISTING); + MG_ASSERT(lock_file_handle_.AcquireLock(), + "Couldn't acquire lock on the storage directory {}!\n" + "Another Memgraph process is currently running with the same " + "storage directory, please stop it first before starting this " + "process!", + config_.durability.storage_directory); + } + if (config_.durability.recover_on_startup) { + auto info = std::optional<durability::RecoveryInfo>{}; + + // durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_, &vertices_, + // &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; + } + } + } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || + config_.durability.snapshot_on_exit) { + bool files_moved = false; + auto backup_root = config_.durability.storage_directory / durability::kBackupDirectory; + for (const auto &[path, dirname, what] : + {std::make_tuple(snapshot_directory_, durability::kSnapshotDirectory, "snapshot"), + std::make_tuple(wal_directory_, durability::kWalDirectory, "WAL")}) { + if (!utils::DirExists(path)) continue; + auto backup_curr = backup_root / dirname; + std::error_code error_code; + for (const auto &item : std::filesystem::directory_iterator(path, error_code)) { + utils::EnsureDirOrDie(backup_root); + utils::EnsureDirOrDie(backup_curr); + std::error_code item_error_code; + std::filesystem::rename(item.path(), backup_curr / item.path().filename(), item_error_code); + MG_ASSERT(!item_error_code, "Couldn't move {} file {} because of: {}", what, item.path(), + item_error_code.message()); + files_moved = true; + } + MG_ASSERT(!error_code, "Couldn't backup {} files because of: {}", what, error_code.message()); + } + if (files_moved) { + spdlog::warn( + "Since Memgraph was not supposed to recover on startup and " + "durability is enabled, your current durability files will likely " + "be overridden. To prevent important data loss, Memgraph has stored " + "those files into a .backup directory inside the storage directory."); + } + } + if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) { + // TODO(antaljanosbenjamin): handle snapshots + // snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] { + // if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) { + // switch (maybe_error.GetError()) { + // case CreateSnapshotError::DisabledForReplica: + // spdlog::warn( + // utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication")); + // break; + // } + // } + // }); + } + + if (timestamp_ == kTimestampInitialId) { + commit_log_.emplace(); + } else { + commit_log_.emplace(timestamp_); + } +} + +Shard::~Shard() { + { + // Clear replication data + replication_server_.reset(); + replication_clients_.WithLock([&](auto &clients) { clients.clear(); }); + } + if (wal_file_) { + wal_file_->FinalizeWal(); + wal_file_ = std::nullopt; + } + if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) { + // TODO(antaljanosbenjamin): stop snapshot creation + } + if (config_.durability.snapshot_on_exit) { + if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) { + switch (maybe_error.GetError()) { + case CreateSnapshotError::DisabledForReplica: + spdlog::warn(utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication")); + break; + } + } + } +} + +Shard::Accessor::Accessor(Shard *shard, IsolationLevel isolation_level) + : shard_(shard), + transaction_(shard->CreateTransaction(isolation_level)), + is_transaction_active_(true), + config_(shard->config_.items) {} + +Shard::Accessor::Accessor(Accessor &&other) noexcept + : shard_(other.shard_), + transaction_(std::move(other.transaction_)), + commit_timestamp_(other.commit_timestamp_), + is_transaction_active_(other.is_transaction_active_), + config_(other.config_) { + // Don't allow the other accessor to abort our transaction in destructor. + other.is_transaction_active_ = false; + other.commit_timestamp_.reset(); +} + +Shard::Accessor::~Accessor() { + if (is_transaction_active_) { + Abort(); + } + + FinalizeTransaction(); +} + +ResultSchema<VertexAccessor> Shard::Accessor::CreateVertexAndValidate( + LabelId primary_label, const std::vector<LabelId> &labels, + const std::vector<std::pair<PropertyId, PropertyValue>> &properties) { + if (primary_label != shard_->primary_label_) { + throw utils::BasicException("Cannot add vertex to shard which does not hold the given primary label!"); + } + auto maybe_schema_violation = GetSchemaValidator().ValidateVertexCreate(primary_label, labels, properties); + if (maybe_schema_violation) { + return {std::move(*maybe_schema_violation)}; + } + OOMExceptionEnabler oom_exception; + // Extract key properties + std::vector<PropertyValue> primary_properties; + for ([[maybe_unused]] const auto &[property_id, property_type] : shard_->GetSchema(primary_label)->second) { + // We know there definitely is key in properties since we have validated + primary_properties.push_back( + std::ranges::find_if(properties, [property_id = property_id](const auto &property_pair) { + return property_pair.first == property_id; + })->second); + } + + // Get secondary properties + std::vector<std::pair<PropertyId, PropertyValue>> secondary_properties; + for (const auto &[property_id, property_value] : properties) { + if (!shard_->schemas_.IsPropertyKey(primary_label, property_id)) { + secondary_properties.emplace_back(property_id, property_value); + } + } + + auto acc = shard_->vertices_.access(); + auto *delta = CreateDeleteObjectDelta(&transaction_); + auto [it, inserted] = acc.insert({Vertex{delta, primary_label, primary_properties, labels, secondary_properties}}); + MG_ASSERT(inserted, "The vertex must be inserted here!"); + MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!"); + delta->prev.Set(&it->vertex); + return VertexAccessor{&it->vertex, &transaction_, &shard_->indices_, + &shard_->constraints_, config_, shard_->schema_validator_}; +} + +std::optional<VertexAccessor> Shard::Accessor::FindVertex(std::vector<PropertyValue> primary_key, View view) { + auto acc = shard_->vertices_.access(); + // Later on use label space + auto it = acc.find(primary_key); + if (it == acc.end()) { + return std::nullopt; + } + return VertexAccessor::Create(&it->vertex, &transaction_, &shard_->indices_, &shard_->constraints_, config_, + shard_->schema_validator_, view); +} + +Result<std::optional<VertexAccessor>> Shard::Accessor::DeleteVertex(VertexAccessor *vertex) { + MG_ASSERT(vertex->transaction_ == &transaction_, + "VertexAccessor must be from the same transaction as the storage " + "accessor when deleting a vertex!"); + auto *vertex_ptr = vertex->vertex_; + + if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; + + if (vertex_ptr->deleted) { + return std::optional<VertexAccessor>{}; + } + + if (!vertex_ptr->in_edges.empty() || !vertex_ptr->out_edges.empty()) return Error::VERTEX_HAS_EDGES; + + CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag()); + vertex_ptr->deleted = true; + + return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &shard_->indices_, &shard_->constraints_, + config_, shard_->schema_validator_, true); +} + +Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Shard::Accessor::DetachDeleteVertex( + VertexAccessor *vertex) { + using ReturnType = std::pair<VertexAccessor, std::vector<EdgeAccessor>>; + + MG_ASSERT(vertex->transaction_ == &transaction_, + "VertexAccessor must be from the same transaction as the storage " + "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; + + { + if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; + + if (vertex_ptr->deleted) return std::optional<ReturnType>{}; + + in_edges = vertex_ptr->in_edges; + out_edges = vertex_ptr->out_edges; + } + + std::vector<EdgeAccessor> deleted_edges; + 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_->schema_validator_); + auto ret = DeleteEdge(&e); + if (ret.HasError()) { + MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!"); + return ret.GetError(); + } + + if (ret.GetValue()) { + deleted_edges.push_back(*ret.GetValue()); + } + } + 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_->schema_validator_); + auto ret = DeleteEdge(&e); + if (ret.HasError()) { + MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!"); + return ret.GetError(); + } + + if (ret.GetValue()) { + deleted_edges.push_back(*ret.GetValue()); + } + } + + // We need to check again for serialization errors because we unlocked the + // vertex. Some other transaction could have modified the vertex in the + // meantime if we didn't have any edges to delete. + + if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; + + MG_ASSERT(!vertex_ptr->deleted, "Invalid database state!"); + + CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag()); + vertex_ptr->deleted = true; + + return std::make_optional<ReturnType>(VertexAccessor{vertex_ptr, &transaction_, &shard_->indices_, + &shard_->constraints_, config_, shard_->schema_validator_, true}, + std::move(deleted_edges)); +} + +Result<EdgeAccessor> Shard::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) { + 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; + } + + 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_->schema_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(); + 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_->schema_validator_); +} + +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_; + + if (config_.properties_on_edges) { + auto *edge_ptr = edge_ref.ptr; + + if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR; + + if (edge_ptr->deleted) return std::optional<EdgeAccessor>{}; + } + + auto *from_vertex = edge->from_vertex_; + auto *to_vertex = edge->to_vertex_; + + if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR; + MG_ASSERT(!from_vertex->deleted, "Invalid database state!"); + + if (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); + if (config_.properties_on_edges) { + MG_ASSERT(it != edges->end(), "Invalid database state!"); + } else if (it == edges->end()) { + return false; + } + 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); + + if (config_.properties_on_edges) { + MG_ASSERT((op1 && op2), "Invalid database state!"); + } else { + MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!"); + if (!op1 && !op2) { + // The edge is already deleted. + return std::optional<EdgeAccessor>{}; + } + } + + if (config_.properties_on_edges) { + auto *edge_ptr = edge_ref.ptr; + CreateAndLinkDelta(&transaction_, edge_ptr, Delta::RecreateObjectTag()); + 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); + + // 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_->schema_validator_, true); +} + +const std::string &Shard::Accessor::LabelToName(LabelId label) const { return shard_->LabelToName(label); } + +const std::string &Shard::Accessor::PropertyToName(PropertyId property) const { + return shard_->PropertyToName(property); +} + +const std::string &Shard::Accessor::EdgeTypeToName(EdgeTypeId edge_type) const { + return shard_->EdgeTypeToName(edge_type); +} + +void Shard::Accessor::AdvanceCommand() { ++transaction_.command_id; } + +utils::BasicResult<ConstraintViolation, void> Shard::Accessor::Commit( + const std::optional<uint64_t> desired_commit_timestamp) { + MG_ASSERT(is_transaction_active_, "The transaction is already terminated!"); + MG_ASSERT(!transaction_.must_abort, "The transaction can't be committed!"); + + if (transaction_.deltas.empty()) { + // We don't have to update the commit timestamp here because no one reads + // it. + shard_->commit_log_->MarkFinished(transaction_.start_timestamp); + } else { + // Validate that existence constraints are satisfied for all modified + // vertices. + for (const auto &delta : transaction_.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) { + continue; + } + // No need to take any locks here because we modified this vertex and no + // one else can touch it until we commit. + auto validation_result = ValidateExistenceConstraints(*prev.vertex, shard_->constraints_); + if (validation_result) { + Abort(); + return {*validation_result}; + } + } + + // Result of validating the vertex against unique constraints. It has to be + // declared outside of the critical section scope because its value is + // tested for Abort call which has to be done out of the scope. + std::optional<ConstraintViolation> unique_constraint_violation; + + // Save these so we can mark them used in the commit log. + uint64_t start_timestamp = transaction_.start_timestamp; + + commit_timestamp_.emplace(shard_->CommitTimestamp(desired_commit_timestamp)); + + // Before committing and validating vertices against unique constraints, + // we have to update unique constraints with the vertices that are going + // to be validated/committed. + for (const auto &delta : transaction_.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) { + continue; + } + shard_->constraints_.unique_constraints.UpdateBeforeCommit(prev.vertex, transaction_); + } + + // Validate that unique constraints are satisfied for all modified + // vertices. + for (const auto &delta : transaction_.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) { + continue; + } + + // No need to take any locks here because we modified this vertex and no + // one else can touch it until we commit. + unique_constraint_violation = + shard_->constraints_.unique_constraints.Validate(*prev.vertex, transaction_, *commit_timestamp_); + if (unique_constraint_violation) { + break; + } + } + + if (!unique_constraint_violation) { + // Write transaction to WAL while holding the engine lock to make sure + // that committed transactions are sorted by the commit timestamp in the + // WAL files. We supply the new commit timestamp to the function so that + // it knows what will be the final commit timestamp. The WAL must be + // written before actually committing the transaction (before setting + // the commit timestamp) so that no other transaction can see the + // modifications before they are written to disk. + // Replica can log only the write transaction received from Main + // so the Wal files are consistent + if (shard_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) { + shard_->AppendToWal(transaction_, *commit_timestamp_); + } + + // TODO(antaljanosbenjamin): Figure out: + // 1. How the committed transactions are sorted in `committed_transactions_` + // 2. Why it was necessary to lock `committed_transactions_` when it was not accessed at all + // TODO: Update all deltas to have a local copy of the commit timestamp + MG_ASSERT(transaction_.commit_timestamp != nullptr, "Invalid database state!"); + transaction_.commit_timestamp->store(*commit_timestamp_, std::memory_order_release); + // Replica can only update the last commit timestamp with + // the commits received from main. + if (shard_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) { + // Update the last commit timestamp + shard_->last_commit_timestamp_ = *commit_timestamp_; + } + + shard_->commit_log_->MarkFinished(start_timestamp); + } + + if (unique_constraint_violation) { + Abort(); + return {*unique_constraint_violation}; + } + } + is_transaction_active_ = false; + + return {}; +} + +void Shard::Accessor::Abort() { + MG_ASSERT(is_transaction_active_, "The transaction is already terminated!"); + + for (const auto &delta : transaction_.deltas) { + auto prev = delta.prev.Get(); + switch (prev.type) { + case PreviousPtr::Type::VERTEX: { + auto *vertex = prev.vertex; + Delta *current = vertex->delta; + while (current != nullptr && + current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { + switch (current->action) { + case Delta::Action::REMOVE_LABEL: { + auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label); + MG_ASSERT(it != vertex->labels.end(), "Invalid database state!"); + std::swap(*it, *vertex->labels.rbegin()); + vertex->labels.pop_back(); + break; + } + case Delta::Action::ADD_LABEL: { + auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label); + MG_ASSERT(it == vertex->labels.end(), "Invalid database state!"); + vertex->labels.push_back(current->label); + break; + } + case Delta::Action::SET_PROPERTY: { + vertex->properties.SetProperty(current->property.key, current->property.value); + 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}; + 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}; + 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); + // Increment edge count. We only increment the count here because + // the information in `ADD_IN_EDGE` and `Edge/RECREATE_OBJECT` is + // redundant. Also, `Edge/RECREATE_OBJECT` isn't available when + // edge properties are disabled. + ++shard_->edge_count_; + 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}; + 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()); + vertex->in_edges.pop_back(); + 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}; + 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()); + vertex->out_edges.pop_back(); + // Decrement edge count. We only decrement the count here because + // the information in `REMOVE_IN_EDGE` and `Edge/DELETE_OBJECT` is + // redundant. Also, `Edge/DELETE_OBJECT` isn't available when edge + // properties are disabled. + --shard_->edge_count_; + break; + } + case Delta::Action::DELETE_OBJECT: { + vertex->deleted = true; + InsertVertexPKIntoList(shard_->deleted_vertices_, vertex->keys.Keys()); + break; + } + case Delta::Action::RECREATE_OBJECT: { + vertex->deleted = false; + break; + } + } + current = current->next.load(std::memory_order_acquire); + } + vertex->delta = current; + if (current != nullptr) { + current->prev.Set(vertex); + } + + break; + } + case PreviousPtr::Type::EDGE: { + auto *edge = prev.edge; + Delta *current = edge->delta; + while (current != nullptr && + current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { + switch (current->action) { + case Delta::Action::SET_PROPERTY: { + edge->properties.SetProperty(current->property.key, current->property.value); + break; + } + case Delta::Action::DELETE_OBJECT: { + edge->deleted = true; + shard_->deleted_edges_.push_back(edge->gid); + break; + } + case Delta::Action::RECREATE_OBJECT: { + edge->deleted = false; + break; + } + case Delta::Action::REMOVE_LABEL: + case Delta::Action::ADD_LABEL: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: { + LOG_FATAL("Invalid database state!"); + break; + } + } + current = current->next.load(std::memory_order_acquire); + } + edge->delta = current; + if (current != nullptr) { + current->prev.Set(edge); + } + + break; + } + case PreviousPtr::Type::DELTA: + // pointer probably couldn't be set because allocation failed + case PreviousPtr::Type::NULLPTR: + break; + } + } + + { + uint64_t mark_timestamp = shard_->timestamp_; + + // Release engine lock because we don't have to hold it anymore and + // emplace back could take a long time. + shard_->garbage_undo_buffers_.emplace_back(mark_timestamp, std::move(transaction_.deltas)); + } + + shard_->commit_log_->MarkFinished(transaction_.start_timestamp); + is_transaction_active_ = false; +} + +void Shard::Accessor::FinalizeTransaction() { + if (commit_timestamp_) { + shard_->commit_log_->MarkFinished(*commit_timestamp_); + shard_->committed_transactions_.emplace_back(std::move(transaction_)); + commit_timestamp_.reset(); + } +} + +const std::string &Shard::LabelToName(LabelId label) const { return name_id_mapper_.IdToName(label.AsUint()); } + +const std::string &Shard::PropertyToName(PropertyId property) const { + return name_id_mapper_.IdToName(property.AsUint()); +} + +const std::string &Shard::EdgeTypeToName(EdgeTypeId edge_type) const { + return name_id_mapper_.IdToName(edge_type.AsUint()); +} + +bool Shard::CreateIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) { + // TODO Fix Index + return false; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +bool Shard::CreateIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { + // TODO Fix Index + // if (!indices_.label_property_index.CreateIndex(label, property, labelspace.access())) return false; + return false; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label, {property}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +bool Shard::DropIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) { + if (!indices_.label_index.DropIndex(label)) return false; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +bool Shard::DropIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { + if (!indices_.label_property_index.DropIndex(label, property)) return false; + // For a description why using `timestamp_` is correct, see + // `CreateIndex(LabelId label)`. + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label, {property}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +IndicesInfo Shard::ListAllIndices() const { + return {indices_.label_index.ListIndices(), indices_.label_property_index.ListIndices()}; +} + +utils::BasicResult<ConstraintViolation, bool> Shard::CreateExistenceConstraint( + LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { + // TODO Fix constraints + // auto ret = ::memgraph::storage::v3::CreateExistenceConstraint(&constraints_, label, property, vertices_.access()); + // if (ret.HasError() || !ret.GetValue()) return ret; + return false; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label, {property}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +bool Shard::DropExistenceConstraint(LabelId label, PropertyId property, + const std::optional<uint64_t> desired_commit_timestamp) { + if (!memgraph::storage::v3::DropExistenceConstraint(&constraints_, label, property)) return false; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label, {property}, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return true; +} + +utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Shard::CreateUniqueConstraint( + LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) { + // TODO Fix constraints + // auto ret = constraints_.unique_constraints.CreateConstraint(label, properties, vertices_.access()); + // if (ret.HasError() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) { + // return ret; + // } + return UniqueConstraints::CreationStatus::ALREADY_EXISTS; + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label, properties, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return UniqueConstraints::CreationStatus::SUCCESS; +} + +UniqueConstraints::DeletionStatus Shard::DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties, + const std::optional<uint64_t> desired_commit_timestamp) { + auto ret = constraints_.unique_constraints.DropConstraint(label, properties); + if (ret != UniqueConstraints::DeletionStatus::SUCCESS) { + return ret; + } + const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); + AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label, properties, commit_timestamp); + commit_log_->MarkFinished(commit_timestamp); + last_commit_timestamp_ = commit_timestamp; + return UniqueConstraints::DeletionStatus::SUCCESS; +} + +const SchemaValidator &Shard::Accessor::GetSchemaValidator() const { return shard_->schema_validator_; } + +ConstraintsInfo Shard::ListAllConstraints() const { + return {ListExistenceConstraints(constraints_), constraints_.unique_constraints.ListConstraints()}; +} + +SchemasInfo Shard::ListAllSchemas() const { return {schemas_.ListSchemas()}; } + +const Schemas::Schema *Shard::GetSchema(const LabelId primary_label) const { return schemas_.GetSchema(primary_label); } + +bool Shard::CreateSchema(const LabelId primary_label, const std::vector<SchemaProperty> &schemas_types) { + return schemas_.CreateSchema(primary_label, schemas_types); +} + +bool Shard::DropSchema(const LabelId primary_label) { return schemas_.DropSchema(primary_label); } + +StorageInfo Shard::GetInfo() const { + auto vertex_count = vertices_.size(); + double average_degree = 0.0; + if (vertex_count) { + average_degree = 2.0 * static_cast<double>(edge_count_) / static_cast<double>(vertex_count); + } + return {vertex_count, edge_count_, average_degree, utils::GetMemoryUsage(), + utils::GetDirDiskUsage(config_.durability.storage_directory)}; +} + +VerticesIterable Shard::Accessor::Vertices(LabelId label, View view) { + return VerticesIterable(shard_->indices_.label_index.Vertices(label, view, &transaction_)); +} + +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_->schema_validator_)); +} + +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_->schema_validator_)); +} + +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_->schema_validator_)); +} + +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}; + + transaction_id = transaction_id_++; + // Replica should have only read queries and the write queries + // can come from main instance with any past timestamp. + // To preserve snapshot isolation we set the start timestamp + // of any query on replica to the last commited transaction + // which is timestamp_ as only commit of transaction with writes + // can change the value of it. + if (replication_role_ == ReplicationRole::REPLICA) { + start_timestamp = timestamp_; + } else { + start_timestamp = timestamp_++; + } + + return {transaction_id, start_timestamp, isolation_level}; +} + +// `force` means there are no active transactions, so everything can be deleted without worrying about removing some +// data that is used by an active transaction +template <bool force> +void Shard::CollectGarbage() { + if constexpr (force) { + // TODO(antaljanosbenjamin): figure out whether is there any active transaction or not (probably accessors should + // increment/decrement a counter). If there are no transactions, then garbage collection can be forced + CollectGarbage<false>(); + return; + } + + // Garbage collection must be performed in two phases. In the first phase, + // deltas that won't be applied by any transaction anymore are unlinked from + // the version chains. They cannot be deleted immediately, because there + // might be a transaction that still needs them to terminate the version + // chain traversal. They are instead marked for deletion and will be deleted + // in the second GC phase in this GC iteration or some of the following + // ones. + + uint64_t oldest_active_start_timestamp = commit_log_->OldestActive(); + // We don't move undo buffers of unlinked transactions to garbage_undo_buffers + // list immediately, because we would have to repeatedly take + // garbage_undo_buffers lock. + std::list<std::pair<uint64_t, std::list<Delta>>> unlinked_undo_buffers; + + // We will only free vertices deleted up until now in this GC cycle, and we + // will do it after cleaning-up the indices. That way we are sure that all + // vertices that appear in an index also exist in main storage. + + // Flag that will be used to determine whether the Index GC should be run. It + // should be run when there were any items that were cleaned up (there were + // updates between this run of the GC and the previous run of the GC). This + // eliminates high CPU usage when the GC doesn't have to clean up anything. + bool run_index_cleanup = !committed_transactions_.empty() || !garbage_undo_buffers_.empty(); + + while (true) { + // We don't want to hold the lock on commited transactions for too long, + // because that prevents other transactions from committing. + Transaction *transaction{nullptr}; + { + if (committed_transactions_.empty()) { + break; + } + transaction = &committed_transactions_.front(); + } + + auto commit_timestamp = transaction->commit_timestamp->load(std::memory_order_acquire); + if (commit_timestamp >= oldest_active_start_timestamp) { + break; + } + + // When unlinking a delta which is the first delta in its version chain, + // special care has to be taken to avoid the following race condition: + // + // [Vertex] --> [Delta A] + // + // GC thread: Delta A is the first in its chain, it must be unlinked from + // vertex and marked for deletion + // TX thread: Update vertex and add Delta B with Delta A as next + // + // [Vertex] --> [Delta B] <--> [Delta A] + // + // GC thread: Unlink delta from Vertex + // + // [Vertex] --> (nullptr) + // + // When processing a delta that is the first one in its chain, we + // obtain the corresponding vertex or edge lock, and then verify that this + // delta still is the first in its chain. + // When processing a delta that is in the middle of the chain we only + // process the final delta of the given transaction in that chain. We + // determine the owner of the chain (either a vertex or an edge), obtain the + // corresponding lock, and then verify that this delta is still in the same + // position as it was before taking the lock. + // + // Even though the delta chain is lock-free (both `next` and `prev`) the + // chain should not be modified without taking the lock from the object that + // owns the chain (either a vertex or an edge). Modifying the chain without + // taking the lock will cause subtle race conditions that will leave the + // chain in a broken state. + // The chain can be only read without taking any locks. + + for (Delta &delta : transaction->deltas) { + while (true) { + auto prev = delta.prev.Get(); + switch (prev.type) { + case PreviousPtr::Type::VERTEX: { + Vertex *vertex = prev.vertex; + if (vertex->delta != &delta) { + // Something changed, we're not the first delta in the chain + // anymore. + continue; + } + vertex->delta = nullptr; + if (vertex->deleted) { + InsertVertexPKIntoList(deleted_vertices_, vertex->keys.Keys()); + } + break; + } + case PreviousPtr::Type::EDGE: { + Edge *edge = prev.edge; + if (edge->delta != &delta) { + // Something changed, we're not the first delta in the chain + // anymore. + continue; + } + edge->delta = nullptr; + if (edge->deleted) { + deleted_edges_.push_back(edge->gid); + } + break; + } + case PreviousPtr::Type::DELTA: { + if (prev.delta->timestamp->load(std::memory_order_acquire) == commit_timestamp) { + // The delta that is newer than this one is also a delta from this + // transaction. We skip the current delta and will remove it as a + // part of the suffix later. + break; + } + { + // We need to find the parent object in order to be able to use + // its lock. + auto parent = prev; + while (parent.type == PreviousPtr::Type::DELTA) { + parent = parent.delta->prev.Get(); + } + switch (parent.type) { + case PreviousPtr::Type::VERTEX: + case PreviousPtr::Type::EDGE: + break; + case PreviousPtr::Type::DELTA: + case PreviousPtr::Type::NULLPTR: + LOG_FATAL("Invalid database state!"); + } + } + Delta *prev_delta = prev.delta; + prev_delta->next.store(nullptr, std::memory_order_release); + break; + } + case PreviousPtr::Type::NULLPTR: { + LOG_FATAL("Invalid pointer!"); + } + } + break; + } + } + + unlinked_undo_buffers.emplace_back(0, std::move(transaction->deltas)); + committed_transactions_.pop_front(); + } + + // After unlinking deltas from vertices, we refresh the indices. That way + // we're sure that none of the vertices from `current_deleted_vertices` + // appears in an index, and we can safely remove the from the main storage + // after the last currently active transaction is finished. + if (run_index_cleanup) { + // This operation is very expensive as it traverses through all of the items + // in every index every time. + RemoveObsoleteEntries(&indices_, oldest_active_start_timestamp); + constraints_.unique_constraints.RemoveObsoleteEntries(oldest_active_start_timestamp); + } + + { + uint64_t mark_timestamp = timestamp_; + for (auto &[timestamp, undo_buffer] : unlinked_undo_buffers) { + timestamp = mark_timestamp; + } + garbage_undo_buffers_.splice(garbage_undo_buffers_.end(), unlinked_undo_buffers); + + for (const auto &vertex : deleted_vertices_) { + garbage_vertices_.emplace_back(mark_timestamp, vertex); + } + } + + // if force is set to true we can simply delete all the leftover undos because + // no transaction is active + if constexpr (force) { + garbage_undo_buffers_.clear(); + } else { + while (!garbage_undo_buffers_.empty() && garbage_undo_buffers_.front().first <= oldest_active_start_timestamp) { + garbage_undo_buffers_.pop_front(); + } + } + + { + auto vertex_acc = vertices_.access(); + if constexpr (force) { + // if force is set to true, then we have unique_lock and no transactions are active + // so we can clean all of the deleted vertices + while (!garbage_vertices_.empty()) { + MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!"); + garbage_vertices_.pop_front(); + } + } else { + while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) { + MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!"); + garbage_vertices_.pop_front(); + } + } + } + { + auto edge_acc = edges_.access(); + for (auto edge : deleted_edges_) { + MG_ASSERT(edge_acc.remove(edge), "Invalid database state!"); + } + } +} + +// tell the linker he can find the CollectGarbage definitions here +template void Shard::CollectGarbage<true>(); +template void Shard::CollectGarbage<false>(); + +bool Shard::InitializeWalFile() { + if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL) + return false; + if (!wal_file_) { + wal_file_.emplace(wal_directory_, uuid_, epoch_id_, config_.items, &name_id_mapper_, wal_seq_num_++, + &file_retainer_); + } + return true; +} + +void Shard::FinalizeWalFile() { + ++wal_unsynced_transactions_; + if (wal_unsynced_transactions_ >= config_.durability.wal_file_flush_every_n_tx) { + wal_file_->Sync(); + wal_unsynced_transactions_ = 0; + } + if (wal_file_->GetSize() / 1024 >= config_.durability.wal_file_size_kibibytes) { + wal_file_->FinalizeWal(); + wal_file_ = std::nullopt; + wal_unsynced_transactions_ = 0; + } else { + // Try writing the internal buffer if possible, if not + // the data should be written as soon as it's possible + // (triggered by the new transaction commit, or some + // reading thread EnabledFlushing) + wal_file_->TryFlushing(); + } +} + +void Shard::AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp) { + if (!InitializeWalFile()) return; + // Traverse deltas and append them to the WAL file. + // A single transaction will always be contained in a single WAL file. + auto current_commit_timestamp = transaction.commit_timestamp->load(std::memory_order_acquire); + + if (replication_role_ == ReplicationRole::MAIN) { + replication_clients_.WithLock([&](auto &clients) { + for (auto &client : clients) { + client->StartTransactionReplication(wal_file_->SequenceNumber()); + } + }); + } + + // Helper lambda that traverses the delta chain on order to find the first + // delta that should be processed and then appends all discovered deltas. + auto find_and_apply_deltas = [&](const auto *delta, const auto &parent, auto filter) { + while (true) { + auto *older = delta->next.load(std::memory_order_acquire); + if (older == nullptr || older->timestamp->load(std::memory_order_acquire) != current_commit_timestamp) break; + delta = older; + } + while (true) { + if (filter(delta->action)) { + wal_file_->AppendDelta(*delta, parent, final_commit_timestamp); + replication_clients_.WithLock([&](auto &clients) { + for (auto &client : clients) { + client->IfStreamingTransaction( + [&](auto &stream) { stream.AppendDelta(*delta, parent, final_commit_timestamp); }); + } + }); + } + auto prev = delta->prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::DELTA) break; + delta = prev.delta; + } + }; + + // The deltas are ordered correctly in the `transaction.deltas` buffer, but we + // don't traverse them in that order. That is because for each delta we need + // information about the vertex or edge they belong to and that information + // isn't stored in the deltas themselves. In order to find out information + // about the corresponding vertex or edge it is necessary to traverse the + // delta chain for each delta until a vertex or edge is encountered. This + // operation is very expensive as the chain grows. + // Instead, we traverse the edges until we find a vertex or edge and traverse + // their delta chains. This approach has a drawback because we lose the + // correct order of the operations. Because of that, we need to traverse the + // deltas several times and we have to manually ensure that the stored deltas + // will be ordered correctly. + + // 1. Process all Vertex deltas and store all operations that create vertices + // and modify vertex data. + for (const auto &delta : transaction.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) continue; + find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { + switch (action) { + case Delta::Action::DELETE_OBJECT: + case Delta::Action::SET_PROPERTY: + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + return true; + + case Delta::Action::RECREATE_OBJECT: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + return false; + } + }); + } + // 2. Process all Vertex deltas and store all operations that create edges. + for (const auto &delta : transaction.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) continue; + find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { + switch (action) { + case Delta::Action::REMOVE_OUT_EDGE: + return true; + + case Delta::Action::DELETE_OBJECT: + case Delta::Action::RECREATE_OBJECT: + case Delta::Action::SET_PROPERTY: + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + return false; + } + }); + } + // 3. Process all Edge deltas and store all operations that modify edge data. + for (const auto &delta : transaction.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::EDGE) continue; + find_and_apply_deltas(&delta, *prev.edge, [](auto action) { + switch (action) { + case Delta::Action::SET_PROPERTY: + return true; + + case Delta::Action::DELETE_OBJECT: + case Delta::Action::RECREATE_OBJECT: + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + return false; + } + }); + } + // 4. Process all Vertex deltas and store all operations that delete edges. + for (const auto &delta : transaction.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) continue; + find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { + switch (action) { + case Delta::Action::ADD_OUT_EDGE: + return true; + + case Delta::Action::DELETE_OBJECT: + case Delta::Action::RECREATE_OBJECT: + case Delta::Action::SET_PROPERTY: + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + return false; + } + }); + } + // 5. Process all Vertex deltas and store all operations that delete vertices. + for (const auto &delta : transaction.deltas) { + auto prev = delta.prev.Get(); + MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); + if (prev.type != PreviousPtr::Type::VERTEX) continue; + find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { + switch (action) { + case Delta::Action::RECREATE_OBJECT: + return true; + + case Delta::Action::DELETE_OBJECT: + case Delta::Action::SET_PROPERTY: + case Delta::Action::ADD_LABEL: + case Delta::Action::REMOVE_LABEL: + case Delta::Action::ADD_IN_EDGE: + case Delta::Action::ADD_OUT_EDGE: + case Delta::Action::REMOVE_IN_EDGE: + case Delta::Action::REMOVE_OUT_EDGE: + return false; + } + }); + } + + // Add a delta that indicates that the transaction is fully written to the WAL + // file. + wal_file_->AppendTransactionEnd(final_commit_timestamp); + + FinalizeWalFile(); + + replication_clients_.WithLock([&](auto &clients) { + for (auto &client : clients) { + client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(final_commit_timestamp); }); + client->FinalizeTransactionReplication(); + } + }); +} + +void Shard::AppendToWal(durability::StorageGlobalOperation operation, LabelId label, + const std::set<PropertyId> &properties, uint64_t final_commit_timestamp) { + if (!InitializeWalFile()) return; + wal_file_->AppendOperation(operation, label, properties, final_commit_timestamp); + { + if (replication_role_ == ReplicationRole::MAIN) { + replication_clients_.WithLock([&](auto &clients) { + for (auto &client : clients) { + client->StartTransactionReplication(wal_file_->SequenceNumber()); + client->IfStreamingTransaction( + [&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); }); + client->FinalizeTransactionReplication(); + } + }); + } + } + FinalizeWalFile(); +} + +utils::BasicResult<Shard::CreateSnapshotError> Shard::CreateSnapshot() { + if (replication_role_ != ReplicationRole::MAIN) { + return CreateSnapshotError::DisabledForReplica; + } + + // Create the transaction used to create the snapshot. + auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION); + + // Create snapshot. + // durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_, + // config_.durability.snapshot_retention_count, &vertices_, &edges_, + // &name_id_mapper_, &indices_, &constraints_, config_.items, schema_validator_, + // uuid_, epoch_id_, epoch_history_, &file_retainer_); + + // Finalize snapshot transaction. + commit_log_->MarkFinished(transaction.start_timestamp); + return {}; +} + +bool Shard::LockPath() { + auto locker_accessor = global_locker_.Access(); + return locker_accessor.AddPath(config_.durability.storage_directory); +} + +bool Shard::UnlockPath() { + { + auto locker_accessor = global_locker_.Access(); + if (!locker_accessor.RemovePath(config_.durability.storage_directory)) { + return false; + } + } + + // We use locker accessor in seperate scope so we don't produce deadlock + // after we call clean queue. + file_retainer_.CleanQueue(); + return true; +} + +void Shard::FreeMemory() { + CollectGarbage<true>(); + + // SkipList is already threadsafe + vertices_.run_gc(); + edges_.run_gc(); + indices_.label_index.RunGC(); + indices_.label_property_index.RunGC(); +} + +uint64_t Shard::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) { + if (!desired_commit_timestamp) { + return timestamp_++; + } + timestamp_ = std::max(timestamp_, *desired_commit_timestamp + 1); + return *desired_commit_timestamp; +} + +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) { + return false; + } + + replication_server_ = std::make_unique<ReplicationServer>(this, std::move(endpoint), config); + + replication_role_ = ReplicationRole::REPLICA; + return true; +} + +bool Shard::SetMainReplicationRole() { + // We don't want to generate new epoch_id and do the + // cleanup if we're already a MAIN + if (replication_role_ == ReplicationRole::MAIN) { + return false; + } + + // Main instance does not need replication server + // This should be always called first so we finalize everything + replication_server_.reset(nullptr); + + if (wal_file_) { + wal_file_->FinalizeWal(); + wal_file_.reset(); + } + + // Generate new epoch id and save the last one to the history. + if (epoch_history_.size() == kEpochHistoryRetention) { + epoch_history_.pop_front(); + } + epoch_history_.emplace_back(std::move(epoch_id_), last_commit_timestamp_); + epoch_id_ = utils::GenerateUUID(); + + replication_role_ = ReplicationRole::MAIN; + return true; +} + +utils::BasicResult<Shard::RegisterReplicaError> Shard::RegisterReplica( + std::string name, io::network::Endpoint endpoint, const replication::ReplicationMode replication_mode, + const replication::ReplicationClientConfig &config) { + MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can register a replica!"); + + const bool name_exists = replication_clients_.WithLock([&](auto &clients) { + return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; }); + }); + + if (name_exists) { + return RegisterReplicaError::NAME_EXISTS; + } + + const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) { + return std::any_of(clients.begin(), clients.end(), + [&endpoint](const auto &client) { return client->Endpoint() == endpoint; }); + }); + + if (end_point_exists) { + return RegisterReplicaError::END_POINT_EXISTS; + } + + MG_ASSERT(replication_mode == replication::ReplicationMode::SYNC || !config.timeout, + "Only SYNC mode can have a timeout set"); + + auto client = std::make_unique<ReplicationClient>(std::move(name), this, endpoint, replication_mode, config); + if (client->State() == replication::ReplicaState::INVALID) { + return RegisterReplicaError::CONNECTION_FAILED; + } + + return replication_clients_.WithLock([&](auto &clients) -> utils::BasicResult<Shard::RegisterReplicaError> { + // Another thread could have added a client with same name while + // we were connecting to this client. + if (std::any_of(clients.begin(), clients.end(), + [&](const auto &other_client) { return client->Name() == other_client->Name(); })) { + return RegisterReplicaError::NAME_EXISTS; + } + + if (std::any_of(clients.begin(), clients.end(), + [&client](const auto &other_client) { return client->Endpoint() == other_client->Endpoint(); })) { + return RegisterReplicaError::END_POINT_EXISTS; + } + + clients.push_back(std::move(client)); + return {}; + }); +} + +bool Shard::UnregisterReplica(const std::string_view name) { + MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can unregister a replica!"); + return replication_clients_.WithLock([&](auto &clients) { + return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; }); + }); +} + +std::optional<replication::ReplicaState> Shard::GetReplicaState(const std::string_view name) { + return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> { + const auto client_it = + std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; }); + if (client_it == clients.cend()) { + return std::nullopt; + } + return (*client_it)->State(); + }); +} + +ReplicationRole Shard::GetReplicationRole() const { return replication_role_; } + +std::vector<Shard::ReplicaInfo> Shard::ReplicasInfo() { + return replication_clients_.WithLock([](auto &clients) { + std::vector<Shard::ReplicaInfo> replica_info; + replica_info.reserve(clients.size()); + std::transform(clients.begin(), clients.end(), std::back_inserter(replica_info), + [](const auto &client) -> ReplicaInfo { + return {client->Name(), client->Mode(), client->Timeout(), client->Endpoint(), client->State()}; + }); + return replica_info; + }); +} + +void Shard::SetIsolationLevel(IsolationLevel isolation_level) { isolation_level_ = isolation_level; } + +} // namespace memgraph::storage::v3 diff --git a/src/storage/v3/shard.hpp b/src/storage/v3/shard.hpp new file mode 100644 index 000000000..8a42e542d --- /dev/null +++ b/src/storage/v3/shard.hpp @@ -0,0 +1,617 @@ +// 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 <atomic> +#include <cstdint> +#include <filesystem> +#include <map> +#include <numeric> +#include <optional> +#include <shared_mutex> +#include <variant> +#include <vector> + +#include "io/network/endpoint.hpp" +#include "kvstore/kvstore.hpp" +#include "storage/v3/commit_log.hpp" +#include "storage/v3/config.hpp" +#include "storage/v3/constraints.hpp" +#include "storage/v3/durability/metadata.hpp" +#include "storage/v3/durability/wal.hpp" +#include "storage/v3/edge.hpp" +#include "storage/v3/edge_accessor.hpp" +#include "storage/v3/id_types.hpp" +#include "storage/v3/indices.hpp" +#include "storage/v3/isolation_level.hpp" +#include "storage/v3/key_store.hpp" +#include "storage/v3/lexicographically_ordered_vertex.hpp" +#include "storage/v3/mvcc.hpp" +#include "storage/v3/name_id_mapper.hpp" +#include "storage/v3/property_value.hpp" +#include "storage/v3/result.hpp" +#include "storage/v3/schema_validator.hpp" +#include "storage/v3/schemas.hpp" +#include "storage/v3/transaction.hpp" +#include "storage/v3/vertex.hpp" +#include "storage/v3/vertex_accessor.hpp" +#include "storage/v3/vertices_skip_list.hpp" +#include "utils/exceptions.hpp" +#include "utils/file_locker.hpp" +#include "utils/on_scope_exit.hpp" +#include "utils/rw_lock.hpp" +#include "utils/scheduler.hpp" +#include "utils/skip_list.hpp" +#include "utils/synchronized.hpp" +#include "utils/uuid.hpp" + +/// REPLICATION /// +#include "rpc/server.hpp" +#include "storage/v3/replication/config.hpp" +#include "storage/v3/replication/enums.hpp" +#include "storage/v3/replication/rpc.hpp" +#include "storage/v3/replication/serialization.hpp" + +namespace memgraph::storage::v3 { + +// 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. + +/// Iterable for iterating through all vertices of a Storage. +/// +/// An instance of this will be usually be wrapped inside VerticesIterable for +/// generic, public use. +class AllVerticesIterable final { + VerticesSkipList::Accessor vertices_accessor_; + Transaction *transaction_; + View view_; + Indices *indices_; + Constraints *constraints_; + Config::Items config_; + const SchemaValidator *schema_validator_; + const Schemas *schemas_; + std::optional<VertexAccessor> vertex_; + + public: + class Iterator final { + AllVerticesIterable *self_; + VerticesSkipList::Iterator it_; + + public: + Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it); + + VertexAccessor operator*() const; + + Iterator &operator++(); + + bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; } + + bool operator!=(const Iterator &other) const { return !(*this == other); } + }; + + AllVerticesIterable(VerticesSkipList::Accessor vertices_accessor, Transaction *transaction, View view, + Indices *indices, Constraints *constraints, Config::Items config, + const SchemaValidator &schema_validator) + : vertices_accessor_(std::move(vertices_accessor)), + transaction_(transaction), + view_(view), + indices_(indices), + constraints_(constraints), + config_(config), + schema_validator_{&schema_validator} {} + + Iterator begin() { return {this, vertices_accessor_.begin()}; } + Iterator end() { return {this, vertices_accessor_.end()}; } +}; + +/// Generic access to different kinds of vertex iterations. +/// +/// This class should be the primary type used by the client code to iterate +/// over vertices inside a Storage instance. +class VerticesIterable final { + enum class Type { ALL, BY_LABEL, BY_LABEL_PROPERTY }; + + Type type_; + union { + AllVerticesIterable all_vertices_; + LabelIndex::Iterable vertices_by_label_; + LabelPropertyIndex::Iterable vertices_by_label_property_; + }; + + public: + explicit VerticesIterable(AllVerticesIterable); + explicit VerticesIterable(LabelIndex::Iterable); + explicit VerticesIterable(LabelPropertyIndex::Iterable); + + VerticesIterable(const VerticesIterable &) = delete; + VerticesIterable &operator=(const VerticesIterable &) = delete; + + VerticesIterable(VerticesIterable &&) noexcept; + VerticesIterable &operator=(VerticesIterable &&) noexcept; + + ~VerticesIterable(); + + class Iterator final { + Type type_; + union { + AllVerticesIterable::Iterator all_it_; + LabelIndex::Iterable::Iterator by_label_it_; + LabelPropertyIndex::Iterable::Iterator by_label_property_it_; + }; + + void Destroy() noexcept; + + public: + explicit Iterator(AllVerticesIterable::Iterator); + explicit Iterator(LabelIndex::Iterable::Iterator); + explicit Iterator(LabelPropertyIndex::Iterable::Iterator); + + Iterator(const Iterator &); + Iterator &operator=(const Iterator &); + + Iterator(Iterator &&) noexcept; + Iterator &operator=(Iterator &&) noexcept; + + ~Iterator(); + + VertexAccessor operator*() const; + + Iterator &operator++(); + + bool operator==(const Iterator &other) const; + bool operator!=(const Iterator &other) const { return !(*this == other); } + }; + + Iterator begin(); + Iterator end(); +}; + +/// Structure used to return information about existing indices in the storage. +struct IndicesInfo { + std::vector<LabelId> label; + std::vector<std::pair<LabelId, PropertyId>> label_property; +}; + +/// Structure used to return information about existing constraints in the +/// storage. +struct ConstraintsInfo { + std::vector<std::pair<LabelId, PropertyId>> existence; + std::vector<std::pair<LabelId, std::set<PropertyId>>> unique; +}; + +/// Structure used to return information about existing schemas in the storage +struct SchemasInfo { + Schemas::SchemasList schemas; +}; + +/// 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 }; + +class Shard final { + public: + /// @throw std::system_error + /// @throw std::bad_alloc + explicit Shard(LabelId primary_label, PrimaryKey min_primary_key, std::optional<PrimaryKey> max_primary_key, + Config config = Config()); + + Shard(const Shard &) = delete; + Shard(Shard &&) noexcept = delete; + Shard &operator=(const Shard &) = delete; + Shard operator=(Shard &&) noexcept = delete; + ~Shard(); + + class Accessor final { + private: + friend class Shard; + + explicit Accessor(Shard *shard, IsolationLevel isolation_level); + + public: + Accessor(const Accessor &) = delete; + Accessor &operator=(const Accessor &) = delete; + Accessor &operator=(Accessor &&other) = delete; + + // NOTE: After the accessor is moved, all objects derived from it (accessors + // and iterators) are *invalid*. You have to get all derived objects again. + Accessor(Accessor &&other) noexcept; + + ~Accessor(); + + /// @throw std::bad_alloc + ResultSchema<VertexAccessor> CreateVertexAndValidate( + LabelId primary_label, const std::vector<LabelId> &labels, + const std::vector<std::pair<PropertyId, PropertyValue>> &properties); + + std::optional<VertexAccessor> FindVertex(std::vector<PropertyValue> primary_key, View view); + + VerticesIterable Vertices(View view) { + return VerticesIterable(AllVerticesIterable(shard_->vertices_.access(), &transaction_, view, &shard_->indices_, + &shard_->constraints_, shard_->config_.items, + shard_->schema_validator_)); + } + + VerticesIterable Vertices(LabelId label, View view); + + VerticesIterable Vertices(LabelId label, PropertyId property, View view); + + VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view); + + VerticesIterable Vertices(LabelId label, PropertyId property, + const std::optional<utils::Bound<PropertyValue>> &lower_bound, + const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view); + + /// Return approximate number of all vertices in the database. + /// Note that this is always an over-estimate and never an under-estimate. + int64_t ApproximateVertexCount() const { return static_cast<int64_t>(shard_->vertices_.size()); } + + /// Return approximate number of vertices with the given label. + /// Note that this is always an over-estimate and never an under-estimate. + int64_t ApproximateVertexCount(LabelId label) const { + return shard_->indices_.label_index.ApproximateVertexCount(label); + } + + /// Return approximate number of vertices with the given label and property. + /// Note that this is always an over-estimate and never an under-estimate. + int64_t ApproximateVertexCount(LabelId label, PropertyId property) const { + return shard_->indices_.label_property_index.ApproximateVertexCount(label, property); + } + + /// Return approximate number of vertices with the given label and the given + /// value for the given property. Note that this is always an over-estimate + /// and never an under-estimate. + int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const { + return shard_->indices_.label_property_index.ApproximateVertexCount(label, property, value); + } + + /// Return approximate number of vertices with the given label and value for + /// the given property in the range defined by provided upper and lower + /// bounds. + int64_t ApproximateVertexCount(LabelId label, PropertyId property, + const std::optional<utils::Bound<PropertyValue>> &lower, + const std::optional<utils::Bound<PropertyValue>> &upper) const { + return shard_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper); + } + + /// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise + /// @throw std::bad_alloc + Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex); + + /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise + /// @throw std::bad_alloc + Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex( + VertexAccessor *vertex); + + /// @throw std::bad_alloc + Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type); + + /// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise + /// @throw std::bad_alloc + Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge); + + const std::string &LabelToName(LabelId label) const; + const std::string &PropertyToName(PropertyId property) const; + const std::string &EdgeTypeToName(EdgeTypeId edge_type) const; + + bool LabelIndexExists(LabelId label) const { return shard_->indices_.label_index.IndexExists(label); } + + bool LabelPropertyIndexExists(LabelId label, PropertyId property) const { + return shard_->indices_.label_property_index.IndexExists(label, property); + } + + IndicesInfo ListAllIndices() const { + return {shard_->indices_.label_index.ListIndices(), shard_->indices_.label_property_index.ListIndices()}; + } + + ConstraintsInfo ListAllConstraints() const { + return {ListExistenceConstraints(shard_->constraints_), + shard_->constraints_.unique_constraints.ListConstraints()}; + } + + const SchemaValidator &GetSchemaValidator() const; + + SchemasInfo ListAllSchemas() const { return {shard_->schemas_.ListSchemas()}; } + + void AdvanceCommand(); + + /// Commit returns `ConstraintViolation` if the changes made by this + /// transaction violate an existence or unique constraint. In that case the + /// transaction is automatically aborted. Otherwise, void is returned. + /// @throw std::bad_alloc + utils::BasicResult<ConstraintViolation, void> Commit(std::optional<uint64_t> desired_commit_timestamp = {}); + + /// @throw std::bad_alloc + void Abort(); + + void FinalizeTransaction(); + + private: + /// @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_; + bool is_transaction_active_; + Config::Items config_; + }; + + Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) { + return Accessor{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; + + /// @throw std::bad_alloc if unable to insert a new mapping + LabelId NameToLabel(std::string_view name); + + /// @throw std::bad_alloc if unable to insert a new mapping + PropertyId NameToProperty(std::string_view name); + + /// @throw std::bad_alloc if unable to insert a new mapping + EdgeTypeId NameToEdgeType(std::string_view name); + + /// @throw std::bad_alloc + bool CreateIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {}); + + /// @throw std::bad_alloc + bool CreateIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); + + bool DropIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {}); + + bool DropIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); + + IndicesInfo ListAllIndices() const; + + /// Creates an existence constraint. Returns true if the constraint was + /// successfully added, false if it already exists and a `ConstraintViolation` + /// if there is an existing vertex violating the constraint. + /// + /// @throw std::bad_alloc + /// @throw std::length_error + utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint( + LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); + + /// Removes an existence constraint. Returns true if the constraint was + /// removed, and false if it doesn't exist. + bool DropExistenceConstraint(LabelId label, PropertyId property, + std::optional<uint64_t> desired_commit_timestamp = {}); + + /// Creates a unique constraint. In the case of two vertices violating the + /// constraint, it returns `ConstraintViolation`. Otherwise returns a + /// `UniqueConstraints::CreationStatus` enum with the following possibilities: + /// * `SUCCESS` if the constraint was successfully created, + /// * `ALREADY_EXISTS` if the constraint already existed, + /// * `EMPTY_PROPERTIES` if the property set is empty, or + // * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the + // limit of maximum number of properties. + /// + /// @throw std::bad_alloc + utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> CreateUniqueConstraint( + LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {}); + + /// Removes a unique constraint. Returns `UniqueConstraints::DeletionStatus` + /// enum with the following possibilities: + /// * `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. + UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties, + std::optional<uint64_t> desired_commit_timestamp = {}); + + ConstraintsInfo ListAllConstraints() const; + + SchemasInfo ListAllSchemas() const; + + const Schemas::Schema *GetSchema(LabelId primary_label) const; + + bool CreateSchema(LabelId primary_label, const std::vector<SchemaProperty> &schemas_types); + + bool DropSchema(LabelId primary_label); + + StorageInfo GetInfo() const; + + bool LockPath(); + bool UnlockPath(); + + bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {}); + + bool SetMainReplicationRole(); + + 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 + utils::BasicResult<RegisterReplicaError, void> RegisterReplica( + std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode, + const replication::ReplicationClientConfig &config = {}); + /// @pre The instance should have a MAIN role + bool UnregisterReplica(std::string_view name); + + std::optional<replication::ReplicaState> GetReplicaState(std::string_view name); + + ReplicationRole GetReplicationRole() const; + + struct ReplicaInfo { + std::string name; + replication::ReplicationMode mode; + std::optional<double> timeout; + io::network::Endpoint endpoint; + replication::ReplicaState state; + }; + + std::vector<ReplicaInfo> ReplicasInfo(); + + void FreeMemory(); + + void SetIsolationLevel(IsolationLevel isolation_level); + + enum class CreateSnapshotError : uint8_t { DisabledForReplica }; + + utils::BasicResult<CreateSnapshotError> CreateSnapshot(); + + private: + Transaction CreateTransaction(IsolationLevel isolation_level); + + /// The force parameter determines the behaviour of the garbage collector. + /// If it's set to true, it will behave as a global operation, i.e. it can't + /// be part of a transaction, and no other transaction can be active at the same time. + /// This allows it to delete immediately vertices without worrying that some other + /// transaction is possibly using it. If there are active transactions when this method + /// is called with force set to true, it will fallback to the same method with the force + /// set to false. + /// If it's set to false, it will execute in parallel with other transactions, ensuring + /// that no object in use can be deleted. + /// @throw std::system_error + /// @throw std::bad_alloc + template <bool force> + void CollectGarbage(); + + bool InitializeWalFile(); + void FinalizeWalFile(); + + void AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp); + void AppendToWal(durability::StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties, + uint64_t final_commit_timestamp); + + uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {}); + + // Main object storage + NameIdMapper name_id_mapper_; + LabelId primary_label_; + 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. + uint64_t edge_count_{0}; + + SchemaValidator schema_validator_; + Constraints constraints_; + Indices indices_; + Schemas schemas_; + + // Transaction engine + uint64_t timestamp_{kTimestampInitialId}; + uint64_t transaction_id_{kTransactionInitialId}; + // TODO: This isn't really a commit log, it doesn't even care if a + // transaction commited or aborted. We could probably combine this with + // `timestamp_` in a sensible unit, something like TransactionClock or + // whatever. + std::optional<CommitLog> commit_log_; + + std::list<Transaction> committed_transactions_; + IsolationLevel isolation_level_; + + Config config_; + + // Undo buffers that were unlinked and now are waiting to be freed. + std::list<std::pair<uint64_t, std::list<Delta>>> garbage_undo_buffers_; + + // Vertices that are logically deleted but still have to be removed from + // indices before removing them from the main storage. + std::list<PrimaryKey> deleted_vertices_; + + // Vertices that are logically deleted and removed from indices and now wait + // to be removed from the main storage. + std::list<std::pair<uint64_t, PrimaryKey>> garbage_vertices_; + + // Edges that are logically deleted and wait to be removed from the main + // storage. + std::list<Gid> deleted_edges_; + + // Durability + std::filesystem::path snapshot_directory_; + std::filesystem::path wal_directory_; + std::filesystem::path lock_file_path_; + utils::OutputFile lock_file_handle_; + + // UUID used to distinguish snapshots and to link snapshots to WALs + std::string uuid_; + // Sequence number used to keep track of the chain of WALs. + uint64_t wal_seq_num_{0}; + + // UUID to distinguish different main instance runs for replication process + // on SAME storage. + // Multiple instances can have same storage UUID and be MAIN at the same time. + // We cannot compare commit timestamps of those instances if one of them + // becomes the replica of the other so we use epoch_id_ as additional + // discriminating property. + // Example of this: + // We have 2 instances of the same storage, S1 and S2. + // S1 and S2 are MAIN and accept their own commits and write them to the WAL. + // At the moment when S1 commited a transaction with timestamp 20, and S2 + // a different transaction with timestamp 15, we change S2's role to REPLICA + // and register it on S1. + // Without using the epoch_id, we don't know that S1 and S2 have completely + // different transactions, we think that the S2 is behind only by 5 commits. + std::string epoch_id_; + // History of the previous epoch ids. + // Each value consists of the epoch id along the last commit belonging to that + // epoch. + std::deque<std::pair<std::string, uint64_t>> epoch_history_; + + std::optional<durability::WalFile> wal_file_; + uint64_t wal_unsynced_transactions_{0}; + + utils::FileRetainer file_retainer_; + + // Global locker that is used for clients file locking + utils::FileRetainer::FileLocker global_locker_; + + // Last commited timestamp + uint64_t last_commit_timestamp_{kTimestampInitialId}; + + class ReplicationServer; + std::unique_ptr<ReplicationServer> replication_server_{nullptr}; + + class ReplicationClient; + // We create ReplicationClient using unique_ptr so we can move + // newly created client into the vector. + // We cannot move the client directly because it contains ThreadPool + // which cannot be moved. Also, the move is necessary because + // we don't want to create the client directly inside the vector + // because that would require the lock on the list putting all + // commits (they iterate list of clients) to halt. + // This way we can initialize client in main thread which means + // that we can immediately notify the user if the initialization + // failed. + using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>; + ReplicationClientList replication_clients_; + + ReplicationRole replication_role_{ReplicationRole::MAIN}; +}; + +} // namespace memgraph::storage::v3 diff --git a/src/storage/v3/storage.cpp b/src/storage/v3/storage.cpp index 23dd45427..28fd8b330 100644 --- a/src/storage/v3/storage.cpp +++ b/src/storage/v3/storage.cpp @@ -11,1853 +11,10 @@ #include "storage/v3/storage.hpp" -#include <algorithm> -#include <atomic> -#include <cstdint> -#include <iterator> -#include <memory> -#include <mutex> -#include <optional> -#include <variant> - -#include <bits/ranges_algo.h> -#include <gflags/gflags.h> -#include <spdlog/spdlog.h> - -#include "io/network/endpoint.hpp" -#include "storage/v3/constraints.hpp" -#include "storage/v3/durability/durability.hpp" -#include "storage/v3/durability/metadata.hpp" -#include "storage/v3/durability/paths.hpp" -#include "storage/v3/durability/snapshot.hpp" -#include "storage/v3/durability/wal.hpp" -#include "storage/v3/edge_accessor.hpp" -#include "storage/v3/id_types.hpp" -#include "storage/v3/indices.hpp" -#include "storage/v3/key_store.hpp" -#include "storage/v3/mvcc.hpp" -#include "storage/v3/property_value.hpp" -#include "storage/v3/replication/config.hpp" -#include "storage/v3/replication/replication_client.hpp" -#include "storage/v3/replication/replication_server.hpp" -#include "storage/v3/replication/rpc.hpp" -#include "storage/v3/schema_validator.hpp" -#include "storage/v3/transaction.hpp" -#include "storage/v3/vertex.hpp" -#include "storage/v3/vertex_accessor.hpp" -#include "storage/v3/vertices_skip_list.hpp" -#include "utils/exceptions.hpp" -#include "utils/file.hpp" -#include "utils/logging.hpp" -#include "utils/memory_tracker.hpp" -#include "utils/message.hpp" -#include "utils/result.hpp" -#include "utils/rw_lock.hpp" -#include "utils/skip_list.hpp" -#include "utils/spin_lock.hpp" -#include "utils/stat.hpp" -#include "utils/uuid.hpp" +#include "storage/v3/config.hpp" namespace memgraph::storage::v3 { -using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler; - -namespace { -inline constexpr uint16_t kEpochHistoryRetention = 1000; - -void InsertVertexPKIntoList(auto &container, const PrimaryKey &primary_key) { container.push_back(primary_key); } -} // namespace - -auto AdvanceToVisibleVertex(VerticesSkipList::Iterator it, VerticesSkipList::Iterator end, - std::optional<VertexAccessor> *vertex, Transaction *tx, View view, Indices *indices, - Constraints *constraints, Config::Items config, const SchemaValidator &schema_validator) { - while (it != end) { - *vertex = VertexAccessor::Create(&it->vertex, tx, indices, constraints, config, schema_validator, view); - if (!*vertex) { - ++it; - continue; - } - break; - } - return it; -} - -AllVerticesIterable::Iterator::Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it) - : self_(self), - it_(AdvanceToVisibleVertex(it, self->vertices_accessor_.end(), &self->vertex_, self->transaction_, self->view_, - self->indices_, self_->constraints_, self->config_, *self_->schema_validator_)) {} - -VertexAccessor AllVerticesIterable::Iterator::operator*() const { return *self_->vertex_; } - -AllVerticesIterable::Iterator &AllVerticesIterable::Iterator::operator++() { - ++it_; - it_ = AdvanceToVisibleVertex(it_, self_->vertices_accessor_.end(), &self_->vertex_, self_->transaction_, self_->view_, - self_->indices_, self_->constraints_, self_->config_, *self_->schema_validator_); - return *this; -} - -VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) { - new (&all_vertices_) AllVerticesIterable(std::move(vertices)); -} - -VerticesIterable::VerticesIterable(LabelIndex::Iterable vertices) : type_(Type::BY_LABEL) { - new (&vertices_by_label_) LabelIndex::Iterable(std::move(vertices)); -} - -VerticesIterable::VerticesIterable(LabelPropertyIndex::Iterable vertices) : type_(Type::BY_LABEL_PROPERTY) { - new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(vertices)); -} - -VerticesIterable::VerticesIterable(VerticesIterable &&other) noexcept : type_(other.type_) { - switch (other.type_) { - case Type::ALL: - new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_)); - break; - case Type::BY_LABEL: - new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_)); - break; - case Type::BY_LABEL_PROPERTY: - new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_)); - break; - } -} - -VerticesIterable &VerticesIterable::operator=(VerticesIterable &&other) noexcept { - switch (type_) { - case Type::ALL: - all_vertices_.AllVerticesIterable::~AllVerticesIterable(); - break; - case Type::BY_LABEL: - vertices_by_label_.LabelIndex::Iterable::~Iterable(); - break; - case Type::BY_LABEL_PROPERTY: - vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable(); - break; - } - type_ = other.type_; - switch (other.type_) { - case Type::ALL: - new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_)); - break; - case Type::BY_LABEL: - new (&vertices_by_label_) LabelIndex::Iterable(std::move(other.vertices_by_label_)); - break; - case Type::BY_LABEL_PROPERTY: - new (&vertices_by_label_property_) LabelPropertyIndex::Iterable(std::move(other.vertices_by_label_property_)); - break; - } - return *this; -} - -VerticesIterable::~VerticesIterable() { - switch (type_) { - case Type::ALL: - all_vertices_.AllVerticesIterable::~AllVerticesIterable(); - break; - case Type::BY_LABEL: - vertices_by_label_.LabelIndex::Iterable::~Iterable(); - break; - case Type::BY_LABEL_PROPERTY: - vertices_by_label_property_.LabelPropertyIndex::Iterable::~Iterable(); - break; - } -} - -VerticesIterable::Iterator VerticesIterable::begin() { - switch (type_) { - case Type::ALL: - return Iterator(all_vertices_.begin()); - case Type::BY_LABEL: - return Iterator(vertices_by_label_.begin()); - case Type::BY_LABEL_PROPERTY: - return Iterator(vertices_by_label_property_.begin()); - } -} - -VerticesIterable::Iterator VerticesIterable::end() { - switch (type_) { - case Type::ALL: - return Iterator(all_vertices_.end()); - case Type::BY_LABEL: - return Iterator(vertices_by_label_.end()); - case Type::BY_LABEL_PROPERTY: - return Iterator(vertices_by_label_property_.end()); - } -} - -VerticesIterable::Iterator::Iterator(AllVerticesIterable::Iterator it) : type_(Type::ALL) { - new (&all_it_) AllVerticesIterable::Iterator(it); -} - -VerticesIterable::Iterator::Iterator(LabelIndex::Iterable::Iterator it) : type_(Type::BY_LABEL) { - new (&by_label_it_) LabelIndex::Iterable::Iterator(it); -} - -VerticesIterable::Iterator::Iterator(LabelPropertyIndex::Iterable::Iterator it) : type_(Type::BY_LABEL_PROPERTY) { - new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(it); -} - -VerticesIterable::Iterator::Iterator(const VerticesIterable::Iterator &other) : type_(other.type_) { - switch (other.type_) { - case Type::ALL: - new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); - break; - case Type::BY_LABEL: - new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); - break; - case Type::BY_LABEL_PROPERTY: - new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); - break; - } -} - -VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(const VerticesIterable::Iterator &other) { - if (this == &other) { - return *this; - } - Destroy(); - type_ = other.type_; - switch (other.type_) { - case Type::ALL: - new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); - break; - case Type::BY_LABEL: - new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); - break; - case Type::BY_LABEL_PROPERTY: - new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); - break; - } - return *this; -} - -VerticesIterable::Iterator::Iterator(VerticesIterable::Iterator &&other) noexcept : type_(other.type_) { - switch (other.type_) { - case Type::ALL: - new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); - break; - case Type::BY_LABEL: - new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); - break; - case Type::BY_LABEL_PROPERTY: - new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); - break; - } -} - -VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(VerticesIterable::Iterator &&other) noexcept { - Destroy(); - type_ = other.type_; - switch (other.type_) { - case Type::ALL: - new (&all_it_) AllVerticesIterable::Iterator(other.all_it_); - break; - case Type::BY_LABEL: - new (&by_label_it_) LabelIndex::Iterable::Iterator(other.by_label_it_); - break; - case Type::BY_LABEL_PROPERTY: - new (&by_label_property_it_) LabelPropertyIndex::Iterable::Iterator(other.by_label_property_it_); - break; - } - return *this; -} - -VerticesIterable::Iterator::~Iterator() { Destroy(); } - -void VerticesIterable::Iterator::Destroy() noexcept { - switch (type_) { - case Type::ALL: - all_it_.AllVerticesIterable::Iterator::~Iterator(); - break; - case Type::BY_LABEL: - by_label_it_.LabelIndex::Iterable::Iterator::~Iterator(); - break; - case Type::BY_LABEL_PROPERTY: - by_label_property_it_.LabelPropertyIndex::Iterable::Iterator::~Iterator(); - break; - } -} - -VertexAccessor VerticesIterable::Iterator::operator*() const { - switch (type_) { - case Type::ALL: - return *all_it_; - case Type::BY_LABEL: - return *by_label_it_; - case Type::BY_LABEL_PROPERTY: - return *by_label_property_it_; - } -} - -VerticesIterable::Iterator &VerticesIterable::Iterator::operator++() { - switch (type_) { - case Type::ALL: - ++all_it_; - break; - case Type::BY_LABEL: - ++by_label_it_; - break; - case Type::BY_LABEL_PROPERTY: - ++by_label_property_it_; - break; - } - return *this; -} - -bool VerticesIterable::Iterator::operator==(const Iterator &other) const { - switch (type_) { - case Type::ALL: - return all_it_ == other.all_it_; - case Type::BY_LABEL: - return by_label_it_ == other.by_label_it_; - case Type::BY_LABEL_PROPERTY: - return by_label_property_it_ == other.by_label_property_it_; - } -} - -Storage::Storage(Config config) - : schema_validator_(schemas_), - indices_(&constraints_, config.items, schema_validator_), - isolation_level_(config.transaction.isolation_level), - config_(config), - snapshot_directory_(config_.durability.storage_directory / durability::kSnapshotDirectory), - wal_directory_(config_.durability.storage_directory / durability::kWalDirectory), - lock_file_path_(config_.durability.storage_directory / durability::kLockFile), - uuid_(utils::GenerateUUID()), - epoch_id_(utils::GenerateUUID()), - global_locker_(file_retainer_.AddLocker()) { - if (config_.durability.snapshot_wal_mode == Config::Durability::SnapshotWalMode::DISABLED && - replication_role_ == ReplicationRole::MAIN) { - spdlog::warn( - "The instance has the MAIN replication role, but durability logs and snapshots are disabled. Please consider " - "enabling durability by using --storage-snapshot-interval-sec and --storage-wal-enabled flags because " - "without write-ahead logs this instance is not replicating any data."); - } - if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || - config_.durability.snapshot_on_exit || config_.durability.recover_on_startup) { - // Create the directory initially to crash the database in case of - // permission errors. This is done early to crash the database on startup - // instead of crashing the database for the first time during runtime (which - // could be an unpleasant surprise). - utils::EnsureDirOrDie(snapshot_directory_); - // Same reasoning as above. - utils::EnsureDirOrDie(wal_directory_); - - // Verify that the user that started the process is the same user that is - // the owner of the storage directory. - durability::VerifyStorageDirectoryOwnerAndProcessUserOrDie(config_.durability.storage_directory); - - // Create the lock file and open a handle to it. This will crash the - // database if it can't open the file for writing or if any other process is - // holding the file opened. - lock_file_handle_.Open(lock_file_path_, utils::OutputFile::Mode::OVERWRITE_EXISTING); - MG_ASSERT(lock_file_handle_.AcquireLock(), - "Couldn't acquire lock on the storage directory {}" - "!\nAnother Memgraph process is currently running with the same " - "storage directory, please stop it first before starting this " - "process!", - config_.durability.storage_directory); - } - if (config_.durability.recover_on_startup) { - auto info = std::optional<durability::RecoveryInfo>{}; - - // durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_, &vertices_, - // &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; - } - } - } else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED || - config_.durability.snapshot_on_exit) { - bool files_moved = false; - auto backup_root = config_.durability.storage_directory / durability::kBackupDirectory; - for (const auto &[path, dirname, what] : - {std::make_tuple(snapshot_directory_, durability::kSnapshotDirectory, "snapshot"), - std::make_tuple(wal_directory_, durability::kWalDirectory, "WAL")}) { - if (!utils::DirExists(path)) continue; - auto backup_curr = backup_root / dirname; - std::error_code error_code; - for (const auto &item : std::filesystem::directory_iterator(path, error_code)) { - utils::EnsureDirOrDie(backup_root); - utils::EnsureDirOrDie(backup_curr); - std::error_code item_error_code; - std::filesystem::rename(item.path(), backup_curr / item.path().filename(), item_error_code); - MG_ASSERT(!item_error_code, "Couldn't move {} file {} because of: {}", what, item.path(), - item_error_code.message()); - files_moved = true; - } - MG_ASSERT(!error_code, "Couldn't backup {} files because of: {}", what, error_code.message()); - } - if (files_moved) { - spdlog::warn( - "Since Memgraph was not supposed to recover on startup and " - "durability is enabled, your current durability files will likely " - "be overridden. To prevent important data loss, Memgraph has stored " - "those files into a .backup directory inside the storage directory."); - } - } - if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) { - // TODO(antaljanosbenjamin): handle snapshots - // snapshot_runner_.Run("Snapshot", config_.durability.snapshot_interval, [this] { - // if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) { - // switch (maybe_error.GetError()) { - // case CreateSnapshotError::DisabledForReplica: - // spdlog::warn( - // utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication")); - // break; - // } - // } - // }); - } - - if (timestamp_ == kTimestampInitialId) { - commit_log_.emplace(); - } else { - commit_log_.emplace(timestamp_); - } -} - -Storage::~Storage() { - { - // Clear replication data - replication_server_.reset(); - replication_clients_.WithLock([&](auto &clients) { clients.clear(); }); - } - if (wal_file_) { - wal_file_->FinalizeWal(); - wal_file_ = std::nullopt; - } - if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED) { - // TODO(antaljanosbenjamin): stop snapshot creation - } - if (config_.durability.snapshot_on_exit) { - if (auto maybe_error = this->CreateSnapshot(); maybe_error.HasError()) { - switch (maybe_error.GetError()) { - case CreateSnapshotError::DisabledForReplica: - spdlog::warn(utils::MessageWithLink("Snapshots are disabled for replicas.", "https://memgr.ph/replication")); - break; - } - } - } -} - -Storage::Accessor::Accessor(Storage *storage, IsolationLevel isolation_level) - : storage_(storage), - transaction_(storage->CreateTransaction(isolation_level)), - is_transaction_active_(true), - config_(storage->config_.items) {} - -Storage::Accessor::Accessor(Accessor &&other) noexcept - : storage_(other.storage_), - transaction_(std::move(other.transaction_)), - commit_timestamp_(other.commit_timestamp_), - is_transaction_active_(other.is_transaction_active_), - config_(other.config_) { - // Don't allow the other accessor to abort our transaction in destructor. - other.is_transaction_active_ = false; - other.commit_timestamp_.reset(); -} - -Storage::Accessor::~Accessor() { - if (is_transaction_active_) { - Abort(); - } - - FinalizeTransaction(); -} - -ResultSchema<VertexAccessor> Storage::Accessor::CreateVertexAndValidate( - LabelId primary_label, const std::vector<LabelId> &labels, - const std::vector<std::pair<PropertyId, PropertyValue>> &properties) { - auto maybe_schema_violation = GetSchemaValidator().ValidateVertexCreate(primary_label, labels, properties); - if (maybe_schema_violation) { - return {std::move(*maybe_schema_violation)}; - } - OOMExceptionEnabler oom_exception; - // Extract key properties - std::vector<PropertyValue> primary_properties; - for ([[maybe_unused]] const auto &[property_id, property_type] : storage_->GetSchema(primary_label)->second) { - // We know there definitely is key in properties since we have validated - primary_properties.push_back( - std::ranges::find_if(properties, [property_id = property_id](const auto &property_pair) { - return property_pair.first == property_id; - })->second); - } - - // Get secondary properties - std::vector<std::pair<PropertyId, PropertyValue>> secondary_properties; - for (const auto &[property_id, property_value] : properties) { - if (!storage_->schemas_.IsPropertyKey(primary_label, property_id)) { - secondary_properties.emplace_back(property_id, property_value); - } - } - - auto acc = storage_->vertices_.access(); - auto *delta = CreateDeleteObjectDelta(&transaction_); - auto [it, inserted] = acc.insert({Vertex{delta, primary_label, primary_properties, labels, secondary_properties}}); - MG_ASSERT(inserted, "The vertex must be inserted here!"); - MG_ASSERT(it != acc.end(), "Invalid Vertex accessor!"); - delta->prev.Set(&it->vertex); - return VertexAccessor{ - &it->vertex, &transaction_, &storage_->indices_, &storage_->constraints_, config_, storage_->schema_validator_}; -} - -std::optional<VertexAccessor> Storage::Accessor::FindVertex(std::vector<PropertyValue> primary_key, View view) { - auto acc = storage_->vertices_.access(); - // Later on use label space - auto it = acc.find(primary_key); - if (it == acc.end()) { - return std::nullopt; - } - return VertexAccessor::Create(&it->vertex, &transaction_, &storage_->indices_, &storage_->constraints_, config_, - storage_->schema_validator_, view); -} - -Result<std::optional<VertexAccessor>> Storage::Accessor::DeleteVertex(VertexAccessor *vertex) { - MG_ASSERT(vertex->transaction_ == &transaction_, - "VertexAccessor must be from the same transaction as the storage " - "accessor when deleting a vertex!"); - auto *vertex_ptr = vertex->vertex_; - - if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; - - if (vertex_ptr->deleted) { - return std::optional<VertexAccessor>{}; - } - - if (!vertex_ptr->in_edges.empty() || !vertex_ptr->out_edges.empty()) return Error::VERTEX_HAS_EDGES; - - CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag()); - vertex_ptr->deleted = true; - - return std::make_optional<VertexAccessor>(vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_, - config_, storage_->schema_validator_, true); -} - -Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> Storage::Accessor::DetachDeleteVertex( - VertexAccessor *vertex) { - using ReturnType = std::pair<VertexAccessor, std::vector<EdgeAccessor>>; - - MG_ASSERT(vertex->transaction_ == &transaction_, - "VertexAccessor must be from the same transaction as the storage " - "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; - - { - if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; - - if (vertex_ptr->deleted) return std::optional<ReturnType>{}; - - in_edges = vertex_ptr->in_edges; - out_edges = vertex_ptr->out_edges; - } - - std::vector<EdgeAccessor> deleted_edges; - for (const auto &item : in_edges) { - auto [edge_type, from_vertex, edge] = item; - EdgeAccessor e(edge, edge_type, from_vertex, vertex_ptr, &transaction_, &storage_->indices_, - &storage_->constraints_, config_, storage_->schema_validator_); - auto ret = DeleteEdge(&e); - if (ret.HasError()) { - MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!"); - return ret.GetError(); - } - - if (ret.GetValue()) { - deleted_edges.push_back(*ret.GetValue()); - } - } - for (const auto &item : out_edges) { - auto [edge_type, to_vertex, edge] = item; - EdgeAccessor e(edge, edge_type, vertex_ptr, to_vertex, &transaction_, &storage_->indices_, &storage_->constraints_, - config_, storage_->schema_validator_); - auto ret = DeleteEdge(&e); - if (ret.HasError()) { - MG_ASSERT(ret.GetError() == Error::SERIALIZATION_ERROR, "Invalid database state!"); - return ret.GetError(); - } - - if (ret.GetValue()) { - deleted_edges.push_back(*ret.GetValue()); - } - } - - // We need to check again for serialization errors because we unlocked the - // vertex. Some other transaction could have modified the vertex in the - // meantime if we didn't have any edges to delete. - - if (!PrepareForWrite(&transaction_, vertex_ptr)) return Error::SERIALIZATION_ERROR; - - MG_ASSERT(!vertex_ptr->deleted, "Invalid database state!"); - - CreateAndLinkDelta(&transaction_, vertex_ptr, Delta::RecreateObjectTag()); - vertex_ptr->deleted = true; - - return std::make_optional<ReturnType>( - VertexAccessor{vertex_ptr, &transaction_, &storage_->indices_, &storage_->constraints_, config_, - storage_->schema_validator_, true}, - std::move(deleted_edges)); -} - -Result<EdgeAccessor> Storage::Accessor::CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) { - 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; - } - - auto gid = Gid::FromUint(storage_->edge_id_++); - EdgeRef edge(gid); - if (config_.properties_on_edges) { - auto acc = storage_->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. - ++storage_->edge_count_; - - return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_, - &storage_->constraints_, config_, storage_->schema_validator_); -} - -Result<EdgeAccessor> Storage::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. - storage_->edge_id_ = std::max(storage_->edge_id_, gid.AsUint() + 1); - - EdgeRef edge(gid); - if (config_.properties_on_edges) { - auto acc = storage_->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. - ++storage_->edge_count_; - - return EdgeAccessor(edge, edge_type, from_vertex, to_vertex, &transaction_, &storage_->indices_, - &storage_->constraints_, config_, storage_->schema_validator_); -} - -Result<std::optional<EdgeAccessor>> Storage::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_; - - if (config_.properties_on_edges) { - auto *edge_ptr = edge_ref.ptr; - - if (!PrepareForWrite(&transaction_, edge_ptr)) return Error::SERIALIZATION_ERROR; - - if (edge_ptr->deleted) return std::optional<EdgeAccessor>{}; - } - - auto *from_vertex = edge->from_vertex_; - auto *to_vertex = edge->to_vertex_; - - if (!PrepareForWrite(&transaction_, from_vertex)) return Error::SERIALIZATION_ERROR; - MG_ASSERT(!from_vertex->deleted, "Invalid database state!"); - - if (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); - if (config_.properties_on_edges) { - MG_ASSERT(it != edges->end(), "Invalid database state!"); - } else if (it == edges->end()) { - return false; - } - 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); - - if (config_.properties_on_edges) { - MG_ASSERT((op1 && op2), "Invalid database state!"); - } else { - MG_ASSERT((op1 && op2) || (!op1 && !op2), "Invalid database state!"); - if (!op1 && !op2) { - // The edge is already deleted. - return std::optional<EdgeAccessor>{}; - } - } - - if (config_.properties_on_edges) { - auto *edge_ptr = edge_ref.ptr; - CreateAndLinkDelta(&transaction_, edge_ptr, Delta::RecreateObjectTag()); - 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); - - // Decrement edge count. - --storage_->edge_count_; - - return std::make_optional<EdgeAccessor>(edge_ref, edge_type, from_vertex, to_vertex, &transaction_, - &storage_->indices_, &storage_->constraints_, config_, - storage_->schema_validator_, true); -} - -const std::string &Storage::Accessor::LabelToName(LabelId label) const { return storage_->LabelToName(label); } - -const std::string &Storage::Accessor::PropertyToName(PropertyId property) const { - return storage_->PropertyToName(property); -} - -const std::string &Storage::Accessor::EdgeTypeToName(EdgeTypeId edge_type) const { - return storage_->EdgeTypeToName(edge_type); -} - -LabelId Storage::Accessor::NameToLabel(const std::string_view name) { return storage_->NameToLabel(name); } - -PropertyId Storage::Accessor::NameToProperty(const std::string_view name) { return storage_->NameToProperty(name); } - -EdgeTypeId Storage::Accessor::NameToEdgeType(const std::string_view name) { return storage_->NameToEdgeType(name); } - -void Storage::Accessor::AdvanceCommand() { ++transaction_.command_id; } - -utils::BasicResult<ConstraintViolation, void> Storage::Accessor::Commit( - const std::optional<uint64_t> desired_commit_timestamp) { - MG_ASSERT(is_transaction_active_, "The transaction is already terminated!"); - MG_ASSERT(!transaction_.must_abort, "The transaction can't be committed!"); - - if (transaction_.deltas.empty()) { - // We don't have to update the commit timestamp here because no one reads - // it. - storage_->commit_log_->MarkFinished(transaction_.start_timestamp); - } else { - // Validate that existence constraints are satisfied for all modified - // vertices. - for (const auto &delta : transaction_.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) { - continue; - } - // No need to take any locks here because we modified this vertex and no - // one else can touch it until we commit. - auto validation_result = ValidateExistenceConstraints(*prev.vertex, storage_->constraints_); - if (validation_result) { - Abort(); - return {*validation_result}; - } - } - - // Result of validating the vertex against unique constraints. It has to be - // declared outside of the critical section scope because its value is - // tested for Abort call which has to be done out of the scope. - std::optional<ConstraintViolation> unique_constraint_violation; - - // Save these so we can mark them used in the commit log. - uint64_t start_timestamp = transaction_.start_timestamp; - - commit_timestamp_.emplace(storage_->CommitTimestamp(desired_commit_timestamp)); - - // Before committing and validating vertices against unique constraints, - // we have to update unique constraints with the vertices that are going - // to be validated/committed. - for (const auto &delta : transaction_.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) { - continue; - } - storage_->constraints_.unique_constraints.UpdateBeforeCommit(prev.vertex, transaction_); - } - - // Validate that unique constraints are satisfied for all modified - // vertices. - for (const auto &delta : transaction_.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) { - continue; - } - - // No need to take any locks here because we modified this vertex and no - // one else can touch it until we commit. - unique_constraint_violation = - storage_->constraints_.unique_constraints.Validate(*prev.vertex, transaction_, *commit_timestamp_); - if (unique_constraint_violation) { - break; - } - } - - if (!unique_constraint_violation) { - // Write transaction to WAL while holding the engine lock to make sure - // that committed transactions are sorted by the commit timestamp in the - // WAL files. We supply the new commit timestamp to the function so that - // it knows what will be the final commit timestamp. The WAL must be - // written before actually committing the transaction (before setting - // the commit timestamp) so that no other transaction can see the - // modifications before they are written to disk. - // Replica can log only the write transaction received from Main - // so the Wal files are consistent - if (storage_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) { - storage_->AppendToWal(transaction_, *commit_timestamp_); - } - - // TODO(antaljanosbenjamin): Figure out: - // 1. How the committed transactions are sorted in `committed_transactions_` - // 2. Why it was necessary to lock `committed_transactions_` when it was not accessed at all - // TODO: Update all deltas to have a local copy of the commit timestamp - MG_ASSERT(transaction_.commit_timestamp != nullptr, "Invalid database state!"); - transaction_.commit_timestamp->store(*commit_timestamp_, std::memory_order_release); - // Replica can only update the last commit timestamp with - // the commits received from main. - if (storage_->replication_role_ == ReplicationRole::MAIN || desired_commit_timestamp.has_value()) { - // Update the last commit timestamp - storage_->last_commit_timestamp_ = *commit_timestamp_; - } - - storage_->commit_log_->MarkFinished(start_timestamp); - } - - if (unique_constraint_violation) { - Abort(); - return {*unique_constraint_violation}; - } - } - is_transaction_active_ = false; - - return {}; -} - -void Storage::Accessor::Abort() { - MG_ASSERT(is_transaction_active_, "The transaction is already terminated!"); - - for (const auto &delta : transaction_.deltas) { - auto prev = delta.prev.Get(); - switch (prev.type) { - case PreviousPtr::Type::VERTEX: { - auto *vertex = prev.vertex; - Delta *current = vertex->delta; - while (current != nullptr && - current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { - switch (current->action) { - case Delta::Action::REMOVE_LABEL: { - auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label); - MG_ASSERT(it != vertex->labels.end(), "Invalid database state!"); - std::swap(*it, *vertex->labels.rbegin()); - vertex->labels.pop_back(); - break; - } - case Delta::Action::ADD_LABEL: { - auto it = std::find(vertex->labels.begin(), vertex->labels.end(), current->label); - MG_ASSERT(it == vertex->labels.end(), "Invalid database state!"); - vertex->labels.push_back(current->label); - break; - } - case Delta::Action::SET_PROPERTY: { - vertex->properties.SetProperty(current->property.key, current->property.value); - 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}; - 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}; - 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); - // Increment edge count. We only increment the count here because - // the information in `ADD_IN_EDGE` and `Edge/RECREATE_OBJECT` is - // redundant. Also, `Edge/RECREATE_OBJECT` isn't available when - // edge properties are disabled. - ++storage_->edge_count_; - 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}; - 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()); - vertex->in_edges.pop_back(); - 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}; - 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()); - vertex->out_edges.pop_back(); - // Decrement edge count. We only decrement the count here because - // the information in `REMOVE_IN_EDGE` and `Edge/DELETE_OBJECT` is - // redundant. Also, `Edge/DELETE_OBJECT` isn't available when edge - // properties are disabled. - --storage_->edge_count_; - break; - } - case Delta::Action::DELETE_OBJECT: { - vertex->deleted = true; - InsertVertexPKIntoList(storage_->deleted_vertices_, vertex->keys.Keys()); - break; - } - case Delta::Action::RECREATE_OBJECT: { - vertex->deleted = false; - break; - } - } - current = current->next.load(std::memory_order_acquire); - } - vertex->delta = current; - if (current != nullptr) { - current->prev.Set(vertex); - } - - break; - } - case PreviousPtr::Type::EDGE: { - auto *edge = prev.edge; - Delta *current = edge->delta; - while (current != nullptr && - current->timestamp->load(std::memory_order_acquire) == transaction_.transaction_id) { - switch (current->action) { - case Delta::Action::SET_PROPERTY: { - edge->properties.SetProperty(current->property.key, current->property.value); - break; - } - case Delta::Action::DELETE_OBJECT: { - edge->deleted = true; - storage_->deleted_edges_.push_back(edge->gid); - break; - } - case Delta::Action::RECREATE_OBJECT: { - edge->deleted = false; - break; - } - case Delta::Action::REMOVE_LABEL: - case Delta::Action::ADD_LABEL: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::ADD_OUT_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - case Delta::Action::REMOVE_OUT_EDGE: { - LOG_FATAL("Invalid database state!"); - break; - } - } - current = current->next.load(std::memory_order_acquire); - } - edge->delta = current; - if (current != nullptr) { - current->prev.Set(edge); - } - - break; - } - case PreviousPtr::Type::DELTA: - // pointer probably couldn't be set because allocation failed - case PreviousPtr::Type::NULLPTR: - break; - } - } - - { - uint64_t mark_timestamp = storage_->timestamp_; - - // Release engine lock because we don't have to hold it anymore and - // emplace back could take a long time. - storage_->garbage_undo_buffers_.emplace_back(mark_timestamp, std::move(transaction_.deltas)); - } - - storage_->commit_log_->MarkFinished(transaction_.start_timestamp); - is_transaction_active_ = false; -} - -void Storage::Accessor::FinalizeTransaction() { - if (commit_timestamp_) { - storage_->commit_log_->MarkFinished(*commit_timestamp_); - storage_->committed_transactions_.emplace_back(std::move(transaction_)); - commit_timestamp_.reset(); - } -} - -const std::string &Storage::LabelToName(LabelId label) const { return name_id_mapper_.IdToName(label.AsUint()); } - -const std::string &Storage::PropertyToName(PropertyId property) const { - return name_id_mapper_.IdToName(property.AsUint()); -} - -const std::string &Storage::EdgeTypeToName(EdgeTypeId edge_type) const { - return name_id_mapper_.IdToName(edge_type.AsUint()); -} - -LabelId Storage::NameToLabel(const std::string_view name) { return LabelId::FromUint(name_id_mapper_.NameToId(name)); } - -PropertyId Storage::NameToProperty(const std::string_view name) { - return PropertyId::FromUint(name_id_mapper_.NameToId(name)); -} - -EdgeTypeId Storage::NameToEdgeType(const std::string_view name) { - return EdgeTypeId::FromUint(name_id_mapper_.NameToId(name)); -} - -bool Storage::CreateIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) { - // TODO Fix Index - return false; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -bool Storage::CreateIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { - // TODO Fix Index - // if (!indices_.label_property_index.CreateIndex(label, property, labelspace.access())) return false; - return false; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label, {property}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -bool Storage::DropIndex(LabelId label, const std::optional<uint64_t> desired_commit_timestamp) { - if (!indices_.label_index.DropIndex(label)) return false; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -bool Storage::DropIndex(LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { - if (!indices_.label_property_index.DropIndex(label, property)) return false; - // For a description why using `timestamp_` is correct, see - // `CreateIndex(LabelId label)`. - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label, {property}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -IndicesInfo Storage::ListAllIndices() const { - return {indices_.label_index.ListIndices(), indices_.label_property_index.ListIndices()}; -} - -utils::BasicResult<ConstraintViolation, bool> Storage::CreateExistenceConstraint( - LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) { - // TODO Fix constraints - // auto ret = ::memgraph::storage::v3::CreateExistenceConstraint(&constraints_, label, property, vertices_.access()); - // if (ret.HasError() || !ret.GetValue()) return ret; - return false; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label, {property}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -bool Storage::DropExistenceConstraint(LabelId label, PropertyId property, - const std::optional<uint64_t> desired_commit_timestamp) { - if (!::memgraph::storage::v3::DropExistenceConstraint(&constraints_, label, property)) return false; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label, {property}, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return true; -} - -utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Storage::CreateUniqueConstraint( - LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) { - // TODO Fix constraints - // auto ret = constraints_.unique_constraints.CreateConstraint(label, properties, vertices_.access()); - // if (ret.HasError() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) { - // return ret; - // } - return UniqueConstraints::CreationStatus::ALREADY_EXISTS; - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label, properties, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return UniqueConstraints::CreationStatus::SUCCESS; -} - -UniqueConstraints::DeletionStatus Storage::DropUniqueConstraint( - LabelId label, const std::set<PropertyId> &properties, const std::optional<uint64_t> desired_commit_timestamp) { - auto ret = constraints_.unique_constraints.DropConstraint(label, properties); - if (ret != UniqueConstraints::DeletionStatus::SUCCESS) { - return ret; - } - const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp); - AppendToWal(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label, properties, commit_timestamp); - commit_log_->MarkFinished(commit_timestamp); - last_commit_timestamp_ = commit_timestamp; - return UniqueConstraints::DeletionStatus::SUCCESS; -} - -const SchemaValidator &Storage::Accessor::GetSchemaValidator() const { return storage_->schema_validator_; } - -ConstraintsInfo Storage::ListAllConstraints() const { - return {ListExistenceConstraints(constraints_), constraints_.unique_constraints.ListConstraints()}; -} - -SchemasInfo Storage::ListAllSchemas() const { return {schemas_.ListSchemas()}; } - -const Schemas::Schema *Storage::GetSchema(const LabelId primary_label) const { - return schemas_.GetSchema(primary_label); -} - -bool Storage::CreateSchema(const LabelId primary_label, const std::vector<SchemaProperty> &schemas_types) { - return schemas_.CreateSchema(primary_label, schemas_types); -} - -bool Storage::DropSchema(const LabelId primary_label) { return schemas_.DropSchema(primary_label); } - -StorageInfo Storage::GetInfo() const { - auto vertex_count = vertices_.size(); - double average_degree = 0.0; - if (vertex_count) { - average_degree = 2.0 * static_cast<double>(edge_count_) / static_cast<double>(vertex_count); - } - return {vertex_count, edge_count_, average_degree, utils::GetMemoryUsage(), - utils::GetDirDiskUsage(config_.durability.storage_directory)}; -} - -VerticesIterable Storage::Accessor::Vertices(LabelId label, View view) { - return VerticesIterable(storage_->indices_.label_index.Vertices(label, view, &transaction_)); -} - -VerticesIterable Storage::Accessor::Vertices(LabelId label, PropertyId property, View view) { - return VerticesIterable(storage_->indices_.label_property_index.Vertices( - label, property, std::nullopt, std::nullopt, view, &transaction_, storage_->schema_validator_)); -} - -VerticesIterable Storage::Accessor::Vertices(LabelId label, PropertyId property, const PropertyValue &value, - View view) { - return VerticesIterable(storage_->indices_.label_property_index.Vertices( - label, property, utils::MakeBoundInclusive(value), utils::MakeBoundInclusive(value), view, &transaction_, - storage_->schema_validator_)); -} - -VerticesIterable Storage::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(storage_->indices_.label_property_index.Vertices( - label, property, lower_bound, upper_bound, view, &transaction_, storage_->schema_validator_)); -} - -Transaction Storage::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}; - - transaction_id = transaction_id_++; - // Replica should have only read queries and the write queries - // can come from main instance with any past timestamp. - // To preserve snapshot isolation we set the start timestamp - // of any query on replica to the last commited transaction - // which is timestamp_ as only commit of transaction with writes - // can change the value of it. - if (replication_role_ == ReplicationRole::REPLICA) { - start_timestamp = timestamp_; - } else { - start_timestamp = timestamp_++; - } - - return {transaction_id, start_timestamp, isolation_level}; -} - -// `force` means there are no active transactions, so everything can be deleted without worrying about removing some -// data that is used by an active transaction -template <bool force> -void Storage::CollectGarbage() { - if constexpr (force) { - // TODO(antaljanosbenjamin): figure out whether is there any active transaction or not (probably accessors should - // increment/decrement a counter). If there are no transactions, then garbage collection can be forced - CollectGarbage<false>(); - return; - } - - // Garbage collection must be performed in two phases. In the first phase, - // deltas that won't be applied by any transaction anymore are unlinked from - // the version chains. They cannot be deleted immediately, because there - // might be a transaction that still needs them to terminate the version - // chain traversal. They are instead marked for deletion and will be deleted - // in the second GC phase in this GC iteration or some of the following - // ones. - - uint64_t oldest_active_start_timestamp = commit_log_->OldestActive(); - // We don't move undo buffers of unlinked transactions to garbage_undo_buffers - // list immediately, because we would have to repeatedly take - // garbage_undo_buffers lock. - std::list<std::pair<uint64_t, std::list<Delta>>> unlinked_undo_buffers; - - // We will only free vertices deleted up until now in this GC cycle, and we - // will do it after cleaning-up the indices. That way we are sure that all - // vertices that appear in an index also exist in main storage. - - // Flag that will be used to determine whether the Index GC should be run. It - // should be run when there were any items that were cleaned up (there were - // updates between this run of the GC and the previous run of the GC). This - // eliminates high CPU usage when the GC doesn't have to clean up anything. - bool run_index_cleanup = !committed_transactions_.empty() || !garbage_undo_buffers_.empty(); - - while (true) { - // We don't want to hold the lock on commited transactions for too long, - // because that prevents other transactions from committing. - Transaction *transaction{nullptr}; - { - if (committed_transactions_.empty()) { - break; - } - transaction = &committed_transactions_.front(); - } - - auto commit_timestamp = transaction->commit_timestamp->load(std::memory_order_acquire); - if (commit_timestamp >= oldest_active_start_timestamp) { - break; - } - - // When unlinking a delta which is the first delta in its version chain, - // special care has to be taken to avoid the following race condition: - // - // [Vertex] --> [Delta A] - // - // GC thread: Delta A is the first in its chain, it must be unlinked from - // vertex and marked for deletion - // TX thread: Update vertex and add Delta B with Delta A as next - // - // [Vertex] --> [Delta B] <--> [Delta A] - // - // GC thread: Unlink delta from Vertex - // - // [Vertex] --> (nullptr) - // - // When processing a delta that is the first one in its chain, we - // obtain the corresponding vertex or edge lock, and then verify that this - // delta still is the first in its chain. - // When processing a delta that is in the middle of the chain we only - // process the final delta of the given transaction in that chain. We - // determine the owner of the chain (either a vertex or an edge), obtain the - // corresponding lock, and then verify that this delta is still in the same - // position as it was before taking the lock. - // - // Even though the delta chain is lock-free (both `next` and `prev`) the - // chain should not be modified without taking the lock from the object that - // owns the chain (either a vertex or an edge). Modifying the chain without - // taking the lock will cause subtle race conditions that will leave the - // chain in a broken state. - // The chain can be only read without taking any locks. - - for (Delta &delta : transaction->deltas) { - while (true) { - auto prev = delta.prev.Get(); - switch (prev.type) { - case PreviousPtr::Type::VERTEX: { - Vertex *vertex = prev.vertex; - if (vertex->delta != &delta) { - // Something changed, we're not the first delta in the chain - // anymore. - continue; - } - vertex->delta = nullptr; - if (vertex->deleted) { - InsertVertexPKIntoList(deleted_vertices_, vertex->keys.Keys()); - } - break; - } - case PreviousPtr::Type::EDGE: { - Edge *edge = prev.edge; - if (edge->delta != &delta) { - // Something changed, we're not the first delta in the chain - // anymore. - continue; - } - edge->delta = nullptr; - if (edge->deleted) { - deleted_edges_.push_back(edge->gid); - } - break; - } - case PreviousPtr::Type::DELTA: { - if (prev.delta->timestamp->load(std::memory_order_acquire) == commit_timestamp) { - // The delta that is newer than this one is also a delta from this - // transaction. We skip the current delta and will remove it as a - // part of the suffix later. - break; - } - { - // We need to find the parent object in order to be able to use - // its lock. - auto parent = prev; - while (parent.type == PreviousPtr::Type::DELTA) { - parent = parent.delta->prev.Get(); - } - switch (parent.type) { - case PreviousPtr::Type::VERTEX: - case PreviousPtr::Type::EDGE: - break; - case PreviousPtr::Type::DELTA: - case PreviousPtr::Type::NULLPTR: - LOG_FATAL("Invalid database state!"); - } - } - Delta *prev_delta = prev.delta; - prev_delta->next.store(nullptr, std::memory_order_release); - break; - } - case PreviousPtr::Type::NULLPTR: { - LOG_FATAL("Invalid pointer!"); - } - } - break; - } - } - - unlinked_undo_buffers.emplace_back(0, std::move(transaction->deltas)); - committed_transactions_.pop_front(); - } - - // After unlinking deltas from vertices, we refresh the indices. That way - // we're sure that none of the vertices from `current_deleted_vertices` - // appears in an index, and we can safely remove the from the main storage - // after the last currently active transaction is finished. - if (run_index_cleanup) { - // This operation is very expensive as it traverses through all of the items - // in every index every time. - RemoveObsoleteEntries(&indices_, oldest_active_start_timestamp); - constraints_.unique_constraints.RemoveObsoleteEntries(oldest_active_start_timestamp); - } - - { - uint64_t mark_timestamp = timestamp_; - for (auto &[timestamp, undo_buffer] : unlinked_undo_buffers) { - timestamp = mark_timestamp; - } - garbage_undo_buffers_.splice(garbage_undo_buffers_.end(), unlinked_undo_buffers); - - for (const auto &vertex : deleted_vertices_) { - garbage_vertices_.emplace_back(mark_timestamp, vertex); - } - } - - // if force is set to true we can simply delete all the leftover undos because - // no transaction is active - if constexpr (force) { - garbage_undo_buffers_.clear(); - } else { - while (!garbage_undo_buffers_.empty() && garbage_undo_buffers_.front().first <= oldest_active_start_timestamp) { - garbage_undo_buffers_.pop_front(); - } - } - - { - auto vertex_acc = vertices_.access(); - if constexpr (force) { - // if force is set to true, then we have unique_lock and no transactions are active - // so we can clean all of the deleted vertices - while (!garbage_vertices_.empty()) { - MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!"); - garbage_vertices_.pop_front(); - } - } else { - while (!garbage_vertices_.empty() && garbage_vertices_.front().first < oldest_active_start_timestamp) { - MG_ASSERT(vertex_acc.remove(garbage_vertices_.front().second), "Invalid database state!"); - garbage_vertices_.pop_front(); - } - } - } - { - auto edge_acc = edges_.access(); - for (auto edge : deleted_edges_) { - MG_ASSERT(edge_acc.remove(edge), "Invalid database state!"); - } - } -} - -// tell the linker he can find the CollectGarbage definitions here -template void Storage::CollectGarbage<true>(); -template void Storage::CollectGarbage<false>(); - -bool Storage::InitializeWalFile() { - if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL) - return false; - if (!wal_file_) { - wal_file_.emplace(wal_directory_, uuid_, epoch_id_, config_.items, &name_id_mapper_, wal_seq_num_++, - &file_retainer_); - } - return true; -} - -void Storage::FinalizeWalFile() { - ++wal_unsynced_transactions_; - if (wal_unsynced_transactions_ >= config_.durability.wal_file_flush_every_n_tx) { - wal_file_->Sync(); - wal_unsynced_transactions_ = 0; - } - if (wal_file_->GetSize() / 1024 >= config_.durability.wal_file_size_kibibytes) { - wal_file_->FinalizeWal(); - wal_file_ = std::nullopt; - wal_unsynced_transactions_ = 0; - } else { - // Try writing the internal buffer if possible, if not - // the data should be written as soon as it's possible - // (triggered by the new transaction commit, or some - // reading thread EnabledFlushing) - wal_file_->TryFlushing(); - } -} - -void Storage::AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp) { - if (!InitializeWalFile()) return; - // Traverse deltas and append them to the WAL file. - // A single transaction will always be contained in a single WAL file. - auto current_commit_timestamp = transaction.commit_timestamp->load(std::memory_order_acquire); - - if (replication_role_ == ReplicationRole::MAIN) { - replication_clients_.WithLock([&](auto &clients) { - for (auto &client : clients) { - client->StartTransactionReplication(wal_file_->SequenceNumber()); - } - }); - } - - // Helper lambda that traverses the delta chain on order to find the first - // delta that should be processed and then appends all discovered deltas. - auto find_and_apply_deltas = [&](const auto *delta, const auto &parent, auto filter) { - while (true) { - auto *older = delta->next.load(std::memory_order_acquire); - if (older == nullptr || older->timestamp->load(std::memory_order_acquire) != current_commit_timestamp) break; - delta = older; - } - while (true) { - if (filter(delta->action)) { - wal_file_->AppendDelta(*delta, parent, final_commit_timestamp); - replication_clients_.WithLock([&](auto &clients) { - for (auto &client : clients) { - client->IfStreamingTransaction( - [&](auto &stream) { stream.AppendDelta(*delta, parent, final_commit_timestamp); }); - } - }); - } - auto prev = delta->prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::DELTA) break; - delta = prev.delta; - } - }; - - // The deltas are ordered correctly in the `transaction.deltas` buffer, but we - // don't traverse them in that order. That is because for each delta we need - // information about the vertex or edge they belong to and that information - // isn't stored in the deltas themselves. In order to find out information - // about the corresponding vertex or edge it is necessary to traverse the - // delta chain for each delta until a vertex or edge is encountered. This - // operation is very expensive as the chain grows. - // Instead, we traverse the edges until we find a vertex or edge and traverse - // their delta chains. This approach has a drawback because we lose the - // correct order of the operations. Because of that, we need to traverse the - // deltas several times and we have to manually ensure that the stored deltas - // will be ordered correctly. - - // 1. Process all Vertex deltas and store all operations that create vertices - // and modify vertex data. - for (const auto &delta : transaction.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) continue; - find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { - switch (action) { - case Delta::Action::DELETE_OBJECT: - case Delta::Action::SET_PROPERTY: - case Delta::Action::ADD_LABEL: - case Delta::Action::REMOVE_LABEL: - return true; - - case Delta::Action::RECREATE_OBJECT: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::ADD_OUT_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - case Delta::Action::REMOVE_OUT_EDGE: - return false; - } - }); - } - // 2. Process all Vertex deltas and store all operations that create edges. - for (const auto &delta : transaction.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) continue; - find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { - switch (action) { - case Delta::Action::REMOVE_OUT_EDGE: - return true; - - case Delta::Action::DELETE_OBJECT: - case Delta::Action::RECREATE_OBJECT: - case Delta::Action::SET_PROPERTY: - case Delta::Action::ADD_LABEL: - case Delta::Action::REMOVE_LABEL: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::ADD_OUT_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - return false; - } - }); - } - // 3. Process all Edge deltas and store all operations that modify edge data. - for (const auto &delta : transaction.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::EDGE) continue; - find_and_apply_deltas(&delta, *prev.edge, [](auto action) { - switch (action) { - case Delta::Action::SET_PROPERTY: - return true; - - case Delta::Action::DELETE_OBJECT: - case Delta::Action::RECREATE_OBJECT: - case Delta::Action::ADD_LABEL: - case Delta::Action::REMOVE_LABEL: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::ADD_OUT_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - case Delta::Action::REMOVE_OUT_EDGE: - return false; - } - }); - } - // 4. Process all Vertex deltas and store all operations that delete edges. - for (const auto &delta : transaction.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) continue; - find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { - switch (action) { - case Delta::Action::ADD_OUT_EDGE: - return true; - - case Delta::Action::DELETE_OBJECT: - case Delta::Action::RECREATE_OBJECT: - case Delta::Action::SET_PROPERTY: - case Delta::Action::ADD_LABEL: - case Delta::Action::REMOVE_LABEL: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - case Delta::Action::REMOVE_OUT_EDGE: - return false; - } - }); - } - // 5. Process all Vertex deltas and store all operations that delete vertices. - for (const auto &delta : transaction.deltas) { - auto prev = delta.prev.Get(); - MG_ASSERT(prev.type != PreviousPtr::Type::NULLPTR, "Invalid pointer!"); - if (prev.type != PreviousPtr::Type::VERTEX) continue; - find_and_apply_deltas(&delta, *prev.vertex, [](auto action) { - switch (action) { - case Delta::Action::RECREATE_OBJECT: - return true; - - case Delta::Action::DELETE_OBJECT: - case Delta::Action::SET_PROPERTY: - case Delta::Action::ADD_LABEL: - case Delta::Action::REMOVE_LABEL: - case Delta::Action::ADD_IN_EDGE: - case Delta::Action::ADD_OUT_EDGE: - case Delta::Action::REMOVE_IN_EDGE: - case Delta::Action::REMOVE_OUT_EDGE: - return false; - } - }); - } - - // Add a delta that indicates that the transaction is fully written to the WAL - // file. - wal_file_->AppendTransactionEnd(final_commit_timestamp); - - FinalizeWalFile(); - - replication_clients_.WithLock([&](auto &clients) { - for (auto &client : clients) { - client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(final_commit_timestamp); }); - client->FinalizeTransactionReplication(); - } - }); -} - -void Storage::AppendToWal(durability::StorageGlobalOperation operation, LabelId label, - const std::set<PropertyId> &properties, uint64_t final_commit_timestamp) { - if (!InitializeWalFile()) return; - wal_file_->AppendOperation(operation, label, properties, final_commit_timestamp); - { - if (replication_role_ == ReplicationRole::MAIN) { - replication_clients_.WithLock([&](auto &clients) { - for (auto &client : clients) { - client->StartTransactionReplication(wal_file_->SequenceNumber()); - client->IfStreamingTransaction( - [&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); }); - client->FinalizeTransactionReplication(); - } - }); - } - } - FinalizeWalFile(); -} - -utils::BasicResult<Storage::CreateSnapshotError> Storage::CreateSnapshot() { - if (replication_role_ != ReplicationRole::MAIN) { - return CreateSnapshotError::DisabledForReplica; - } - - // Create the transaction used to create the snapshot. - auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION); - - // Create snapshot. - // durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_, - // config_.durability.snapshot_retention_count, &vertices_, &edges_, - // &name_id_mapper_, &indices_, &constraints_, config_.items, schema_validator_, - // uuid_, epoch_id_, epoch_history_, &file_retainer_); - - // Finalize snapshot transaction. - commit_log_->MarkFinished(transaction.start_timestamp); - return {}; -} - -bool Storage::LockPath() { - auto locker_accessor = global_locker_.Access(); - return locker_accessor.AddPath(config_.durability.storage_directory); -} - -bool Storage::UnlockPath() { - { - auto locker_accessor = global_locker_.Access(); - if (!locker_accessor.RemovePath(config_.durability.storage_directory)) { - return false; - } - } - - // We use locker accessor in seperate scope so we don't produce deadlock - // after we call clean queue. - file_retainer_.CleanQueue(); - return true; -} - -void Storage::FreeMemory() { - CollectGarbage<true>(); - - // SkipList is already threadsafe - vertices_.run_gc(); - edges_.run_gc(); - indices_.label_index.RunGC(); - indices_.label_property_index.RunGC(); -} - -uint64_t Storage::CommitTimestamp(const std::optional<uint64_t> desired_commit_timestamp) { - if (!desired_commit_timestamp) { - return timestamp_++; - } - timestamp_ = std::max(timestamp_, *desired_commit_timestamp + 1); - return *desired_commit_timestamp; -} - -bool Storage::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) { - return false; - } - - replication_server_ = std::make_unique<ReplicationServer>(this, std::move(endpoint), config); - - replication_role_ = ReplicationRole::REPLICA; - return true; -} - -bool Storage::SetMainReplicationRole() { - // We don't want to generate new epoch_id and do the - // cleanup if we're already a MAIN - if (replication_role_ == ReplicationRole::MAIN) { - return false; - } - - // Main instance does not need replication server - // This should be always called first so we finalize everything - replication_server_.reset(nullptr); - - if (wal_file_) { - wal_file_->FinalizeWal(); - wal_file_.reset(); - } - - // Generate new epoch id and save the last one to the history. - if (epoch_history_.size() == kEpochHistoryRetention) { - epoch_history_.pop_front(); - } - epoch_history_.emplace_back(std::move(epoch_id_), last_commit_timestamp_); - epoch_id_ = utils::GenerateUUID(); - - replication_role_ = ReplicationRole::MAIN; - return true; -} - -utils::BasicResult<Storage::RegisterReplicaError> Storage::RegisterReplica( - std::string name, io::network::Endpoint endpoint, const replication::ReplicationMode replication_mode, - const replication::ReplicationClientConfig &config) { - MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can register a replica!"); - - const bool name_exists = replication_clients_.WithLock([&](auto &clients) { - return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; }); - }); - - if (name_exists) { - return RegisterReplicaError::NAME_EXISTS; - } - - const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) { - return std::any_of(clients.begin(), clients.end(), - [&endpoint](const auto &client) { return client->Endpoint() == endpoint; }); - }); - - if (end_point_exists) { - return RegisterReplicaError::END_POINT_EXISTS; - } - - MG_ASSERT(replication_mode == replication::ReplicationMode::SYNC || !config.timeout, - "Only SYNC mode can have a timeout set"); - - auto client = std::make_unique<ReplicationClient>(std::move(name), this, endpoint, replication_mode, config); - if (client->State() == replication::ReplicaState::INVALID) { - return RegisterReplicaError::CONNECTION_FAILED; - } - - return replication_clients_.WithLock([&](auto &clients) -> utils::BasicResult<Storage::RegisterReplicaError> { - // Another thread could have added a client with same name while - // we were connecting to this client. - if (std::any_of(clients.begin(), clients.end(), - [&](const auto &other_client) { return client->Name() == other_client->Name(); })) { - return RegisterReplicaError::NAME_EXISTS; - } - - if (std::any_of(clients.begin(), clients.end(), - [&client](const auto &other_client) { return client->Endpoint() == other_client->Endpoint(); })) { - return RegisterReplicaError::END_POINT_EXISTS; - } - - clients.push_back(std::move(client)); - return {}; - }); -} - -bool Storage::UnregisterReplica(const std::string_view name) { - MG_ASSERT(replication_role_ == ReplicationRole::MAIN, "Only main instance can unregister a replica!"); - return replication_clients_.WithLock([&](auto &clients) { - return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; }); - }); -} - -std::optional<replication::ReplicaState> Storage::GetReplicaState(const std::string_view name) { - return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> { - const auto client_it = - std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; }); - if (client_it == clients.cend()) { - return std::nullopt; - } - return (*client_it)->State(); - }); -} - -ReplicationRole Storage::GetReplicationRole() const { return replication_role_; } - -std::vector<Storage::ReplicaInfo> Storage::ReplicasInfo() { - return replication_clients_.WithLock([](auto &clients) { - std::vector<Storage::ReplicaInfo> replica_info; - replica_info.reserve(clients.size()); - std::transform(clients.begin(), clients.end(), std::back_inserter(replica_info), - [](const auto &client) -> ReplicaInfo { - return {client->Name(), client->Mode(), client->Timeout(), client->Endpoint(), client->State()}; - }); - return replica_info; - }); -} - -void Storage::SetIsolationLevel(IsolationLevel isolation_level) { isolation_level_ = isolation_level; } +Storage::Storage(Config config) : config_{config} {} } // namespace memgraph::storage::v3 diff --git a/src/storage/v3/storage.hpp b/src/storage/v3/storage.hpp index ed884c86e..9305ba707 100644 --- a/src/storage/v3/storage.hpp +++ b/src/storage/v3/storage.hpp @@ -11,609 +11,24 @@ #pragma once -#include <atomic> -#include <cstdint> -#include <filesystem> -#include <map> -#include <numeric> -#include <optional> -#include <shared_mutex> -#include <variant> #include <vector> -#include "io/network/endpoint.hpp" -#include "kvstore/kvstore.hpp" -#include "storage/v3/commit_log.hpp" -#include "storage/v3/config.hpp" -#include "storage/v3/constraints.hpp" -#include "storage/v3/durability/metadata.hpp" -#include "storage/v3/durability/wal.hpp" -#include "storage/v3/edge.hpp" -#include "storage/v3/edge_accessor.hpp" -#include "storage/v3/id_types.hpp" -#include "storage/v3/indices.hpp" -#include "storage/v3/isolation_level.hpp" -#include "storage/v3/key_store.hpp" -#include "storage/v3/lexicographically_ordered_vertex.hpp" -#include "storage/v3/mvcc.hpp" -#include "storage/v3/name_id_mapper.hpp" -#include "storage/v3/property_value.hpp" -#include "storage/v3/result.hpp" -#include "storage/v3/schema_validator.hpp" -#include "storage/v3/schemas.hpp" -#include "storage/v3/transaction.hpp" -#include "storage/v3/vertex.hpp" -#include "storage/v3/vertex_accessor.hpp" -#include "storage/v3/vertices_skip_list.hpp" -#include "utils/exceptions.hpp" -#include "utils/file_locker.hpp" -#include "utils/on_scope_exit.hpp" -#include "utils/rw_lock.hpp" -#include "utils/scheduler.hpp" -#include "utils/skip_list.hpp" -#include "utils/synchronized.hpp" -#include "utils/uuid.hpp" +#include <boost/asio/thread_pool.hpp> -/// REPLICATION /// -#include "rpc/server.hpp" -#include "storage/v3/replication/config.hpp" -#include "storage/v3/replication/enums.hpp" -#include "storage/v3/replication/rpc.hpp" -#include "storage/v3/replication/serialization.hpp" +#include "storage/v3/shard.hpp" namespace memgraph::storage::v3 { -// 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. - -/// Iterable for iterating through all vertices of a Storage. -/// -/// An instance of this will be usually be wrapped inside VerticesIterable for -/// generic, public use. -class AllVerticesIterable final { - VerticesSkipList::Accessor vertices_accessor_; - Transaction *transaction_; - View view_; - Indices *indices_; - Constraints *constraints_; - Config::Items config_; - const SchemaValidator *schema_validator_; - const Schemas *schemas_; - std::optional<VertexAccessor> vertex_; - +class Storage { public: - class Iterator final { - AllVerticesIterable *self_; - VerticesSkipList::Iterator it_; - - public: - Iterator(AllVerticesIterable *self, VerticesSkipList::Iterator it); - - VertexAccessor operator*() const; - - Iterator &operator++(); - - bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; } - - bool operator!=(const Iterator &other) const { return !(*this == other); } - }; - - AllVerticesIterable(VerticesSkipList::Accessor vertices_accessor, Transaction *transaction, View view, - Indices *indices, Constraints *constraints, Config::Items config, - const SchemaValidator &schema_validator) - : vertices_accessor_(std::move(vertices_accessor)), - transaction_(transaction), - view_(view), - indices_(indices), - constraints_(constraints), - config_(config), - schema_validator_{&schema_validator} {} - - Iterator begin() { return {this, vertices_accessor_.begin()}; } - Iterator end() { return {this, vertices_accessor_.end()}; } -}; - -/// Generic access to different kinds of vertex iterations. -/// -/// This class should be the primary type used by the client code to iterate -/// over vertices inside a Storage instance. -class VerticesIterable final { - enum class Type { ALL, BY_LABEL, BY_LABEL_PROPERTY }; - - Type type_; - union { - AllVerticesIterable all_vertices_; - LabelIndex::Iterable vertices_by_label_; - LabelPropertyIndex::Iterable vertices_by_label_property_; - }; - - public: - explicit VerticesIterable(AllVerticesIterable); - explicit VerticesIterable(LabelIndex::Iterable); - explicit VerticesIterable(LabelPropertyIndex::Iterable); - - VerticesIterable(const VerticesIterable &) = delete; - VerticesIterable &operator=(const VerticesIterable &) = delete; - - VerticesIterable(VerticesIterable &&) noexcept; - VerticesIterable &operator=(VerticesIterable &&) noexcept; - - ~VerticesIterable(); - - class Iterator final { - Type type_; - union { - AllVerticesIterable::Iterator all_it_; - LabelIndex::Iterable::Iterator by_label_it_; - LabelPropertyIndex::Iterable::Iterator by_label_property_it_; - }; - - void Destroy() noexcept; - - public: - explicit Iterator(AllVerticesIterable::Iterator); - explicit Iterator(LabelIndex::Iterable::Iterator); - explicit Iterator(LabelPropertyIndex::Iterable::Iterator); - - Iterator(const Iterator &); - Iterator &operator=(const Iterator &); - - Iterator(Iterator &&) noexcept; - Iterator &operator=(Iterator &&) noexcept; - - ~Iterator(); - - VertexAccessor operator*() const; - - Iterator &operator++(); - - bool operator==(const Iterator &other) const; - bool operator!=(const Iterator &other) const { return !(*this == other); } - }; - - Iterator begin(); - Iterator end(); -}; - -/// Structure used to return information about existing indices in the storage. -struct IndicesInfo { - std::vector<LabelId> label; - std::vector<std::pair<LabelId, PropertyId>> label_property; -}; - -/// Structure used to return information about existing constraints in the -/// storage. -struct ConstraintsInfo { - std::vector<std::pair<LabelId, PropertyId>> existence; - std::vector<std::pair<LabelId, std::set<PropertyId>>> unique; -}; - -/// Structure used to return information about existing schemas in the storage -struct SchemasInfo { - Schemas::SchemasList schemas; -}; - -/// 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 }; - -class Storage final { - public: - /// @throw std::system_error - /// @throw std::bad_alloc - explicit Storage(Config config = Config()); - - ~Storage(); - - class Accessor final { - private: - friend class Storage; - - explicit Accessor(Storage *storage, IsolationLevel isolation_level); - - public: - Accessor(const Accessor &) = delete; - Accessor &operator=(const Accessor &) = delete; - Accessor &operator=(Accessor &&other) = delete; - - // NOTE: After the accessor is moved, all objects derived from it (accessors - // and iterators) are *invalid*. You have to get all derived objects again. - Accessor(Accessor &&other) noexcept; - - ~Accessor(); - - /// @throw std::bad_alloc - ResultSchema<VertexAccessor> CreateVertexAndValidate( - LabelId primary_label, const std::vector<LabelId> &labels, - const std::vector<std::pair<PropertyId, PropertyValue>> &properties); - - std::optional<VertexAccessor> FindVertex(std::vector<PropertyValue> primary_key, View view); - - VerticesIterable Vertices(View view) { - return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view, - &storage_->indices_, &storage_->constraints_, storage_->config_.items, - storage_->schema_validator_)); - } - - VerticesIterable Vertices(LabelId label, View view); - - VerticesIterable Vertices(LabelId label, PropertyId property, View view); - - VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view); - - VerticesIterable Vertices(LabelId label, PropertyId property, - const std::optional<utils::Bound<PropertyValue>> &lower_bound, - const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view); - - /// Return approximate number of all vertices in the database. - /// Note that this is always an over-estimate and never an under-estimate. - int64_t ApproximateVertexCount() const { return static_cast<int64_t>(storage_->vertices_.size()); } - - /// Return approximate number of vertices with the given label. - /// Note that this is always an over-estimate and never an under-estimate. - int64_t ApproximateVertexCount(LabelId label) const { - return storage_->indices_.label_index.ApproximateVertexCount(label); - } - - /// Return approximate number of vertices with the given label and property. - /// Note that this is always an over-estimate and never an under-estimate. - int64_t ApproximateVertexCount(LabelId label, PropertyId property) const { - return storage_->indices_.label_property_index.ApproximateVertexCount(label, property); - } - - /// Return approximate number of vertices with the given label and the given - /// value for the given property. Note that this is always an over-estimate - /// and never an under-estimate. - int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const { - return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value); - } - - /// Return approximate number of vertices with the given label and value for - /// the given property in the range defined by provided upper and lower - /// bounds. - int64_t ApproximateVertexCount(LabelId label, PropertyId property, - const std::optional<utils::Bound<PropertyValue>> &lower, - const std::optional<utils::Bound<PropertyValue>> &upper) const { - return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper); - } - - /// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise - /// @throw std::bad_alloc - Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex); - - /// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise - /// @throw std::bad_alloc - Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex( - VertexAccessor *vertex); - - /// @throw std::bad_alloc - Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type); - - /// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise - /// @throw std::bad_alloc - Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge); - - const std::string &LabelToName(LabelId label) const; - const std::string &PropertyToName(PropertyId property) const; - const std::string &EdgeTypeToName(EdgeTypeId edge_type) const; - - /// @throw std::bad_alloc if unable to insert a new mapping - LabelId NameToLabel(std::string_view name); - - /// @throw std::bad_alloc if unable to insert a new mapping - PropertyId NameToProperty(std::string_view name); - - /// @throw std::bad_alloc if unable to insert a new mapping - EdgeTypeId NameToEdgeType(std::string_view name); - - bool LabelIndexExists(LabelId label) const { return storage_->indices_.label_index.IndexExists(label); } - - bool LabelPropertyIndexExists(LabelId label, PropertyId property) const { - return storage_->indices_.label_property_index.IndexExists(label, property); - } - - IndicesInfo ListAllIndices() const { - return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()}; - } - - ConstraintsInfo ListAllConstraints() const { - return {ListExistenceConstraints(storage_->constraints_), - storage_->constraints_.unique_constraints.ListConstraints()}; - } - - const SchemaValidator &GetSchemaValidator() const; - - SchemasInfo ListAllSchemas() const { return {storage_->schemas_.ListSchemas()}; } - - void AdvanceCommand(); - - /// Commit returns `ConstraintViolation` if the changes made by this - /// transaction violate an existence or unique constraint. In that case the - /// transaction is automatically aborted. Otherwise, void is returned. - /// @throw std::bad_alloc - utils::BasicResult<ConstraintViolation, void> Commit(std::optional<uint64_t> desired_commit_timestamp = {}); - - /// @throw std::bad_alloc - void Abort(); - - void FinalizeTransaction(); - - private: - /// @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); - - Storage *storage_; - Transaction transaction_; - std::optional<uint64_t> commit_timestamp_; - bool is_transaction_active_; - Config::Items config_; - }; - - Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) { - return Accessor{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; - - /// @throw std::bad_alloc if unable to insert a new mapping - LabelId NameToLabel(std::string_view name); - - /// @throw std::bad_alloc if unable to insert a new mapping - PropertyId NameToProperty(std::string_view name); - - /// @throw std::bad_alloc if unable to insert a new mapping - EdgeTypeId NameToEdgeType(std::string_view name); - - /// @throw std::bad_alloc - bool CreateIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {}); - - /// @throw std::bad_alloc - bool CreateIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); - - bool DropIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {}); - - bool DropIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); - - IndicesInfo ListAllIndices() const; - - /// Creates an existence constraint. Returns true if the constraint was - /// successfully added, false if it already exists and a `ConstraintViolation` - /// if there is an existing vertex violating the constraint. - /// - /// @throw std::bad_alloc - /// @throw std::length_error - utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint( - LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {}); - - /// Removes an existence constraint. Returns true if the constraint was - /// removed, and false if it doesn't exist. - bool DropExistenceConstraint(LabelId label, PropertyId property, - std::optional<uint64_t> desired_commit_timestamp = {}); - - /// Creates a unique constraint. In the case of two vertices violating the - /// constraint, it returns `ConstraintViolation`. Otherwise returns a - /// `UniqueConstraints::CreationStatus` enum with the following possibilities: - /// * `SUCCESS` if the constraint was successfully created, - /// * `ALREADY_EXISTS` if the constraint already existed, - /// * `EMPTY_PROPERTIES` if the property set is empty, or - // * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the - // limit of maximum number of properties. - /// - /// @throw std::bad_alloc - utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> CreateUniqueConstraint( - LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {}); - - /// Removes a unique constraint. Returns `UniqueConstraints::DeletionStatus` - /// enum with the following possibilities: - /// * `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. - UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties, - std::optional<uint64_t> desired_commit_timestamp = {}); - - ConstraintsInfo ListAllConstraints() const; - - SchemasInfo ListAllSchemas() const; - - const Schemas::Schema *GetSchema(LabelId primary_label) const; - - bool CreateSchema(LabelId primary_label, const std::vector<SchemaProperty> &schemas_types); - - bool DropSchema(LabelId primary_label); - - StorageInfo GetInfo() const; - - bool LockPath(); - bool UnlockPath(); - - bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {}); - - bool SetMainReplicationRole(); - - 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 - utils::BasicResult<RegisterReplicaError, void> RegisterReplica( - std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode, - const replication::ReplicationClientConfig &config = {}); - /// @pre The instance should have a MAIN role - bool UnregisterReplica(std::string_view name); - - std::optional<replication::ReplicaState> GetReplicaState(std::string_view name); - - ReplicationRole GetReplicationRole() const; - - struct ReplicaInfo { - std::string name; - replication::ReplicationMode mode; - std::optional<double> timeout; - io::network::Endpoint endpoint; - replication::ReplicaState state; - }; - - std::vector<ReplicaInfo> ReplicasInfo(); - - void FreeMemory(); - - void SetIsolationLevel(IsolationLevel isolation_level); - - enum class CreateSnapshotError : uint8_t { DisabledForReplica }; - - utils::BasicResult<CreateSnapshotError> CreateSnapshot(); + explicit Storage(Config config); + // Interface toward shard manipulation + // Shard handler -> will use rsm client private: - Transaction CreateTransaction(IsolationLevel isolation_level); - - /// The force parameter determines the behaviour of the garbage collector. - /// If it's set to true, it will behave as a global operation, i.e. it can't - /// be part of a transaction, and no other transaction can be active at the same time. - /// This allows it to delete immediately vertices without worrying that some other - /// transaction is possibly using it. If there are active transactions when this method - /// is called with force set to true, it will fallback to the same method with the force - /// set to false. - /// If it's set to false, it will execute in parallel with other transactions, ensuring - /// that no object in use can be deleted. - /// @throw std::system_error - /// @throw std::bad_alloc - template <bool force> - void CollectGarbage(); - - bool InitializeWalFile(); - void FinalizeWalFile(); - - void AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp); - void AppendToWal(durability::StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties, - uint64_t final_commit_timestamp); - - uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {}); - - // Main object storage - 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. - uint64_t edge_count_{0}; - - NameIdMapper name_id_mapper_; - - SchemaValidator schema_validator_; - Constraints constraints_; - Indices indices_; - Schemas schemas_; - - // Transaction engine - uint64_t timestamp_{kTimestampInitialId}; - uint64_t transaction_id_{kTransactionInitialId}; - // TODO: This isn't really a commit log, it doesn't even care if a - // transaction commited or aborted. We could probably combine this with - // `timestamp_` in a sensible unit, something like TransactionClock or - // whatever. - std::optional<CommitLog> commit_log_; - - std::list<Transaction> committed_transactions_; - IsolationLevel isolation_level_; - + std::vector<Shard> shards_; + boost::asio::thread_pool shard_handlers_; Config config_; - - // Undo buffers that were unlinked and now are waiting to be freed. - std::list<std::pair<uint64_t, std::list<Delta>>> garbage_undo_buffers_; - - // Vertices that are logically deleted but still have to be removed from - // indices before removing them from the main storage. - std::list<PrimaryKey> deleted_vertices_; - - // Vertices that are logically deleted and removed from indices and now wait - // to be removed from the main storage. - std::list<std::pair<uint64_t, PrimaryKey>> garbage_vertices_; - - // Edges that are logically deleted and wait to be removed from the main - // storage. - std::list<Gid> deleted_edges_; - - // Durability - std::filesystem::path snapshot_directory_; - std::filesystem::path wal_directory_; - std::filesystem::path lock_file_path_; - utils::OutputFile lock_file_handle_; - - // UUID used to distinguish snapshots and to link snapshots to WALs - std::string uuid_; - // Sequence number used to keep track of the chain of WALs. - uint64_t wal_seq_num_{0}; - - // UUID to distinguish different main instance runs for replication process - // on SAME storage. - // Multiple instances can have same storage UUID and be MAIN at the same time. - // We cannot compare commit timestamps of those instances if one of them - // becomes the replica of the other so we use epoch_id_ as additional - // discriminating property. - // Example of this: - // We have 2 instances of the same storage, S1 and S2. - // S1 and S2 are MAIN and accept their own commits and write them to the WAL. - // At the moment when S1 commited a transaction with timestamp 20, and S2 - // a different transaction with timestamp 15, we change S2's role to REPLICA - // and register it on S1. - // Without using the epoch_id, we don't know that S1 and S2 have completely - // different transactions, we think that the S2 is behind only by 5 commits. - std::string epoch_id_; - // History of the previous epoch ids. - // Each value consists of the epoch id along the last commit belonging to that - // epoch. - std::deque<std::pair<std::string, uint64_t>> epoch_history_; - - std::optional<durability::WalFile> wal_file_; - uint64_t wal_unsynced_transactions_{0}; - - utils::FileRetainer file_retainer_; - - // Global locker that is used for clients file locking - utils::FileRetainer::FileLocker global_locker_; - - // Last commited timestamp - uint64_t last_commit_timestamp_{kTimestampInitialId}; - - class ReplicationServer; - std::unique_ptr<ReplicationServer> replication_server_{nullptr}; - - class ReplicationClient; - // We create ReplicationClient using unique_ptr so we can move - // newly created client into the vector. - // We cannot move the client directly because it contains ThreadPool - // which cannot be moved. Also, the move is necessary because - // we don't want to create the client directly inside the vector - // because that would require the lock on the list putting all - // commits (they iterate list of clients) to halt. - // This way we can initialize client in main thread which means - // that we can immediately notify the user if the initialization - // failed. - using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>; - ReplicationClientList replication_clients_; - - ReplicationRole replication_role_{ReplicationRole::MAIN}; }; } // namespace memgraph::storage::v3 diff --git a/src/storage/v3/vertex_accessor.hpp b/src/storage/v3/vertex_accessor.hpp index 061cf399d..71e297478 100644 --- a/src/storage/v3/vertex_accessor.hpp +++ b/src/storage/v3/vertex_accessor.hpp @@ -25,7 +25,7 @@ namespace memgraph::storage::v3 { class EdgeAccessor; -class Storage; +class Shard; struct Indices; struct Constraints; @@ -46,7 +46,7 @@ class VertexAccessor final { private: const Vertex *vertex_; }; - friend class Storage; + friend class Shard; public: // Be careful when using VertexAccessor since it can be instantiated with diff --git a/tests/unit/CMakeLists.txt b/tests/unit/CMakeLists.txt index afd1f8dd9..cceade081 100644 --- a/tests/unit/CMakeLists.txt +++ b/tests/unit/CMakeLists.txt @@ -337,15 +337,15 @@ target_link_libraries(${test_prefix}storage_v3_key_store mg-storage-v3 rapidchec add_unit_test(storage_v3_vertex_accessors.cpp) target_link_libraries(${test_prefix}storage_v3_vertex_accessors mg-storage-v3) -# Test mg-query-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) # add_unit_test(query_v2_query_plan_accumulate_aggregate.cpp) # target_link_libraries(${test_prefix}query_v2_query_plan_accumulate_aggregate mg-query-v2) -# add_unit_test(query_v2_query_plan_create_set_remove_delete.cpp) -# target_link_libraries(${test_prefix}query_v2_query_plan_create_set_remove_delete mg-query-v2) +# # add_unit_test(query_v2_query_plan_create_set_remove_delete.cpp) +# # target_link_libraries(${test_prefix}query_v2_query_plan_create_set_remove_delete mg-query-v2) # add_unit_test(query_v2_query_plan_bag_semantics.cpp) # target_link_libraries(${test_prefix}query_v2_query_plan_bag_semantics mg-query-v2) diff --git a/tests/unit/query_v2_interpreter.cpp b/tests/unit/query_v2_interpreter.cpp index fdc15b032..c24f2b995 100644 --- a/tests/unit/query_v2_interpreter.cpp +++ b/tests/unit/query_v2_interpreter.cpp @@ -29,27 +29,29 @@ #include "query_v2_query_common.hpp" #include "result_stream_faker.hpp" #include "storage/v3/isolation_level.hpp" +#include "storage/v3/key_store.hpp" +#include "storage/v3/name_id_mapper.hpp" #include "storage/v3/property_value.hpp" #include "utils/csv_parsing.hpp" #include "utils/logging.hpp" namespace memgraph::query::v2::tests { -// auto ToEdgeList(const memgraph::communication::bolt::Value &v) { -// std::vector<memgraph::communication::bolt::Edge> list; -// for (auto x : v.ValueList()) { -// list.push_back(x.ValueEdge()); -// } -// return list; -// } +auto ToEdgeList(const memgraph::communication::bolt::Value &v) { + std::vector<memgraph::communication::bolt::Edge> list; + for (auto x : v.ValueList()) { + list.push_back(x.ValueEdge()); + } + return list; +} -// auto StringToUnorderedSet(const std::string &element) { -// const auto element_split = memgraph::utils::Split(element, ", "); -// return std::unordered_set<std::string>(element_split.begin(), element_split.end()); -// }; +auto StringToUnorderedSet(const std::string &element) { + const auto element_split = memgraph::utils::Split(element, ", "); + return std::unordered_set<std::string>(element_split.begin(), element_split.end()); +}; struct InterpreterFaker { - InterpreterFaker(memgraph::storage::v3::Storage *db, const memgraph::query::v2::InterpreterConfig config, + InterpreterFaker(memgraph::storage::v3::Shard *db, const memgraph::query::v2::InterpreterConfig config, const std::filesystem::path &data_directory) : interpreter_context(db, config, data_directory), interpreter(&interpreter_context) { interpreter_context.auth_checker = &auth_checker; @@ -113,10 +115,20 @@ class InterpreterTest : public ::testing::Test { return default_interpreter.Interpret(query, params); } - memgraph::storage::v3::Storage db_; + storage::v3::LabelId NameToLabelId(std::string_view label_name) { + return storage::v3::LabelId::FromUint(id_mapper.NameToId(label_name)); + } + + storage::v3::PropertyId NameToPropertyId(std::string_view property_name) { + return storage::v3::PropertyId::FromUint(id_mapper.NameToId(property_name)); + } + + storage::v3::PrimaryKey pk{storage::v3::PropertyValue(0)}; + memgraph::storage::v3::NameIdMapper id_mapper; + const storage::v3::LabelId label{NameToLabelId("label")}; + storage::v3::Shard db_{label, pk, std::nullopt}; std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "MG_tests_unit_query_v2_interpreter"}; - const storage::v3::LabelId label{db_.NameToLabel("label")}; - const storage::v3::PropertyId property{db_.NameToProperty("property")}; + const storage::v3::PropertyId property{NameToPropertyId("property")}; InterpreterFaker default_interpreter{&db_, {}, data_directory}; }; @@ -150,8 +162,8 @@ TEST_F(InterpreterTest, DummyTestToForceQueryV2Compilation) { // } // } -// // Run query with different ast twice to see if query executes correctly when -// // ast is read from cache. +// Run query with different ast twice to see if query executes correctly when +// ast is read from cache. // TEST_F(InterpreterTest, AstCache) { // { // auto stream = Interpret("RETURN 2 + 3"); @@ -1447,8 +1459,8 @@ TEST_F(InterpreterTest, DummyTestToForceQueryV2Compilation) { // auto notification = notifications[0].ValueMap(); // ASSERT_EQ(notification["severity"].ValueString(), "INFO"); // ASSERT_EQ(notification["code"].ValueString(), "ConstraintDoesNotExist"); -// ASSERT_EQ(notification["title"].ValueString(), "Constraint EXISTS on label L1 on properties name doesn't -// exist."); ASSERT_EQ(notification["description"].ValueString(), ""); +// ASSERT_EQ(notification["title"].ValueString(), "Constraint EXISTS on label L1 on properties name doesn'texist."); +// ASSERT_EQ(notification["description"].ValueString(), ""); // } // } diff --git a/tests/unit/result_stream_faker.hpp b/tests/unit/result_stream_faker.hpp index 60c5884e7..6e7aab261 100644 --- a/tests/unit/result_stream_faker.hpp +++ b/tests/unit/result_stream_faker.hpp @@ -26,7 +26,7 @@ */ class ResultStreamFaker { public: - explicit ResultStreamFaker(memgraph::storage::v3::Storage *store) : store_(store) {} + explicit ResultStreamFaker(memgraph::storage::v3::Shard *store) : store_(store) {} ResultStreamFaker(const ResultStreamFaker &) = delete; ResultStreamFaker &operator=(const ResultStreamFaker &) = delete; @@ -124,7 +124,7 @@ class ResultStreamFaker { } private: - memgraph::storage::v3::Storage *store_; + memgraph::storage::v3::Shard *store_; // the data that the record stream can accept std::vector<std::string> header_; std::vector<std::vector<memgraph::communication::bolt::Value>> results_; diff --git a/tests/unit/storage_v3.cpp b/tests/unit/storage_v3.cpp index fbcd85704..4163c96a2 100644 --- a/tests/unit/storage_v3.cpp +++ b/tests/unit/storage_v3.cpp @@ -19,12 +19,14 @@ #include "storage/v3/delta.hpp" #include "storage/v3/id_types.hpp" #include "storage/v3/key_store.hpp" +#include "storage/v3/name_id_mapper.hpp" #include "storage/v3/property_value.hpp" #include "storage/v3/result.hpp" #include "storage/v3/schema_validator.hpp" -#include "storage/v3/storage.hpp" +#include "storage/v3/shard.hpp" #include "storage/v3/vertex_accessor.hpp" #include "storage_v3_test_utils.hpp" +#include "utils/exceptions.hpp" using testing::UnorderedElementsAre; @@ -37,7 +39,7 @@ class StorageV3 : public ::testing::Test { store.CreateSchema(primary_label, {storage::v3::SchemaProperty{primary_property, common::SchemaType::INT}})); } - VertexAccessor CreateVertexAndValidate(Storage::Accessor &acc, LabelId primary_label, + VertexAccessor CreateVertexAndValidate(Shard::Accessor &acc, LabelId primary_label, const std::vector<LabelId> &labels, const std::vector<std::pair<PropertyId, PropertyValue>> &properties) { auto vtx = acc.CreateVertexAndValidate(primary_label, labels, properties); @@ -45,10 +47,21 @@ class StorageV3 : public ::testing::Test { return *vtx; } - Storage store; - const LabelId primary_label{store.NameToLabel("label")}; - const PropertyId primary_property{store.NameToProperty("property")}; + LabelId NameToLabelId(std::string_view label_name) { return LabelId::FromUint(id_mapper.NameToId(label_name)); } + + PropertyId NameToPropertyId(std::string_view property_name) { + return PropertyId::FromUint(id_mapper.NameToId(property_name)); + } + + EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) { + return EdgeTypeId::FromUint(id_mapper.NameToId(edge_type_name)); + } + + NameIdMapper id_mapper; const std::vector<PropertyValue> pk{PropertyValue{0}}; + const LabelId primary_label{NameToLabelId("label")}; + Shard store{primary_label, pk, std::nullopt}; + const PropertyId primary_property{NameToPropertyId("property")}; }; // NOLINTNEXTLINE(hicpp-special-member-functions) @@ -236,7 +249,7 @@ TEST_F(StorageV3, AccessorMove) { ASSERT_TRUE(acc.FindVertex(pk, View::NEW).has_value()); EXPECT_EQ(CountVertices(acc, View::NEW), 1U); - Storage::Accessor moved(std::move(acc)); + Shard::Accessor moved(std::move(acc)); ASSERT_FALSE(moved.FindVertex(pk, View::OLD).has_value()); EXPECT_EQ(CountVertices(moved, View::OLD), 0U); @@ -600,7 +613,7 @@ TEST_F(StorageV3, VertexDeleteLabel) { auto vertex = acc.FindVertex(pk, View::NEW); ASSERT_TRUE(vertex); - auto label5 = acc.NameToLabel("label5"); + auto label5 = NameToLabelId("label5"); // Check whether label 5 exists ASSERT_FALSE(vertex->HasLabel(label5, View::OLD).GetValue()); @@ -651,7 +664,7 @@ TEST_F(StorageV3, VertexDeleteLabel) { auto vertex = acc.FindVertex(pk, View::NEW); ASSERT_TRUE(vertex); - auto label5 = acc.NameToLabel("label5"); + auto label5 = NameToLabelId("label5"); // Check whether label 5 exists ASSERT_FALSE(vertex->HasLabel(label5, View::OLD).GetValue()); @@ -744,7 +757,7 @@ TEST_F(StorageV3, VertexDeleteProperty) { auto vertex = acc.FindVertex(pk, View::NEW); ASSERT_TRUE(vertex); - auto property5 = acc.NameToProperty("property5"); + auto property5 = NameToPropertyId("property5"); // Check whether property 5 exists ASSERT_TRUE(vertex->GetProperty(property5, View::OLD)->IsNull()); @@ -790,7 +803,7 @@ TEST_F(StorageV3, VertexDeleteProperty) { auto vertex = acc.FindVertex(pk, View::NEW); ASSERT_TRUE(vertex); - auto property5 = acc.NameToProperty("property5"); + auto property5 = NameToPropertyId("property5"); // Check whether property 5 exists ASSERT_TRUE(vertex->GetProperty(property5, View::OLD)->IsNull()); @@ -866,7 +879,7 @@ TEST_F(StorageV3, VertexLabelCommit) { auto acc = store.Access(); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}}); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex.HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex.Labels(View::NEW)->size(), 0); @@ -897,7 +910,7 @@ TEST_F(StorageV3, VertexLabelCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue()); { @@ -913,7 +926,7 @@ TEST_F(StorageV3, VertexLabelCommit) { ASSERT_EQ(labels[0], label); } - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -925,7 +938,7 @@ TEST_F(StorageV3, VertexLabelCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); { auto res = vertex->RemoveLabelAndValidate(label); @@ -956,14 +969,14 @@ TEST_F(StorageV3, VertexLabelCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0); ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0); - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -987,7 +1000,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0); @@ -1020,14 +1033,14 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0); ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0); - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -1041,7 +1054,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0); @@ -1074,7 +1087,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue()); { @@ -1090,7 +1103,7 @@ TEST_F(StorageV3, VertexLabelAbort) { ASSERT_EQ(labels[0], label); } - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -1104,7 +1117,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); { auto res = vertex->RemoveLabelAndValidate(label); @@ -1137,7 +1150,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_TRUE(vertex->HasLabel(label, View::OLD).GetValue()); { @@ -1153,7 +1166,7 @@ TEST_F(StorageV3, VertexLabelAbort) { ASSERT_EQ(labels[0], label); } - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -1167,7 +1180,7 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); { auto res = vertex->RemoveLabelAndValidate(label); @@ -1200,14 +1213,14 @@ TEST_F(StorageV3, VertexLabelAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label = acc.NameToLabel("label5"); + auto label = NameToLabelId("label5"); ASSERT_FALSE(vertex->HasLabel(label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label, View::NEW).GetValue()); ASSERT_EQ(vertex->Labels(View::OLD)->size(), 0); ASSERT_EQ(vertex->Labels(View::NEW)->size(), 0); - auto other_label = acc.NameToLabel("other"); + auto other_label = NameToLabelId("other"); ASSERT_FALSE(vertex->HasLabel(other_label, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(other_label, View::NEW).GetValue()); @@ -1232,8 +1245,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) { auto vertex = acc1.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label1 = acc1.NameToLabel("label1"); - auto label2 = acc1.NameToLabel("label2"); + auto label1 = NameToLabelId("label1"); + auto label2 = NameToLabelId("label2"); ASSERT_FALSE(vertex->HasLabel(label1, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label1, View::NEW).GetValue()); @@ -1271,8 +1284,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) { auto vertex = acc2.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label1 = acc2.NameToLabel("label1"); - auto label2 = acc2.NameToLabel("label2"); + auto label1 = NameToLabelId("label1"); + auto label2 = NameToLabelId("label2"); ASSERT_FALSE(vertex->HasLabel(label1, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label1, View::NEW).GetValue()); @@ -1297,8 +1310,8 @@ TEST_F(StorageV3, VertexLabelSerializationError) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto label1 = acc.NameToLabel("label1"); - auto label2 = acc.NameToLabel("label2"); + auto label1 = NameToLabelId("label1"); + auto label2 = NameToLabelId("label2"); ASSERT_TRUE(vertex->HasLabel(label1, View::OLD).GetValue()); ASSERT_FALSE(vertex->HasLabel(label2, View::OLD).GetValue()); @@ -1326,7 +1339,7 @@ TEST_F(StorageV3, VertexPropertyCommit) { auto acc = store.Access(); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}}); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex.GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex.Properties(View::NEW)->size(), 0); @@ -1364,7 +1377,7 @@ TEST_F(StorageV3, VertexPropertyCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare"); { @@ -1380,7 +1393,7 @@ TEST_F(StorageV3, VertexPropertyCommit) { ASSERT_EQ(properties[property].ValueString(), "nandare"); } - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1392,7 +1405,7 @@ TEST_F(StorageV3, VertexPropertyCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); { auto old_value = vertex->SetPropertyAndValidate(property, PropertyValue()); @@ -1423,14 +1436,14 @@ TEST_F(StorageV3, VertexPropertyCommit) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0); ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0); - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1454,7 +1467,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0); @@ -1494,14 +1507,14 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0); ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0); - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1515,7 +1528,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0); @@ -1555,7 +1568,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare"); { @@ -1571,7 +1584,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { ASSERT_EQ(properties[property].ValueString(), "nandare"); } - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1585,7 +1598,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare"); { @@ -1626,7 +1639,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare"); { @@ -1642,7 +1655,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { ASSERT_EQ(properties[property].ValueString(), "nandare"); } - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1656,7 +1669,7 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_EQ(vertex->GetProperty(property, View::OLD)->ValueString(), "nandare"); { @@ -1697,14 +1710,14 @@ TEST_F(StorageV3, VertexPropertyAbort) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property = acc.NameToProperty("property5"); + auto property = NameToPropertyId("property5"); ASSERT_TRUE(vertex->GetProperty(property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(property, View::NEW)->IsNull()); ASSERT_EQ(vertex->Properties(View::OLD)->size(), 0); ASSERT_EQ(vertex->Properties(View::NEW)->size(), 0); - auto other_property = acc.NameToProperty("other"); + auto other_property = NameToPropertyId("other"); ASSERT_TRUE(vertex->GetProperty(other_property, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(other_property, View::NEW)->IsNull()); @@ -1729,8 +1742,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) { auto vertex = acc1.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property1 = acc1.NameToProperty("property1"); - auto property2 = acc1.NameToProperty("property2"); + auto property1 = NameToPropertyId("property1"); + auto property2 = NameToPropertyId("property2"); ASSERT_TRUE(vertex->GetProperty(property1, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(property1, View::NEW)->IsNull()); @@ -1762,8 +1775,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) { auto vertex = acc2.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property1 = acc2.NameToProperty("property1"); - auto property2 = acc2.NameToProperty("property2"); + auto property1 = NameToPropertyId("property1"); + auto property2 = NameToPropertyId("property2"); ASSERT_TRUE(vertex->GetProperty(property1, View::OLD)->IsNull()); ASSERT_TRUE(vertex->GetProperty(property1, View::NEW)->IsNull()); @@ -1788,8 +1801,8 @@ TEST_F(StorageV3, VertexPropertySerializationError) { auto vertex = acc.FindVertex(pk, View::OLD); ASSERT_TRUE(vertex); - auto property1 = acc.NameToProperty("property1"); - auto property2 = acc.NameToProperty("property2"); + auto property1 = NameToPropertyId("property1"); + auto property2 = NameToPropertyId("property2"); ASSERT_EQ(vertex->GetProperty(property1, View::OLD)->ValueInt(), 123); ASSERT_TRUE(vertex->GetProperty(property2, View::OLD)->IsNull()); @@ -1816,8 +1829,8 @@ TEST_F(StorageV3, VertexLabelPropertyMixed) { auto acc = store.Access(); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}}); - auto label = acc.NameToLabel("label5"); - auto property = acc.NameToProperty("property5"); + auto label = NameToLabelId("label5"); + auto property = NameToPropertyId("property5"); // Check whether label 5 and property 5 exist ASSERT_FALSE(vertex.HasLabel(label, View::NEW).GetValue()); @@ -2051,8 +2064,8 @@ TEST_F(StorageV3, VertexLabelPropertyMixed) { } TEST_F(StorageV3, VertexPropertyClear) { - auto property1 = store.NameToProperty("property1"); - auto property2 = store.NameToProperty("property2"); + auto property1 = NameToPropertyId("property1"); + auto property2 = NameToPropertyId("property2"); { auto acc = store.Access(); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}}); @@ -2153,8 +2166,8 @@ TEST_F(StorageV3, VertexPropertyClear) { } TEST_F(StorageV3, VertexNonexistentLabelPropertyEdgeAPI) { - auto label1 = store.NameToLabel("label1"); - auto property1 = store.NameToProperty("property1"); + auto label1 = NameToLabelId("label1"); + auto property1 = NameToPropertyId("property1"); auto acc = store.Access(); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue{0}}}); @@ -2182,7 +2195,7 @@ 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, acc.NameToEdgeType("edge")).HasValue()); + ASSERT_TRUE(acc.CreateEdge(&vertex, &vertex, NameToEdgeTypeId("edge")).HasValue()); // Check state after (OLD view). ASSERT_EQ(vertex.Labels(View::OLD).GetError(), Error::NONEXISTENT_OBJECT); @@ -2218,14 +2231,14 @@ TEST_F(StorageV3, VertexVisibilitySingleTransaction) { EXPECT_FALSE(acc2.FindVertex(pk, View::OLD)); EXPECT_FALSE(acc2.FindVertex(pk, View::NEW)); - ASSERT_TRUE(vertex.AddLabelAndValidate(acc1.NameToLabel("label1")).HasValue()); + ASSERT_TRUE(vertex.AddLabelAndValidate(NameToLabelId("label1")).HasValue()); EXPECT_FALSE(acc1.FindVertex(pk, View::OLD)); EXPECT_TRUE(acc1.FindVertex(pk, View::NEW)); EXPECT_FALSE(acc2.FindVertex(pk, View::OLD)); EXPECT_FALSE(acc2.FindVertex(pk, View::NEW)); - ASSERT_TRUE(vertex.SetPropertyAndValidate(acc1.NameToProperty("meaning"), PropertyValue(42)).HasValue()); + ASSERT_TRUE(vertex.SetPropertyAndValidate(NameToPropertyId("meaning"), PropertyValue(42)).HasValue()); auto acc3 = store.Access(); @@ -2302,7 +2315,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) { EXPECT_TRUE(acc2.FindVertex(pk, View::OLD)); EXPECT_TRUE(acc2.FindVertex(pk, View::NEW)); - ASSERT_TRUE(vertex->AddLabelAndValidate(acc1.NameToLabel("label1")).HasValue()); + ASSERT_TRUE(vertex->AddLabelAndValidate(NameToLabelId("label1")).HasValue()); EXPECT_TRUE(acc1.FindVertex(pk, View::OLD)); EXPECT_TRUE(acc1.FindVertex(pk, View::NEW)); @@ -2323,7 +2336,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) { EXPECT_TRUE(acc2.FindVertex(pk, View::OLD)); EXPECT_TRUE(acc2.FindVertex(pk, View::NEW)); - ASSERT_TRUE(vertex->SetPropertyAndValidate(acc1.NameToProperty("meaning"), PropertyValue(42)).HasValue()); + ASSERT_TRUE(vertex->SetPropertyAndValidate(NameToPropertyId("meaning"), PropertyValue(42)).HasValue()); auto acc3 = store.Access(); @@ -2497,7 +2510,7 @@ TEST_F(StorageV3, VertexVisibilityMultipleTransactions) { // NOLINTNEXTLINE(hicpp-special-member-functions) TEST_F(StorageV3, DeletedVertexAccessor) { - const auto property1 = store.NameToProperty("property1"); + const auto property1 = NameToPropertyId("property1"); const PropertyValue property_value{"property_value"}; // Create the vertex @@ -2538,8 +2551,8 @@ TEST_F(StorageV3, DeletedVertexAccessor) { TEST_F(StorageV3, TestCreateVertexAndValidate) { { auto acc = store.Access(); - const auto label1 = store.NameToLabel("label1"); - const auto prop1 = store.NameToProperty("prop1"); + const auto label1 = NameToLabelId("label1"); + const auto prop1 = NameToPropertyId("prop1"); auto vertex = acc.CreateVertexAndValidate(primary_label, {label1}, {{primary_property, PropertyValue(0)}, {prop1, PropertyValue(111)}}); ASSERT_TRUE(vertex.HasValue()); @@ -2552,25 +2565,20 @@ TEST_F(StorageV3, TestCreateVertexAndValidate) { (std::map<PropertyId, PropertyValue>{{prop1, PropertyValue(111)}})); } { - const auto label1 = store.NameToLabel("new_primary_label"); - const auto prop1 = store.NameToProperty("key1"); - const auto prop2 = store.NameToProperty("key2"); - ASSERT_TRUE(store.CreateSchema( - label1, {SchemaProperty{prop1, common::SchemaType::INT}, SchemaProperty{prop2, common::SchemaType::STRING}})); auto acc = store.Access(); - auto vertex = acc.CreateVertexAndValidate(label1, {}, {{prop1, PropertyValue(21)}, {prop2, PropertyValue("test")}}); - ASSERT_TRUE(vertex.HasValue()); - ASSERT_TRUE(vertex->PrimaryLabel(View::NEW).HasValue()); - EXPECT_EQ(vertex->PrimaryLabel(View::NEW).GetValue(), label1); - ASSERT_TRUE(vertex->PrimaryKey(View::NEW).HasValue()); - EXPECT_EQ(vertex->PrimaryKey(View::NEW).GetValue(), (PrimaryKey{{PropertyValue(21), PropertyValue("test")}})); - ASSERT_TRUE(vertex->Properties(View::NEW).HasValue()); - EXPECT_TRUE(vertex->Properties(View::NEW).GetValue().empty()); + const auto label1 = NameToLabelId("label1"); + const auto prop1 = NameToPropertyId("prop1"); + EXPECT_THROW( + { + auto vertex = acc.CreateVertexAndValidate( + label1, {}, {{primary_property, PropertyValue(0)}, {prop1, PropertyValue(111)}}); + }, + utils::BasicException); } { ASSERT_DEATH( { - Storage store; + Shard store(primary_label, pk, std::nullopt); ASSERT_TRUE(store.CreateSchema(primary_label, {storage::v3::SchemaProperty{primary_property, common::SchemaType::INT}})); auto acc = store.Access(); diff --git a/tests/unit/storage_v3_test_utils.cpp b/tests/unit/storage_v3_test_utils.cpp index b1a59b7aa..4a8c7b310 100644 --- a/tests/unit/storage_v3_test_utils.cpp +++ b/tests/unit/storage_v3_test_utils.cpp @@ -13,7 +13,7 @@ namespace memgraph::storage::v3::tests { -size_t CountVertices(Storage::Accessor &storage_accessor, View view) { +size_t CountVertices(Shard::Accessor &storage_accessor, View view) { auto vertices = storage_accessor.Vertices(view); size_t count = 0U; for (auto it = vertices.begin(); it != vertices.end(); ++it, ++count) diff --git a/tests/unit/storage_v3_test_utils.hpp b/tests/unit/storage_v3_test_utils.hpp index 1cc772036..7cbb7414b 100644 --- a/tests/unit/storage_v3_test_utils.hpp +++ b/tests/unit/storage_v3_test_utils.hpp @@ -16,6 +16,6 @@ namespace memgraph::storage::v3::tests { -size_t CountVertices(Storage::Accessor &storage_accessor, View view); +size_t CountVertices(Shard::Accessor &storage_accessor, View view); } // namespace memgraph::storage::v3::tests diff --git a/tests/unit/storage_v3_vertex_accessors.cpp b/tests/unit/storage_v3_vertex_accessors.cpp index dcf94788a..8dea70e59 100644 --- a/tests/unit/storage_v3_vertex_accessors.cpp +++ b/tests/unit/storage_v3_vertex_accessors.cpp @@ -21,7 +21,7 @@ #include "storage/v3/property_value.hpp" #include "storage/v3/result.hpp" #include "storage/v3/schema_validator.hpp" -#include "storage/v3/storage.hpp" +#include "storage/v3/shard.hpp" #include "storage/v3/vertex_accessor.hpp" #include "storage_v3_test_utils.hpp" @@ -35,7 +35,7 @@ class StorageV3Accessor : public ::testing::Test { ASSERT_TRUE(storage.CreateSchema(primary_label, {SchemaProperty{primary_property, common::SchemaType::INT}})); } - VertexAccessor CreateVertexAndValidate(Storage::Accessor &acc, LabelId primary_label, + VertexAccessor CreateVertexAndValidate(Shard::Accessor &acc, LabelId primary_label, const std::vector<LabelId> &labels, const std::vector<std::pair<PropertyId, PropertyValue>> &properties) { auto vtx = acc.CreateVertexAndValidate(primary_label, labels, properties); @@ -43,9 +43,17 @@ class StorageV3Accessor : public ::testing::Test { return *vtx; } - Storage storage; - const LabelId primary_label{storage.NameToLabel("label")}; - const PropertyId primary_property{storage.NameToProperty("property")}; + LabelId NameToLabelId(std::string_view label_name) { return LabelId::FromUint(id_mapper.NameToId(label_name)); } + + PropertyId NameToPropertyId(std::string_view property_name) { + return PropertyId::FromUint(id_mapper.NameToId(property_name)); + } + + const std::vector<PropertyValue> pk{PropertyValue{0}}; + NameIdMapper id_mapper; + Shard storage{NameToLabelId("label"), pk, std::nullopt}; + const LabelId primary_label{NameToLabelId("label")}; + const PropertyId primary_property{NameToPropertyId("property")}; }; TEST_F(StorageV3Accessor, TestPrimaryLabel) { @@ -85,9 +93,9 @@ TEST_F(StorageV3Accessor, TestPrimaryLabel) { TEST_F(StorageV3Accessor, TestAddLabels) { { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label1"); - const auto label2 = storage.NameToLabel("label2"); - const auto label3 = storage.NameToLabel("label3"); + const auto label1 = NameToLabelId("label1"); + const auto label2 = NameToLabelId("label2"); + const auto label3 = NameToLabelId("label3"); const auto vertex = CreateVertexAndValidate(acc, primary_label, {label1, label2, label3}, {{primary_property, PropertyValue(0)}}); ASSERT_TRUE(vertex.Labels(View::NEW).HasValue()); @@ -96,9 +104,9 @@ TEST_F(StorageV3Accessor, TestAddLabels) { } { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label1"); - const auto label2 = storage.NameToLabel("label2"); - const auto label3 = storage.NameToLabel("label3"); + const auto label1 = NameToLabelId("label1"); + const auto label2 = NameToLabelId("label2"); + const auto label3 = NameToLabelId("label3"); auto vertex = CreateVertexAndValidate(acc, primary_label, {label1}, {{primary_property, PropertyValue(1)}}); ASSERT_TRUE(vertex.Labels(View::NEW).HasValue()); ASSERT_FALSE(vertex.Labels(View::OLD).HasValue()); @@ -111,7 +119,7 @@ TEST_F(StorageV3Accessor, TestAddLabels) { } { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label"); + const auto label1 = NameToLabelId("label"); auto vertex = acc.CreateVertexAndValidate(primary_label, {label1}, {{primary_property, PropertyValue(2)}}); ASSERT_TRUE(vertex.HasError()); ASSERT_TRUE(std::holds_alternative<SchemaViolation>(vertex.GetError())); @@ -120,7 +128,7 @@ TEST_F(StorageV3Accessor, TestAddLabels) { } { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label"); + const auto label1 = NameToLabelId("label"); auto vertex = acc.CreateVertexAndValidate(primary_label, {}, {{primary_property, PropertyValue(3)}}); ASSERT_TRUE(vertex.HasValue()); const auto schema_violation = vertex->AddLabelAndValidate(label1); @@ -134,9 +142,9 @@ TEST_F(StorageV3Accessor, TestAddLabels) { TEST_F(StorageV3Accessor, TestRemoveLabels) { { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label1"); - const auto label2 = storage.NameToLabel("label2"); - const auto label3 = storage.NameToLabel("label3"); + const auto label1 = NameToLabelId("label1"); + const auto label2 = NameToLabelId("label2"); + const auto label3 = NameToLabelId("label3"); auto vertex = CreateVertexAndValidate(acc, primary_label, {label1, label2, label3}, {{primary_property, PropertyValue(0)}}); ASSERT_TRUE(vertex.Labels(View::NEW).HasValue()); @@ -157,7 +165,7 @@ TEST_F(StorageV3Accessor, TestRemoveLabels) { } { auto acc = storage.Access(); - const auto label1 = storage.NameToLabel("label1"); + const auto label1 = NameToLabelId("label1"); auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue(1)}}); ASSERT_TRUE(vertex.Labels(View::NEW).HasValue()); EXPECT_TRUE(vertex.Labels(View::NEW).GetValue().empty()); @@ -179,7 +187,7 @@ TEST_F(StorageV3Accessor, TestRemoveLabels) { TEST_F(StorageV3Accessor, TestSetKeysAndProperties) { { auto acc = storage.Access(); - const PropertyId prop1{storage.NameToProperty("prop1")}; + const PropertyId prop1{NameToPropertyId("prop1")}; auto vertex = CreateVertexAndValidate(acc, primary_label, {}, {{primary_property, PropertyValue(0)}}); const auto res = vertex.SetPropertyAndValidate(prop1, PropertyValue(1)); ASSERT_TRUE(res.HasValue());