Refactor interpreter to support multiple distributed clocks (Part 1) (#1281)

* Interpreter transaction ID decoupled from storage transaction ID
* Transactional scope for indices, statistics and constraints
* Storage::Accessor now has 2 modes (unique and shared)
* Introduced ResourceLock to fix pthread mutex problems
* Split InfoQuery in two: non-transactional SystemInfoQuery and transactional DatabaseInfoQuery
* Replicable and durable statistics
* Bumped WAL/Snapshot versions
* Initial implementation of the Lamport clock

---------

Co-authored-by: Andreja Tonev <andreja.tonev@memgraph.io>
This commit is contained in:
Gareth Andrew Lloyd 2023-10-05 15:58:39 +01:00 committed by GitHub
parent d71b6a5007
commit 3cc2bc2791
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
116 changed files with 4564 additions and 1661 deletions

View File

@ -20,6 +20,7 @@ add_subdirectory(auth)
add_subdirectory(audit)
add_subdirectory(dbms)
add_subdirectory(flags)
add_subdirectory(distributed)
string(TOLOWER ${CMAKE_BUILD_TYPE} lower_build_type)

View File

@ -208,7 +208,7 @@ class Session {
Version version_;
virtual std::string GetDatabaseName() const = 0;
virtual std::string GetCurrentDB() const = 0;
std::string UUID() const { return session_uuid_; }
private:

View File

@ -208,7 +208,7 @@ State HandleRunV1(TSession &session, const State state, const Marker marker) {
DMG_ASSERT(!session.encoder_buffer_.HasData(), "There should be no data to write in this state");
spdlog::debug("[Run - {}] '{}'", session.GetDatabaseName(), query.ValueString());
spdlog::debug("[Run - {}] '{}'", session.GetCurrentDB(), query.ValueString());
try {
// Interpret can throw.
@ -272,7 +272,7 @@ State HandleRunV4(TSession &session, const State state, const Marker marker) {
return HandleFailure(session, e);
}
spdlog::debug("[Run - {}] '{}'", session.GetDatabaseName(), query.ValueString());
spdlog::debug("[Run - {}] '{}'", session.GetCurrentDB(), query.ValueString());
try {
// Interpret can throw.

View File

@ -60,6 +60,11 @@ class Database {
return storage_->Access(override_isolation_level);
}
std::unique_ptr<storage::Storage::Accessor> UniqueAccess(
std::optional<storage::IsolationLevel> override_isolation_level = {}) {
return storage_->UniqueAccess(override_isolation_level);
}
/**
* @brief Unique storage identified (name)
*

View File

@ -0,0 +1,4 @@
add_library(mg-distributed)
add_library(mg::distributed ALIAS mg-distributed)
target_include_directories(mg-distributed PUBLIC include )
target_sources(mg-distributed PRIVATE lamport_clock.cpp)

View File

@ -0,0 +1,61 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <compare>
#include <cstdint>
#include <numeric>
namespace memgraph::distributed {
// forward declare, for strong timestamps
template <typename Tag>
struct LamportClock;
template <typename Tag>
struct timestamp {
friend std::strong_ordering operator<=>(timestamp const &, timestamp const &) = default;
private:
friend struct LamportClock<Tag>;
explicit timestamp(uint64_t value) : value_{value} {}
uint64_t value_;
};
constexpr struct internal_t {
} internal;
constexpr struct send_t {
} send;
constexpr struct receive_t {
} receive;
template <typename Tag>
struct LamportClock {
using timestamp_t = timestamp<Tag>;
auto get_timestamp(internal_t) -> timestamp_t { return timestamp_t{++internal}; };
auto get_timestamp(send_t) -> timestamp_t { return timestamp_t{++internal}; };
auto get_timestamp(receive_t, timestamp_t received_timestamp) -> timestamp_t {
while (true) {
auto local_current = internal.load(std::memory_order_acquire);
auto next = std::max(received_timestamp.value_, local_current) + 1;
bool res = internal.compare_exchange_weak(local_current, next, std::memory_order_acq_rel);
if (res) return timestamp_t{next};
}
};
private:
std::atomic<uint64_t> internal = 0;
};
} // namespace memgraph::distributed

View File

@ -0,0 +1,11 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "distributed/lamport_clock.hpp"

View File

@ -114,9 +114,9 @@ std::string SessionHL::GetDefaultDB() {
}
#endif
std::string SessionHL::GetDatabaseName() const {
if (!interpreter_.db_acc_) return "";
const auto *db = interpreter_.db_acc_->get();
std::string SessionHL::GetCurrentDB() const {
if (!interpreter_.current_db_.db_acc_) return "";
const auto *db = interpreter_.current_db_.db_acc_->get();
return db->id();
}
@ -127,18 +127,23 @@ std::optional<std::string> SessionHL::GetServerNameForInit() {
bool SessionHL::Authenticate(const std::string &username, const std::string &password) {
bool res = true;
interpreter_.ResetUser();
{
auto locked_auth = auth_->Lock();
if (locked_auth->HasUsers()) {
user_ = locked_auth->Authenticate(username, password);
res = user_.has_value();
if (user_.has_value()) {
interpreter_.SetUser(user_->username());
} else {
res = false;
}
}
}
#ifdef MG_ENTERPRISE
// Start off with the default database
interpreter_.SetCurrentDB(GetDefaultDB());
interpreter_.SetCurrentDB(GetDefaultDB(), false);
#endif
implicit_db_.emplace(GetDatabaseName());
implicit_db_.emplace(GetCurrentDB());
return res;
}
@ -159,7 +164,7 @@ std::map<std::string, memgraph::communication::bolt::Value> SessionHL::Pull(Sess
std::optional<int> n,
std::optional<int> qid) {
// TODO: Update once interpreter can handle non-database queries (db_acc will be nullopt)
auto *db = interpreter_.db_acc_->get();
auto *db = interpreter_.current_db_.db_acc_->get();
try {
TypedValueResultStream<TEncoder> stream(encoder, db->storage());
return DecodeSummary(interpreter_.Pull(&stream, n, qid));
@ -184,14 +189,14 @@ std::pair<std::vector<std::string>, std::optional<int>> SessionHL::Interpret(
#ifdef MG_ENTERPRISE
// TODO: Update once interpreter can handle non-database queries (db_acc will be nullopt)
auto *db = interpreter_.db_acc_->get();
auto *db = interpreter_.current_db_.db_acc_->get();
if (memgraph::license::global_license_checker.IsEnterpriseValidFast()) {
audit_log_->Record(endpoint_.address().to_string(), user_ ? *username : "", query,
memgraph::storage::PropertyValue(params_pv), db->id());
}
#endif
try {
auto result = interpreter_.Prepare(query, params_pv, username, ToQueryExtras(extra), UUID());
auto result = interpreter_.Prepare(query, params_pv, ToQueryExtras(extra));
const std::string db_name = result.db ? *result.db : "";
if (user_ && !AuthChecker::IsUserAuthorized(*user_, result.privileges, db_name)) {
interpreter_.Abort();
@ -230,7 +235,7 @@ void SessionHL::Configure(const std::map<std::string, memgraph::communication::b
throw memgraph::communication::bolt::ClientError("Malformed database name.");
}
db = db_info.ValueString();
const auto &current = GetDatabaseName();
const auto &current = GetCurrentDB();
update = db != current;
if (!in_explicit_db_) implicit_db_.emplace(current); // Still not in an explicit database, save for recovery
in_explicit_db_ = true;
@ -241,14 +246,14 @@ void SessionHL::Configure(const std::map<std::string, memgraph::communication::b
} else {
db = GetDefaultDB();
}
update = db != GetDatabaseName();
update = db != GetCurrentDB();
in_explicit_db_ = false;
}
// Check if the underlying database needs to be updated
if (update) {
MultiDatabaseAuth(user_, db);
interpreter_.SetCurrentDB(db);
interpreter_.SetCurrentDB(db, in_explicit_db_);
}
#endif
}
@ -288,7 +293,7 @@ SessionHL::~SessionHL() {
std::map<std::string, memgraph::communication::bolt::Value> SessionHL::DecodeSummary(
const std::map<std::string, memgraph::query::TypedValue> &summary) {
// TODO: Update once interpreter can handle non-database queries (db_acc will be nullopt)
auto *db = interpreter_.db_acc_->get();
auto *db = interpreter_.current_db_.db_acc_->get();
std::map<std::string, memgraph::communication::bolt::Value> decoded_summary;
for (const auto &kv : summary) {
auto maybe_value = ToBoltValue(kv.second, *db->storage(), memgraph::storage::View::NEW);

View File

@ -68,7 +68,7 @@ class SessionHL final : public memgraph::communication::bolt::Session<memgraph::
std::optional<std::string> GetServerNameForInit() override;
std::string GetDatabaseName() const override;
std::string GetCurrentDB() const override;
private:
std::map<std::string, memgraph::communication::bolt::Value> DecodeSummary(

View File

@ -360,7 +360,7 @@ int main(int argc, char **argv) {
auto db_acc_opt = db_gatekeeper.access();
MG_ASSERT(db_acc_opt, "Failed to access the main database");
auto &db_acc = *db_acc_opt;
memgraph::query::InterpreterContext interpreter_context_(interp_config, nullptr, auth_handler.get(),
memgraph::query::InterpreterContext interpreter_context_(interp_config, &db_gatekeeper, auth_handler.get(),
auth_checker.get());
#endif
MG_ASSERT(db_acc, "Failed to access the main database");

View File

@ -38,7 +38,8 @@ set(mg_query_sources
graph.cpp
db_accessor.cpp
auth_query_handler.cpp
interpreter_context.cpp)
interpreter_context.cpp
)
add_library(mg-query STATIC ${mg_query_sources})
target_include_directories(mg-query PUBLIC ${CMAKE_SOURCE_DIR}/include)

View File

@ -533,7 +533,7 @@ class DbAccessor final {
void AdvanceCommand() { accessor_->AdvanceCommand(); }
utils::BasicResult<storage::StorageDataManipulationError, void> Commit() { return accessor_->Commit(); }
utils::BasicResult<storage::StorageManipulationError, void> Commit() { return accessor_->Commit(); }
void Abort() { accessor_->Abort(); }
@ -554,20 +554,12 @@ class DbAccessor final {
return accessor_->GetIndexStats(label, property);
}
std::vector<std::pair<storage::LabelId, storage::PropertyId>> ClearLabelPropertyIndexStats() {
return accessor_->ClearLabelPropertyIndexStats();
}
std::vector<storage::LabelId> ClearLabelIndexStats() { return accessor_->ClearLabelIndexStats(); }
std::vector<std::pair<storage::LabelId, storage::PropertyId>> DeleteLabelPropertyIndexStats(
const std::span<std::string> labels) {
return accessor_->DeleteLabelPropertyIndexStats(labels);
const storage::LabelId &label) {
return accessor_->DeleteLabelPropertyIndexStats(label);
}
std::vector<storage::LabelId> DeleteLabelIndexStats(const std::span<std::string> labels) {
return accessor_->DeleteLabelIndexStats(labels);
}
bool DeleteLabelIndexStats(const storage::LabelId &label) { return accessor_->DeleteLabelIndexStats(label); }
void SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats) {
accessor_->SetIndexStats(label, stats);
@ -602,6 +594,44 @@ class DbAccessor final {
storage::ConstraintsInfo ListAllConstraints() const { return accessor_->ListAllConstraints(); }
const std::string &id() const { return accessor_->id(); }
utils::BasicResult<storage::StorageIndexDefinitionError, void> CreateIndex(storage::LabelId label) {
return accessor_->CreateIndex(label);
}
utils::BasicResult<storage::StorageIndexDefinitionError, void> CreateIndex(storage::LabelId label,
storage::PropertyId property) {
return accessor_->CreateIndex(label, property);
}
utils::BasicResult<storage::StorageIndexDefinitionError, void> DropIndex(storage::LabelId label) {
return accessor_->DropIndex(label);
}
utils::BasicResult<storage::StorageIndexDefinitionError, void> DropIndex(storage::LabelId label,
storage::PropertyId property) {
return accessor_->DropIndex(label, property);
}
utils::BasicResult<storage::StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
storage::LabelId label, storage::PropertyId property) {
return accessor_->CreateExistenceConstraint(label, property);
}
utils::BasicResult<storage::StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
storage::LabelId label, storage::PropertyId property) {
return accessor_->DropExistenceConstraint(label, property);
}
utils::BasicResult<storage::StorageUniqueConstraintDefinitionError, storage::UniqueConstraints::CreationStatus>
CreateUniqueConstraint(storage::LabelId label, const std::set<storage::PropertyId> &properties) {
return accessor_->CreateUniqueConstraint(label, properties);
}
storage::UniqueConstraints::DeletionStatus DropUniqueConstraint(storage::LabelId label,
const std::set<storage::PropertyId> &properties) {
return accessor_->DropUniqueConstraint(label, properties);
}
};
class SubgraphDbAccessor final {

View File

@ -223,7 +223,11 @@ constexpr utils::TypeInfo query::Unwind::kType{utils::TypeId::AST_UNWIND, "Unwin
constexpr utils::TypeInfo query::AuthQuery::kType{utils::TypeId::AST_AUTH_QUERY, "AuthQuery", &query::Query::kType};
constexpr utils::TypeInfo query::InfoQuery::kType{utils::TypeId::AST_INFO_QUERY, "InfoQuery", &query::Query::kType};
constexpr utils::TypeInfo query::DatabaseInfoQuery::kType{utils::TypeId::AST_DATABASE_INFO_QUERY, "DatabaseInfoQuery",
&query::Query::kType};
constexpr utils::TypeInfo query::SystemInfoQuery::kType{utils::TypeId::AST_SYSTEM_INFO_QUERY, "SystemInfoQuery",
&query::Query::kType};
constexpr utils::TypeInfo query::Constraint::kType{utils::TypeId::AST_CONSTRAINT, "Constraint", nullptr};

View File

@ -2897,19 +2897,37 @@ const std::vector<AuthQuery::Privilege> kPrivilegesAll = {AuthQuery::Privilege::
AuthQuery::Privilege::MULTI_DATABASE_EDIT,
AuthQuery::Privilege::MULTI_DATABASE_USE};
class InfoQuery : public memgraph::query::Query {
class DatabaseInfoQuery : public memgraph::query::Query {
public:
static const utils::TypeInfo kType;
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
enum class InfoType { STORAGE, INDEX, CONSTRAINT, BUILD };
enum class InfoType { INDEX, CONSTRAINT };
DEFVISITABLE(QueryVisitor<void>);
memgraph::query::InfoQuery::InfoType info_type_;
memgraph::query::DatabaseInfoQuery::InfoType info_type_;
InfoQuery *Clone(AstStorage *storage) const override {
InfoQuery *object = storage->Create<InfoQuery>();
DatabaseInfoQuery *Clone(AstStorage *storage) const override {
DatabaseInfoQuery *object = storage->Create<DatabaseInfoQuery>();
object->info_type_ = info_type_;
return object;
}
};
class SystemInfoQuery : public memgraph::query::Query {
public:
static const utils::TypeInfo kType;
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
enum class InfoType { STORAGE, BUILD };
DEFVISITABLE(QueryVisitor<void>);
memgraph::query::SystemInfoQuery::InfoType info_type_;
SystemInfoQuery *Clone(AstStorage *storage) const override {
SystemInfoQuery *object = storage->Create<SystemInfoQuery>();
object->info_type_ = info_type_;
return object;
}

View File

@ -82,7 +82,8 @@ class AuthQuery;
class ExplainQuery;
class ProfileQuery;
class IndexQuery;
class InfoQuery;
class DatabaseInfoQuery;
class SystemInfoQuery;
class ConstraintQuery;
class RegexMatch;
class DumpQuery;
@ -140,10 +141,10 @@ class ExpressionVisitor
template <class TResult>
class QueryVisitor
: public utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, AuthQuery, InfoQuery,
ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery, FreeMemoryQuery, TriggerQuery,
IsolationLevelQuery, CreateSnapshotQuery, StreamQuery, SettingQuery, VersionQuery,
ShowConfigQuery, TransactionQueueQuery, StorageModeQuery, AnalyzeGraphQuery,
MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery> {};
: public utils::Visitor<TResult, CypherQuery, ExplainQuery, ProfileQuery, IndexQuery, AuthQuery, DatabaseInfoQuery,
SystemInfoQuery, ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery,
FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, StreamQuery,
SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, StorageModeQuery,
AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery> {};
} // namespace memgraph::query

View File

@ -112,25 +112,36 @@ antlrcpp::Any CypherMainVisitor::visitProfileQuery(MemgraphCypher::ProfileQueryC
return profile_query;
}
antlrcpp::Any CypherMainVisitor::visitInfoQuery(MemgraphCypher::InfoQueryContext *ctx) {
MG_ASSERT(ctx->children.size() == 2, "InfoQuery should have exactly two children!");
auto *info_query = storage_->Create<InfoQuery>();
antlrcpp::Any CypherMainVisitor::visitDatabaseInfoQuery(MemgraphCypher::DatabaseInfoQueryContext *ctx) {
MG_ASSERT(ctx->children.size() == 2, "DatabaseInfoQuery should have exactly two children!");
auto *info_query = storage_->Create<DatabaseInfoQuery>();
query_ = info_query;
if (ctx->indexInfo()) {
info_query->info_type_ = DatabaseInfoQuery::InfoType::INDEX;
return info_query;
}
if (ctx->constraintInfo()) {
info_query->info_type_ = DatabaseInfoQuery::InfoType::CONSTRAINT;
return info_query;
}
// Should never get here
throw utils::NotYetImplemented("Database info query: '{}'", ctx->getText());
}
antlrcpp::Any CypherMainVisitor::visitSystemInfoQuery(MemgraphCypher::SystemInfoQueryContext *ctx) {
MG_ASSERT(ctx->children.size() == 2, "SystemInfoQuery should have exactly two children!");
auto *info_query = storage_->Create<SystemInfoQuery>();
query_ = info_query;
if (ctx->storageInfo()) {
info_query->info_type_ = InfoQuery::InfoType::STORAGE;
info_query->info_type_ = SystemInfoQuery::InfoType::STORAGE;
return info_query;
} else if (ctx->indexInfo()) {
info_query->info_type_ = InfoQuery::InfoType::INDEX;
return info_query;
} else if (ctx->constraintInfo()) {
info_query->info_type_ = InfoQuery::InfoType::CONSTRAINT;
return info_query;
} else if (ctx->buildInfo()) {
info_query->info_type_ = InfoQuery::InfoType::BUILD;
return info_query;
} else {
throw utils::NotYetImplemented("Info query: '{}'", ctx->getText());
}
if (ctx->buildInfo()) {
info_query->info_type_ = SystemInfoQuery::InfoType::BUILD;
return info_query;
}
// Should never get here
throw utils::NotYetImplemented("System info query: '{}'", ctx->getText());
}
antlrcpp::Any CypherMainVisitor::visitConstraintQuery(MemgraphCypher::ConstraintQueryContext *ctx) {

View File

@ -157,9 +157,14 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
antlrcpp::Any visitProfileQuery(MemgraphCypher::ProfileQueryContext *ctx) override;
/**
* @return InfoQuery*
* @return DatabaseInfoQuery*
*/
antlrcpp::Any visitInfoQuery(MemgraphCypher::InfoQueryContext *ctx) override;
antlrcpp::Any visitDatabaseInfoQuery(MemgraphCypher::DatabaseInfoQueryContext *ctx) override;
/**
* @return SystemInfoQuery*
*/
antlrcpp::Any visitSystemInfoQuery(MemgraphCypher::SystemInfoQueryContext *ctx) override;
/**
* @return Constraint

View File

@ -27,7 +27,8 @@ query : cypherQuery
| indexQuery
| explainQuery
| profileQuery
| infoQuery
| databaseInfoQuery
| systemInfoQuery
| constraintQuery
;
@ -48,7 +49,9 @@ constraintInfo : CONSTRAINT INFO ;
buildInfo : BUILD INFO ;
infoQuery : SHOW ( storageInfo | indexInfo | constraintInfo | buildInfo) ;
databaseInfoQuery : SHOW ( indexInfo | constraintInfo ) ;
systemInfoQuery : SHOW ( storageInfo | buildInfo ) ;
explainQuery : EXPLAIN cypherQuery ;

View File

@ -128,7 +128,8 @@ query : cypherQuery
| indexQuery
| explainQuery
| profileQuery
| infoQuery
| databaseInfoQuery
| systemInfoQuery
| constraintQuery
| authQuery
| dumpQuery

View File

@ -35,18 +35,14 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis
void Visit(ProfileQuery &query) override { query.cypher_query_->Accept(dynamic_cast<QueryVisitor &>(*this)); }
void Visit(InfoQuery &info_query) override {
void Visit(DatabaseInfoQuery &info_query) override {
switch (info_query.info_type_) {
case InfoQuery::InfoType::INDEX:
case DatabaseInfoQuery::InfoType::INDEX:
// TODO: This should be INDEX | STATS, but we don't have support for
// *or* with privileges.
AddPrivilege(AuthQuery::Privilege::INDEX);
break;
case InfoQuery::InfoType::STORAGE:
case InfoQuery::InfoType::BUILD:
AddPrivilege(AuthQuery::Privilege::STATS);
break;
case InfoQuery::InfoType::CONSTRAINT:
case DatabaseInfoQuery::InfoType::CONSTRAINT:
// TODO: This should be CONSTRAINT | STATS, but we don't have support
// for *or* with privileges.
AddPrivilege(AuthQuery::Privilege::CONSTRAINT);
@ -54,6 +50,15 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis
}
}
void Visit(SystemInfoQuery &info_query) override {
switch (info_query.info_type_) {
case SystemInfoQuery::InfoType::STORAGE:
case SystemInfoQuery::InfoType::BUILD:
AddPrivilege(AuthQuery::Privilege::STATS);
break;
}
}
void Visit(ConstraintQuery &constraint_query) override { AddPrivilege(AuthQuery::Privilege::CONSTRAINT); }
void Visit(CypherQuery &query) override {

File diff suppressed because it is too large Load Diff

View File

@ -139,6 +139,34 @@ struct QueryExtras {
std::optional<int64_t> tx_timeout;
};
struct CurrentDB {
CurrentDB() = default; // TODO: remove, we should always have an implicit default obtainable from somewhere
// ATM: it is provided by the DatabaseAccess
// future: should be a name + ptr to dbms_handler, lazy fetch when needed
explicit CurrentDB(memgraph::dbms::DatabaseAccess db_acc) : db_acc_{std::move(db_acc)} {}
CurrentDB(CurrentDB const &) = delete;
CurrentDB &operator=(CurrentDB const &) = delete;
void SetupDatabaseTransaction(std::optional<storage::IsolationLevel> override_isolation_level, bool could_commit,
bool unique = false);
void CleanupDBTransaction(bool abort);
void SetCurrentDB(memgraph::dbms::DatabaseAccess new_db, bool in_explicit_db) {
// do we lock here?
db_acc_ = std::move(new_db);
in_explicit_db_ = in_explicit_db;
}
// TODO: don't provide explicitly via constructor, instead have a lazy way of getting the current/default
// DatabaseAccess
// hence, explict bolt "use DB" in metadata wouldn't necessarily get access unless query required it.
std::optional<memgraph::dbms::DatabaseAccess> db_acc_; // Current db (TODO: expand to support multiple)
std::unique_ptr<storage::Storage::Accessor> db_transactional_accessor_;
std::optional<DbAccessor> execution_db_accessor_;
std::optional<TriggerContextCollector> trigger_context_collector_;
bool in_explicit_db_{false};
};
class Interpreter final {
public:
Interpreter(InterpreterContext *interpreter_context);
@ -158,16 +186,14 @@ class Interpreter final {
std::optional<std::string> username_;
bool in_explicit_transaction_{false};
bool in_explicit_db_{false};
CurrentDB current_db_;
bool expect_rollback_{false};
std::shared_ptr<utils::AsyncTimer> explicit_transaction_timer_{};
std::shared_ptr<utils::AsyncTimer> current_timeout_timer_{};
std::optional<std::map<std::string, storage::PropertyValue>> metadata_{}; //!< User defined transaction metadata
std::optional<memgraph::dbms::DatabaseAccess> db_acc_; // Current db (TODO: expand to support multiple)
#ifdef MG_ENTERPRISE
void SetCurrentDB(std::string_view db_name);
void SetCurrentDB(memgraph::dbms::DatabaseAccess new_db);
void SetCurrentDB(std::string_view db_name, bool explicit_db);
void OnChangeCB(auto cb) { on_change_.emplace(cb); }
#endif
@ -179,9 +205,9 @@ class Interpreter final {
*
* @throw query::QueryException
*/
PrepareResult Prepare(const std::string &query, const std::map<std::string, storage::PropertyValue> &params,
const std::string *username, QueryExtras const &extras = {},
const std::string &session_uuid = {});
Interpreter::PrepareResult Prepare(const std::string &query,
const std::map<std::string, storage::PropertyValue> &params,
QueryExtras const &extras);
/**
* Execute the last prepared query and stream *all* of the results into the
@ -243,27 +269,30 @@ class Interpreter final {
*/
void Abort();
std::atomic<TransactionStatus> transaction_status_{TransactionStatus::IDLE};
std::atomic<TransactionStatus> transaction_status_{TransactionStatus::IDLE}; // Tie to current_transaction_
std::optional<uint64_t> current_transaction_;
void ResetUser();
void SetUser(std::string_view username);
private:
struct QueryExecution {
std::optional<PreparedQuery> prepared_query;
std::variant<utils::MonotonicBufferResource, utils::PoolResource> execution_memory;
utils::ResourceWithOutOfMemoryException execution_memory_with_exception;
std::optional<PreparedQuery> prepared_query;
std::map<std::string, TypedValue> summary;
std::vector<Notification> notifications;
explicit QueryExecution(utils::MonotonicBufferResource monotonic_memory)
: execution_memory(std::move(monotonic_memory)) {
std::visit(
[&](auto &memory_resource) {
execution_memory_with_exception = utils::ResourceWithOutOfMemoryException(&memory_resource);
},
execution_memory);
};
static auto Create(std::variant<utils::MonotonicBufferResource, utils::PoolResource> memory_resource,
std::optional<PreparedQuery> prepared_query = std::nullopt) -> std::unique_ptr<QueryExecution> {
return std::make_unique<QueryExecution>(std::move(memory_resource), std::move(prepared_query));
}
explicit QueryExecution(utils::PoolResource pool_resource) : execution_memory(std::move(pool_resource)) {
explicit QueryExecution(std::variant<utils::MonotonicBufferResource, utils::PoolResource> memory_resource,
std::optional<PreparedQuery> prepared_query)
: execution_memory(std::move(memory_resource)), prepared_query{std::move(prepared_query)} {
std::visit(
[&](auto &memory_resource) {
execution_memory_with_exception = utils::ResourceWithOutOfMemoryException(&memory_resource);
@ -311,12 +340,6 @@ class Interpreter final {
InterpreterContext *interpreter_context_;
// 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::Storage::Accessor> db_accessor_;
std::optional<DbAccessor> execution_db_accessor_;
std::optional<TriggerContextCollector> trigger_context_collector_;
std::optional<FrameChangeCollector> frame_change_collector_;
std::optional<storage::IsolationLevel> interpreter_isolation_level;
@ -334,22 +357,8 @@ class Interpreter final {
}
std::optional<std::function<void(std::string_view)>> on_change_{};
};
class TransactionQueueQueryHandler {
public:
TransactionQueueQueryHandler() = default;
virtual ~TransactionQueueQueryHandler() = default;
TransactionQueueQueryHandler(const TransactionQueueQueryHandler &) = default;
TransactionQueueQueryHandler &operator=(const TransactionQueueQueryHandler &) = default;
TransactionQueueQueryHandler(TransactionQueueQueryHandler &&) = default;
TransactionQueueQueryHandler &operator=(TransactionQueueQueryHandler &&) = default;
static std::vector<std::vector<TypedValue>> ShowTransactions(
const std::unordered_set<Interpreter *> &interpreters, const std::optional<std::string> &username,
bool hasTransactionManagementPrivilege, std::optional<memgraph::dbms::DatabaseAccess> &filter_db_acc);
void SetupInterpreterTransaction(const QueryExtras &extras);
void SetupDatabaseTransaction(bool couldCommit, bool unique = false);
};
template <typename TStream>
@ -411,7 +420,7 @@ std::map<std::string, TypedValue> Interpreter::Pull(TStream *result_stream, std:
// The only cases in which we have nothing to do are those where
// we're either in an explicit transaction or the query is such that
// a transaction wasn't started on a call to `Prepare()`.
MG_ASSERT(in_explicit_transaction_ || !db_accessor_);
MG_ASSERT(in_explicit_transaction_ || !current_db_.db_transactional_accessor_);
break;
}
// As the transaction is done we can clear all the executions

View File

@ -13,15 +13,15 @@
#include "query/interpreter.hpp"
namespace memgraph::query {
std::vector<std::vector<TypedValue>> InterpreterContext::KillTransactions(
std::vector<std::vector<TypedValue>> InterpreterContext::TerminateTransactions(
std::vector<std::string> maybe_kill_transaction_ids, const std::optional<std::string> &username,
bool hasTransactionManagementPrivilege, Interpreter &calling_interpreter) {
std::function<bool(std::string const &)> privilege_checker) {
auto not_found_midpoint = maybe_kill_transaction_ids.end();
// Multiple simultaneous TERMINATE TRANSACTIONS aren't allowed
// TERMINATE and SHOW TRANSACTIONS are mutually exclusive
interpreters.WithLock([&not_found_midpoint, &maybe_kill_transaction_ids, username, hasTransactionManagementPrivilege,
filter_db_acc = &calling_interpreter.db_acc_](const auto &interpreters) {
interpreters.WithLock([&not_found_midpoint, &maybe_kill_transaction_ids, username,
privilege_checker = std::move(privilege_checker)](const auto &interpreters) {
for (Interpreter *interpreter : interpreters) {
TransactionStatus alive_status = TransactionStatus::ACTIVE;
// if it is just checking kill, commit and abort should wait for the end of the check
@ -38,7 +38,6 @@ std::vector<std::vector<TypedValue>> InterpreterContext::KillTransactions(
interpreter->transaction_status_.store(TransactionStatus::ACTIVE, std::memory_order_release);
}
});
if (interpreter->db_acc_ != *filter_db_acc) continue;
std::optional<uint64_t> intr_trans = interpreter->GetTransactionId();
if (!intr_trans.has_value()) continue;
@ -49,7 +48,11 @@ std::vector<std::vector<TypedValue>> InterpreterContext::KillTransactions(
// update the maybe_kill_transaction_ids (partitioning not found + killed)
--not_found_midpoint;
std::iter_swap(it, not_found_midpoint);
if (interpreter->username_ == username || hasTransactionManagementPrivilege) {
auto get_interpreter_db_name = [&]() -> std::string const & {
static std::string all;
return interpreter->current_db_.db_acc_ ? interpreter->current_db_.db_acc_->get()->id() : all;
};
if (interpreter->username_ == username || privilege_checker(get_interpreter_db_name())) {
killed = true; // Note: this is used by the above `clean_status` (OnScopeExit)
spdlog::warn("Transaction {} successfully killed", transaction_id);
} else {

View File

@ -11,6 +11,8 @@
#pragma once
#include <atomic>
#include <cstdint>
#include <optional>
#include <string>
#include <unordered_set>
@ -34,6 +36,8 @@ class Database;
namespace memgraph::query {
constexpr uint64_t kInterpreterTransactionInitialId = 1ULL << 63U;
class AuthQueryHandler;
class AuthChecker;
class Interpreter;
@ -72,14 +76,20 @@ struct InterpreterContext {
// TODO: Have a way to read the current database
memgraph::utils::Synchronized<std::unordered_set<Interpreter *>, memgraph::utils::SpinLock> interpreters;
struct {
auto next() -> uint64_t { return transaction_id++; }
private:
std::atomic<uint64_t> transaction_id = kInterpreterTransactionInitialId;
} id_handler;
/// Function that is used to tell all active interpreters that they should stop
/// their ongoing execution.
void Shutdown() { is_shutting_down.store(true, std::memory_order_release); }
std::vector<std::vector<TypedValue>> KillTransactions(std::vector<std::string> maybe_kill_transaction_ids,
const std::optional<std::string> &username,
bool hasTransactionManagementPrivilege,
Interpreter &calling_interpreter);
std::vector<std::vector<TypedValue>> TerminateTransactions(
std::vector<std::string> maybe_kill_transaction_ids, const std::optional<std::string> &username,
std::function<bool(std::string const &)> privilege_checker);
};
} // namespace memgraph::query

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -21,7 +21,7 @@
namespace memgraph::query::plan {
PRE_VISIT(CreateNode, RWType::W, true)
PRE_VISIT(CreateExpand, RWType::R, true)
PRE_VISIT(CreateExpand, RWType::R, true) // ?? RWType::RW
PRE_VISIT(Delete, RWType::W, true)
PRE_VISIT(SetProperty, RWType::W, true)

View File

@ -497,7 +497,7 @@ Streams::StreamsMap::iterator Streams::CreateConsumer(StreamsMap &map, const std
#ifdef MG_ENTERPRISE
interpreter->OnChangeCB([](auto) { return false; }); // Disable database change
#endif
auto accessor = interpreter->db_acc_->get()->Access();
auto accessor = interpreter->current_db_.db_acc_->get()->Access();
// register new interpreter into interpreter_context
interpreter_context->interpreters->insert(interpreter.get());
utils::OnScopeExit interpreter_cleanup{
@ -527,7 +527,7 @@ Streams::StreamsMap::iterator Streams::CreateConsumer(StreamsMap &map, const std
std::string query{query_value.ValueString()};
spdlog::trace("Executing query '{}' in stream '{}'", query, stream_name);
auto prepare_result =
interpreter->Prepare(query, params_prop.IsNull() ? empty_parameters : params_prop.ValueMap(), nullptr);
interpreter->Prepare(query, params_prop.IsNull() ? empty_parameters : params_prop.ValueMap(), {});
if (!interpreter_context->auth_checker->IsUserAuthorized(owner, prepare_result.privileges, "")) {
throw StreamsException{
"Couldn't execute query '{}' for stream '{}' because the owner is not authorized to execute the "

View File

@ -320,8 +320,9 @@ DiskStorage::~DiskStorage() {
kvstore_->options_.comparator = nullptr;
}
DiskStorage::DiskAccessor::DiskAccessor(DiskStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode)
: Accessor(storage, isolation_level, storage_mode), config_(storage->config_.items) {
DiskStorage::DiskAccessor::DiskAccessor(auto tag, DiskStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode)
: Accessor(tag, storage, isolation_level, storage_mode), config_(storage->config_.items) {
rocksdb::WriteOptions write_options;
auto txOptions = rocksdb::TransactionOptions{.set_snapshot = true};
disk_transaction_ = storage->kvstore_->db_->BeginTransaction(write_options, txOptions);
@ -1014,7 +1015,7 @@ void DiskStorage::SetEdgeImportMode(EdgeImportMode edge_import_status) {
}
EdgeImportMode DiskStorage::GetEdgeImportMode() const {
std::shared_lock<utils::RWLock> storage_guard_(main_lock_);
std::shared_lock storage_guard_(main_lock_);
return edge_import_status_;
}
@ -1348,24 +1349,24 @@ bool DiskStorage::DiskAccessor::DeleteEdgeFromDisk(const std::string &edge) {
return true;
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void>
[[nodiscard]] utils::BasicResult<StorageManipulationError, void>
DiskStorage::DiskAccessor::CheckVertexConstraintsBeforeCommit(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage) const {
if (auto existence_constraint_validation_result = storage_->constraints_.existence_constraints_->Validate(vertex);
existence_constraint_validation_result.has_value()) {
return StorageDataManipulationError{existence_constraint_validation_result.value()};
return StorageManipulationError{existence_constraint_validation_result.value()};
}
auto *disk_unique_constraints =
static_cast<DiskUniqueConstraints *>(storage_->constraints_.unique_constraints_.get());
if (auto unique_constraint_validation_result = disk_unique_constraints->Validate(vertex, unique_storage);
unique_constraint_validation_result.has_value()) {
return StorageDataManipulationError{unique_constraint_validation_result.value()};
return StorageManipulationError{unique_constraint_validation_result.value()};
}
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::FlushVertices(
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::FlushVertices(
const auto &vertex_acc, std::vector<std::vector<PropertyValue>> &unique_storage) {
auto *disk_unique_constraints =
static_cast<DiskUniqueConstraints *>(storage_->constraints_.unique_constraints_.get());
@ -1388,26 +1389,25 @@ DiskStorage::DiskAccessor::CheckVertexConstraintsBeforeCommit(
/// NOTE: this deletion has to come before writing, otherwise RocksDB thinks that all entries are deleted
if (auto maybe_old_disk_key = utils::GetOldDiskKeyOrNull(vertex.delta); maybe_old_disk_key.has_value()) {
if (!DeleteVertexFromDisk(maybe_old_disk_key.value())) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
}
if (!WriteVertexToDisk(vertex)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
if (!disk_unique_constraints->SyncVertexToUniqueConstraintsStorage(vertex, *commit_timestamp_) ||
!disk_label_index->SyncVertexToLabelIndexStorage(vertex, *commit_timestamp_) ||
!disk_label_property_index->SyncVertexToLabelPropertyIndexStorage(vertex, *commit_timestamp_)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
}
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void>
DiskStorage::DiskAccessor::ClearDanglingVertices() {
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::ClearDanglingVertices() {
auto *disk_unique_constraints =
static_cast<DiskUniqueConstraints *>(storage_->constraints_.unique_constraints_.get());
auto *disk_label_index = static_cast<DiskLabelIndex *>(storage_->indices_.label_index_.get());
@ -1419,12 +1419,12 @@ DiskStorage::DiskAccessor::ClearDanglingVertices() {
!disk_label_index->DeleteVerticesWithRemovedIndexingLabel(transaction_.start_timestamp, *commit_timestamp_) ||
!disk_label_property_index->DeleteVerticesWithRemovedIndexingLabel(transaction_.start_timestamp,
*commit_timestamp_)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::FlushIndexCache() {
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::FlushIndexCache() {
std::vector<std::vector<PropertyValue>> unique_storage;
for (const auto &vec : index_storage_) {
@ -1436,7 +1436,7 @@ DiskStorage::DiskAccessor::ClearDanglingVertices() {
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::FlushDeletedVertices() {
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::FlushDeletedVertices() {
auto *disk_unique_constraints =
static_cast<DiskUniqueConstraints *>(storage_->constraints_.unique_constraints_.get());
auto *disk_label_index = static_cast<DiskLabelIndex *>(storage_->indices_.label_index_.get());
@ -1448,23 +1448,23 @@ DiskStorage::DiskAccessor::ClearDanglingVertices() {
!disk_unique_constraints->ClearDeletedVertex(vertex_gid, *commit_timestamp_) ||
!disk_label_index->ClearDeletedVertex(vertex_gid, *commit_timestamp_) ||
!disk_label_property_index->ClearDeletedVertex(vertex_gid, *commit_timestamp_)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
}
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::FlushDeletedEdges() {
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::FlushDeletedEdges() {
for (const auto &edge_to_delete : edges_to_delete_) {
if (!DeleteEdgeFromDisk(edge_to_delete)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
}
return {};
}
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::FlushModifiedEdges(
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::FlushModifiedEdges(
const auto &edge_acc) {
for (const auto &modified_edge : transaction_.modified_edges_) {
const storage::Gid &gid = modified_edge.first;
@ -1476,21 +1476,21 @@ DiskStorage::DiskAccessor::ClearDanglingVertices() {
/// If the object was created then flush it, otherwise since properties on edges are false
/// edge wasn't modified for sure.
if (action == Delta::Action::DELETE_OBJECT && !WriteEdgeToDisk(ser_edge_key, "")) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
} else {
// If the delta is DELETE_OBJECT, the edge is just created so there is nothing to delete.
// If the edge was deserialized, only properties can be modified -> key stays the same as when deserialized
// so we can delete it.
if (action == Delta::Action::DELETE_DESERIALIZED_OBJECT && !DeleteEdgeFromDisk(ser_edge_key)) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
const auto &edge = edge_acc.find(gid);
MG_ASSERT(edge != edge_acc.end(),
"Database in invalid state, commit not possible! Please restart your DB and start the import again.");
if (!WriteEdgeToDisk(ser_edge_key, utils::SerializeProperties(edge->properties))) {
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
}
}
@ -1545,7 +1545,7 @@ DiskStorage::CheckExistingVerticesBeforeCreatingUniqueConstraint(LabelId label,
}
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor::Commit(
utils::BasicResult<StorageManipulationError, void> DiskStorage::DiskAccessor::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!");
@ -1553,10 +1553,84 @@ utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor
auto *disk_storage = static_cast<DiskStorage *>(storage_);
bool edge_import_mode_active = disk_storage->edge_import_status_ == EdgeImportMode::ACTIVE;
if (transaction_.deltas.use().empty() ||
(!edge_import_mode_active &&
std::all_of(transaction_.deltas.use().begin(), transaction_.deltas.use().end(),
[](const Delta &delta) { return delta.action == Delta::Action::DELETE_DESERIALIZED_OBJECT; }))) {
if (!transaction_.md_deltas.empty()) {
// This is usually done by the MVCC, but it does not handle the metadata deltas
transaction_.EnsureCommitTimestampExists();
std::unique_lock<utils::SpinLock> engine_guard(storage_->engine_lock_);
commit_timestamp_.emplace(disk_storage->CommitTimestamp(desired_commit_timestamp));
transaction_.commit_timestamp->store(*commit_timestamp_, std::memory_order_release);
for (const auto &md_delta : transaction_.md_deltas) {
switch (md_delta.action) {
case MetadataDelta::Action::LABEL_INDEX_CREATE: {
if (!disk_storage->PersistLabelIndexCreation(md_delta.label)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_CREATE: {
const auto &info = md_delta.label_property;
if (!disk_storage->PersistLabelPropertyIndexAndExistenceConstraintCreation(info.label, info.property,
label_property_index_str)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::LABEL_INDEX_DROP: {
if (!disk_storage->PersistLabelIndexDeletion(md_delta.label)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_DROP: {
const auto &info = md_delta.label_property;
if (!disk_storage->PersistLabelPropertyIndexAndExistenceConstraintDeletion(info.label, info.property,
label_property_index_str)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::LABEL_INDEX_STATS_SET: {
throw utils::NotYetImplemented("SetIndexStats(stats) is not implemented for DiskStorage.");
} break;
case MetadataDelta::Action::LABEL_INDEX_STATS_CLEAR: {
throw utils::NotYetImplemented("ClearIndexStats(stats) is not implemented for DiskStorage.");
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_STATS_SET: {
throw utils::NotYetImplemented("SetIndexStats(stats) is not implemented for DiskStorage.");
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_STATS_CLEAR: {
throw utils::NotYetImplemented("ClearIndexStats(stats) is not implemented for DiskStorage.");
} break;
case MetadataDelta::Action::EXISTENCE_CONSTRAINT_CREATE: {
const auto &info = md_delta.label_property;
if (!disk_storage->PersistLabelPropertyIndexAndExistenceConstraintCreation(info.label, info.property,
existence_constraints_str)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::EXISTENCE_CONSTRAINT_DROP: {
const auto &info = md_delta.label_property;
if (!disk_storage->PersistLabelPropertyIndexAndExistenceConstraintDeletion(info.label, info.property,
existence_constraints_str)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::UNIQUE_CONSTRAINT_CREATE: {
const auto &info = md_delta.label_properties;
if (!disk_storage->PersistUniqueConstraintCreation(info.label, info.properties)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
case MetadataDelta::Action::UNIQUE_CONSTRAINT_DROP: {
const auto &info = md_delta.label_properties;
if (!disk_storage->PersistUniqueConstraintDeletion(info.label, info.properties)) {
return StorageManipulationError{PersistenceError{}};
}
} break;
}
}
} else if (transaction_.deltas.use().empty() ||
(!edge_import_mode_active &&
std::all_of(transaction_.deltas.use().begin(), transaction_.deltas.use().end(), [](const Delta &delta) {
return delta.action == Delta::Action::DELETE_DESERIALIZED_OBJECT;
}))) {
} else {
std::unique_lock<utils::SpinLock> engine_guard(storage_->engine_lock_);
commit_timestamp_.emplace(disk_storage->CommitTimestamp(desired_commit_timestamp));
@ -1610,7 +1684,7 @@ utils::BasicResult<StorageDataManipulationError, void> DiskStorage::DiskAccessor
disk_transaction_ = nullptr;
if (!commitStatus.ok()) {
spdlog::error("rocksdb: Commit failed with status {}", commitStatus.ToString());
return StorageDataManipulationError{SerializationError{}};
return StorageManipulationError{SerializationError{}};
}
spdlog::trace("rocksdb: Commit successful");
@ -1755,156 +1829,122 @@ void DiskStorage::DiskAccessor::FinalizeTransaction() {
}
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::CreateIndex(
LabelId label, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *disk_label_index = static_cast<DiskLabelIndex *>(indices_.label_index_.get());
if (!disk_label_index->CreateIndex(label, SerializeVerticesForLabelIndex(label))) {
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::CreateIndex(LabelId label) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_label_index = static_cast<DiskLabelIndex *>(on_disk->indices_.label_index_.get());
if (!disk_label_index->CreateIndex(label, on_disk->SerializeVerticesForLabelIndex(label))) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
if (!PersistLabelIndexCreation(label)) {
return StorageIndexDefinitionError{IndexPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_create, label);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelIndices);
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::CreateIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *disk_label_property_index = static_cast<DiskLabelPropertyIndex *>(indices_.label_property_index_.get());
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::CreateIndex(LabelId label,
PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_label_property_index =
static_cast<DiskLabelPropertyIndex *>(on_disk->indices_.label_property_index_.get());
if (!disk_label_property_index->CreateIndex(label, property,
SerializeVerticesForLabelPropertyIndex(label, property))) {
on_disk->SerializeVerticesForLabelPropertyIndex(label, property))) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
if (!PersistLabelPropertyIndexAndExistenceConstraintCreation(label, property, label_property_index_str)) {
return StorageIndexDefinitionError{IndexPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_create, label, property);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DropIndex(
LabelId label, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (!indices_.label_index_->DropIndex(label)) {
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::DropIndex(LabelId label) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_label_index = static_cast<DiskLabelIndex *>(on_disk->indices_.label_index_.get());
if (!disk_label_index->DropIndex(label)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
if (!PersistLabelIndexDeletion(label)) {
return StorageIndexDefinitionError{IndexPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_drop, label);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelIndices);
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DropIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (!indices_.label_property_index_->DropIndex(label, property)) {
utils::BasicResult<StorageIndexDefinitionError, void> DiskStorage::DiskAccessor::DropIndex(LabelId label,
PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_label_property_index =
static_cast<DiskLabelPropertyIndex *>(on_disk->indices_.label_property_index_.get());
if (!disk_label_property_index->DropIndex(label, property)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
if (!PersistLabelPropertyIndexAndExistenceConstraintDeletion(label, property, label_property_index_str)) {
return StorageIndexDefinitionError{IndexPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_drop, label, property);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
return {};
}
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> DiskStorage::CreateExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (constraints_.existence_constraints_->ConstraintExists(label, property)) {
utils::BasicResult<StorageExistenceConstraintDefinitionError, void>
DiskStorage::DiskAccessor::CreateExistenceConstraint(LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create existence constraint requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *existence_constraints = on_disk->constraints_.existence_constraints_.get();
if (existence_constraints->ConstraintExists(label, property)) {
return StorageExistenceConstraintDefinitionError{ConstraintDefinitionError{}};
}
if (auto check = CheckExistingVerticesBeforeCreatingExistenceConstraint(label, property); check.has_value()) {
if (auto check = on_disk->CheckExistingVerticesBeforeCreatingExistenceConstraint(label, property);
check.has_value()) {
return StorageExistenceConstraintDefinitionError{check.value()};
}
constraints_.existence_constraints_->InsertConstraint(label, property);
if (!PersistLabelPropertyIndexAndExistenceConstraintCreation(label, property, existence_constraints_str)) {
return StorageExistenceConstraintDefinitionError{ConstraintsPersistenceError{}};
}
existence_constraints->InsertConstraint(label, property);
transaction_.md_deltas.emplace_back(MetadataDelta::existence_constraint_create, label, property);
return {};
}
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DiskStorage::DropExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> /*desired_commit_timestamp*/) {
if (!constraints_.existence_constraints_->DropConstraint(label, property)) {
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DiskStorage::DiskAccessor::DropExistenceConstraint(
LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Drop existence constraint requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *existence_constraints = on_disk->constraints_.existence_constraints_.get();
if (!existence_constraints->DropConstraint(label, property)) {
return StorageExistenceConstraintDroppingError{ConstraintDefinitionError{}};
}
if (!PersistLabelPropertyIndexAndExistenceConstraintDeletion(label, property, existence_constraints_str)) {
return StorageExistenceConstraintDroppingError{ConstraintsPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::existence_constraint_drop, label, property);
return {};
}
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
DiskStorage::CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *disk_unique_constraints = static_cast<DiskUniqueConstraints *>(constraints_.unique_constraints_.get());
DiskStorage::DiskAccessor::CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) {
MG_ASSERT(unique_guard_.owns_lock(), "Create unique constraint requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_unique_constraints = static_cast<DiskUniqueConstraints *>(on_disk->constraints_.unique_constraints_.get());
if (auto constraint_check = disk_unique_constraints->CheckIfConstraintCanBeCreated(label, properties);
constraint_check != UniqueConstraints::CreationStatus::SUCCESS) {
return constraint_check;
}
auto check = CheckExistingVerticesBeforeCreatingUniqueConstraint(label, properties);
auto check = on_disk->CheckExistingVerticesBeforeCreatingUniqueConstraint(label, properties);
if (check.HasError()) {
return StorageUniqueConstraintDefinitionError{check.GetError()};
}
if (!disk_unique_constraints->InsertConstraint(label, properties, check.GetValue())) {
return StorageUniqueConstraintDefinitionError{ConstraintDefinitionError{}};
}
if (!PersistUniqueConstraintCreation(label, properties)) {
return StorageUniqueConstraintDefinitionError{ConstraintsPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::unique_constraint_create, label, properties);
return UniqueConstraints::CreationStatus::SUCCESS;
}
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
DiskStorage::DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> /*desired_commit_timestamp*/) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto ret = constraints_.unique_constraints_->DropConstraint(label, properties);
if (ret != UniqueConstraints::DeletionStatus::SUCCESS) {
UniqueConstraints::DeletionStatus DiskStorage::DiskAccessor::DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties) {
MG_ASSERT(unique_guard_.owns_lock(), "Drop unique constraint requires a unique access to the storage!");
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_unique_constraints = static_cast<DiskUniqueConstraints *>(on_disk->constraints_.unique_constraints_.get());
if (auto ret = disk_unique_constraints->DropConstraint(label, properties);
ret != UniqueConstraints::DeletionStatus::SUCCESS) {
return ret;
}
if (!PersistUniqueConstraintDeletion(label, properties)) {
return StorageUniqueConstraintDroppingError{ConstraintsPersistenceError{}};
}
transaction_.md_deltas.emplace_back(MetadataDelta::unique_constraint_create, label, properties);
return UniqueConstraints::DeletionStatus::SUCCESS;
}
@ -1934,4 +1974,32 @@ uint64_t DiskStorage::CommitTimestamp(const std::optional<uint64_t> desired_comm
return *desired_commit_timestamp;
}
std::unique_ptr<Storage::Accessor> DiskStorage::Access(std::optional<IsolationLevel> override_isolation_level) {
auto isolation_level = override_isolation_level.value_or(isolation_level_);
if (isolation_level != IsolationLevel::SNAPSHOT_ISOLATION) {
throw utils::NotYetImplemented("Disk storage supports only SNAPSHOT isolation level.");
}
return std::unique_ptr<DiskAccessor>(
new DiskAccessor{Storage::Accessor::shared_access, this, isolation_level, storage_mode_});
}
std::unique_ptr<Storage::Accessor> DiskStorage::UniqueAccess(std::optional<IsolationLevel> override_isolation_level) {
auto isolation_level = override_isolation_level.value_or(isolation_level_);
if (isolation_level != IsolationLevel::SNAPSHOT_ISOLATION) {
throw utils::NotYetImplemented("Disk storage supports only SNAPSHOT isolation level.");
}
return std::unique_ptr<DiskAccessor>(
new DiskAccessor{Storage::Accessor::unique_access, this, isolation_level, storage_mode_});
}
IndicesInfo DiskStorage::DiskAccessor::ListAllIndices() const {
auto *on_disk = static_cast<DiskStorage *>(storage_);
auto *disk_label_index = static_cast<DiskLabelIndex *>(on_disk->indices_.label_index_.get());
auto *disk_label_property_index =
static_cast<DiskLabelPropertyIndex *>(on_disk->indices_.label_property_index_.get());
return {disk_label_index->ListIndices(), disk_label_property_index->ListIndices()};
}
ConstraintsInfo DiskStorage::DiskAccessor::ListAllConstraints() const {
auto *disk_storage = static_cast<DiskStorage *>(storage_);
return {disk_storage->constraints_.existence_constraints_->ListConstraints(),
disk_storage->constraints_.unique_constraints_->ListConstraints()};
}
} // namespace memgraph::storage

View File

@ -44,7 +44,7 @@ class DiskStorage final : public Storage {
private:
friend class DiskStorage;
explicit DiskAccessor(DiskStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
explicit DiskAccessor(auto tag, DiskStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
/// TODO: const methods?
void LoadVerticesToMainMemoryCache();
@ -146,20 +146,12 @@ class DiskStorage final : public Storage {
return {};
}
std::vector<LabelId> ClearLabelIndexStats() override {
throw utils::NotYetImplemented("ClearIndexStats() is not implemented for DiskStorage.");
}
std::vector<std::pair<LabelId, PropertyId>> ClearLabelPropertyIndexStats() override {
throw utils::NotYetImplemented("ClearIndexStats() is not implemented for DiskStorage.");
}
std::vector<LabelId> DeleteLabelIndexStats(std::span<std::string> /*labels*/) override {
bool DeleteLabelIndexStats(const storage::LabelId & /*labels*/) override {
throw utils::NotYetImplemented("DeleteIndexStatsForLabels(labels) is not implemented for DiskStorage.");
}
std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(
const std::span<std::string> /*labels*/) override {
const storage::LabelId & /*labels*/) override {
throw utils::NotYetImplemented("DeleteIndexStatsForLabels(labels) is not implemented for DiskStorage.");
}
@ -195,18 +187,12 @@ class DiskStorage final : public Storage {
return disk_storage->indices_.label_property_index_->IndexExists(label, property);
}
IndicesInfo ListAllIndices() const override {
auto *disk_storage = static_cast<DiskStorage *>(storage_);
return disk_storage->ListAllIndices();
}
IndicesInfo ListAllIndices() const override;
ConstraintsInfo ListAllConstraints() const override {
auto *disk_storage = static_cast<DiskStorage *>(storage_);
return disk_storage->ListAllConstraints();
}
ConstraintsInfo ListAllConstraints() const override;
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> Commit(
utils::BasicResult<StorageManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
void UpdateObjectsCountOnAbort();
@ -228,6 +214,26 @@ class DiskStorage final : public Storage {
std::optional<storage::EdgeAccessor> DeserializeEdge(const rocksdb::Slice &key, const rocksdb::Slice &value,
const rocksdb::Slice &ts);
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) override;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) override;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) override;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) override;
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property) override;
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property) override;
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) override;
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override;
private:
VertexAccessor CreateVertexFromDisk(utils::SkipList<Vertex>::Accessor &accessor, storage::Gid gid,
std::vector<LabelId> &&label_ids, PropertyStore &&properties, Delta *delta);
@ -243,20 +249,20 @@ class DiskStorage final : public Storage {
/// At the time of calling, the commit_timestamp_ must already exist.
/// After this method, the vertex and edge caches are cleared.
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushIndexCache();
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushIndexCache();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushDeletedVertices();
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushDeletedVertices();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushDeletedEdges();
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushDeletedEdges();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushVertices(
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushVertices(
const auto &vertex_acc, std::vector<std::vector<PropertyValue>> &unique_storage);
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushModifiedEdges(const auto &edge_acc);
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> FlushModifiedEdges(const auto &edge_acc);
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> ClearDanglingVertices();
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> ClearDanglingVertices();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> CheckVertexConstraintsBeforeCommit(
[[nodiscard]] utils::BasicResult<StorageManipulationError, void> CheckVertexConstraintsBeforeCommit(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage) const;
bool WriteVertexToDisk(const Vertex &vertex);
@ -277,42 +283,14 @@ class DiskStorage final : public Storage {
std::vector<std::pair<std::string, std::string>> vertices_to_delete_;
rocksdb::Transaction *disk_transaction_;
bool scanned_all_vertices_ = false;
};
}; // Accessor
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override {
auto isolation_level = override_isolation_level.value_or(isolation_level_);
if (isolation_level != IsolationLevel::SNAPSHOT_ISOLATION) {
throw utils::NotYetImplemented("Disk storage supports only SNAPSHOT isolation level.");
}
return std::unique_ptr<DiskAccessor>(new DiskAccessor{this, isolation_level, storage_mode_});
}
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override;
std::unique_ptr<Storage::Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level) override;
RocksDBStorage *GetRocksDBStorage() const { return kvstore_.get(); }
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode) override;
void SetEdgeImportMode(EdgeImportMode edge_import_status);
@ -366,7 +344,7 @@ class DiskStorage final : public Storage {
StorageInfo GetInfo() const override;
void FreeMemory(std::unique_lock<utils::RWLock> /*lock*/) override {}
void FreeMemory(std::unique_lock<utils::ResourceLock> /*lock*/) override {}
void EstablishNewEpoch() override { throw utils::BasicException("Disk storage mode does not support replication."); }

View File

@ -133,8 +133,8 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
spdlog::info("Recreating indices from metadata.");
// Recover label indices.
spdlog::info("Recreating {} label indices from metadata.", indices_constraints.indices.label.size());
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(indices->label_index_.get());
for (const auto &item : indices_constraints.indices.label) {
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(indices->label_index_.get());
if (!mem_label_index->CreateIndex(item, vertices->access(), parallel_exec_info))
throw RecoveryFailure("The label index must be created here!");
@ -142,6 +142,15 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
}
spdlog::info("Label indices are recreated.");
spdlog::info("Recreating index statistics from metadata.");
// Recover label indices statistics.
spdlog::info("Recreating {} label index statistics from metadata.", indices_constraints.indices.label_stats.size());
for (const auto &item : indices_constraints.indices.label_stats) {
mem_label_index->SetIndexStats(item.first, item.second);
spdlog::info("A label index statistics is recreated from metadata.");
}
spdlog::info("Label indices statistics are recreated.");
// Recover label+property indices.
spdlog::info("Recreating {} label+property indices from metadata.",
indices_constraints.indices.label_property.size());
@ -152,6 +161,19 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
spdlog::info("A label+property index is recreated from metadata.");
}
spdlog::info("Label+property indices are recreated.");
// Recover label+property indices statistics.
spdlog::info("Recreating {} label+property indices statistics from metadata.",
indices_constraints.indices.label_property_stats.size());
for (const auto &item : indices_constraints.indices.label_property_stats) {
const auto label_id = item.first;
const auto property_id = item.second.first;
const auto &stats = item.second.second;
mem_label_property_index->SetIndexStats({label_id, property_id}, stats);
spdlog::info("A label+property index statistics is recreated from metadata.");
}
spdlog::info("Label+property indices statistics are recreated.");
spdlog::info("Indices are recreated.");
spdlog::info("Recreating constraints from metadata.");

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -56,6 +56,10 @@ enum class Marker : uint8_t {
DELTA_EXISTENCE_CONSTRAINT_DROP = 0x5e,
DELTA_UNIQUE_CONSTRAINT_CREATE = 0x5f,
DELTA_UNIQUE_CONSTRAINT_DROP = 0x60,
DELTA_LABEL_INDEX_STATS_SET = 0x61,
DELTA_LABEL_INDEX_STATS_CLEAR = 0x62,
DELTA_LABEL_PROPERTY_INDEX_STATS_SET = 0x63,
DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR = 0x64,
VALUE_FALSE = 0x00,
VALUE_TRUE = 0xff,
@ -93,6 +97,10 @@ static const Marker kMarkersAll[] = {
Marker::DELTA_TRANSACTION_END,
Marker::DELTA_LABEL_INDEX_CREATE,
Marker::DELTA_LABEL_INDEX_DROP,
Marker::DELTA_LABEL_INDEX_STATS_SET,
Marker::DELTA_LABEL_INDEX_STATS_CLEAR,
Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET,
Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR,
Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE,
Marker::DELTA_LABEL_PROPERTY_INDEX_DROP,
Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE,

View File

@ -19,6 +19,8 @@
#include "storage/v2/durability/exceptions.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
namespace memgraph::storage::durability {
@ -39,6 +41,8 @@ struct RecoveredIndicesAndConstraints {
struct {
std::vector<LabelId> label;
std::vector<std::pair<LabelId, PropertyId>> label_property;
std::vector<std::pair<LabelId, LabelIndexStats>> label_stats;
std::vector<std::pair<LabelId, std::pair<PropertyId, LabelPropertyIndexStats>>> label_property_stats;
} indices;
struct {
@ -74,4 +78,19 @@ void RemoveRecoveredIndexConstraint(std::vector<TObj> *list, TObj obj, const cha
}
}
// Helper function used to remove indices stats from the recovered
// indices/constraints object.
// @note multiple stats can be pushed one after the other; when removing, remove from the back
// @throw RecoveryFailure
template <typename TObj, typename K>
void RemoveRecoveredIndexStats(std::vector<TObj> *list, K label, const char *error_message) {
for (auto it = list->rbegin(); it != list->rend(); ++it) {
if (it->first == label) {
list->erase(std::next(it).base()); // erase using a reverse iterator
return;
}
}
throw RecoveryFailure(error_message);
}
} // namespace memgraph::storage::durability

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -344,6 +344,10 @@ std::optional<PropertyValue> Decoder::ReadPropertyValue() {
case Marker::DELTA_TRANSACTION_END:
case Marker::DELTA_LABEL_INDEX_CREATE:
case Marker::DELTA_LABEL_INDEX_DROP:
case Marker::DELTA_LABEL_INDEX_STATS_SET:
case Marker::DELTA_LABEL_INDEX_STATS_CLEAR:
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET:
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR:
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
@ -443,6 +447,10 @@ bool Decoder::SkipPropertyValue() {
case Marker::DELTA_TRANSACTION_END:
case Marker::DELTA_LABEL_INDEX_CREATE:
case Marker::DELTA_LABEL_INDEX_DROP:
case Marker::DELTA_LABEL_INDEX_STATS_SET:
case Marker::DELTA_LABEL_INDEX_STATS_CLEAR:
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET:
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR:
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:

View File

@ -23,6 +23,10 @@
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
@ -1077,10 +1081,11 @@ RecoveredSnapshot LoadSnapshotVersion14(const std::filesystem::path &path, utils
return {info, ret, std::move(indices_constraints)};
}
RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config) {
RecoveredSnapshot LoadSnapshotVersion15(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count,
const Config &config) {
RecoveryInfo recovery_info;
RecoveredIndicesAndConstraints indices_constraints;
@ -1089,9 +1094,7 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipLis
if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version));
if (*version == 14U) {
return LoadSnapshotVersion14(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config.items);
}
if (*version != 15U) throw RecoveryFailure(fmt::format("Expected snapshot version is 15, but got {}", *version));
// Cleanup of loaded data in case of failure.
bool success = false;
@ -1364,6 +1367,349 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipLis
return {info, recovery_info, std::move(indices_constraints)};
}
RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config) {
RecoveryInfo recovery_info;
RecoveredIndicesAndConstraints indices_constraints;
Decoder snapshot;
const auto version = snapshot.Initialize(path, kSnapshotMagic);
if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version));
if (*version == 14U) {
return LoadSnapshotVersion14(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config.items);
}
if (*version == 15U) {
return LoadSnapshotVersion15(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config);
}
// Cleanup of loaded data in case of failure.
bool success = false;
utils::OnScopeExit cleanup([&] {
if (!success) {
edges->clear();
vertices->clear();
epoch_history->clear();
}
});
// Read snapshot info.
const auto info = ReadSnapshotInfo(path);
spdlog::info("Recovering {} vertices and {} edges.", info.vertices_count, info.edges_count);
// Check for edges.
bool snapshot_has_edges = info.offset_edges != 0;
// Recover mapper.
std::unordered_map<uint64_t, uint64_t> snapshot_id_map;
{
spdlog::info("Recovering mapper metadata.");
if (!snapshot.SetPosition(info.offset_mapper)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_MAPPER) throw RecoveryFailure("Invalid snapshot data!");
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t i = 0; i < *size; ++i) {
auto id = snapshot.ReadUint();
if (!id) throw RecoveryFailure("Invalid snapshot data!");
auto name = snapshot.ReadString();
if (!name) throw RecoveryFailure("Invalid snapshot data!");
auto my_id = name_id_mapper->NameToId(*name);
snapshot_id_map.emplace(*id, my_id);
SPDLOG_TRACE("Mapping \"{}\"from snapshot id {} to actual id {}.", *name, *id, my_id);
}
}
auto get_label_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return LabelId::FromUint(it->second);
};
auto get_property_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return PropertyId::FromUint(it->second);
};
auto get_edge_type_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return EdgeTypeId::FromUint(it->second);
};
// Reset current edge count.
edge_count->store(0, std::memory_order_release);
{
spdlog::info("Recovering edges.");
// Recover edges.
if (snapshot_has_edges) {
// We don't need to check whether we store properties on edge or not, because `LoadPartialEdges` will always
// iterate over the edges in the snapshot (if they exist) and the current configuration of properties on edge only
// affect what it does:
// 1. If properties are allowed on edges, then it loads the edges.
// 2. If properties are not allowed on edges, then it checks that none of the edges have any properties.
if (!snapshot.SetPosition(info.offset_edge_batches)) {
throw RecoveryFailure("Couldn't read data from snapshot!");
}
const auto edge_batches = ReadBatchInfos(snapshot);
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, edges, items = config.items, &get_property_from_id](const size_t /*batch_index*/,
const BatchInfo &batch) {
LoadPartialEdges(path, *edges, batch.offset, batch.count, items, get_property_from_id);
},
edge_batches);
}
spdlog::info("Edges are recovered.");
// Recover vertices (labels and properties).
spdlog::info("Recovering vertices.", info.vertices_count);
uint64_t last_vertex_gid{0};
if (!snapshot.SetPosition(info.offset_vertex_batches)) {
throw RecoveryFailure("Couldn't read data from snapshot!");
}
const auto vertex_batches = ReadBatchInfos(snapshot);
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, vertices, &vertex_batches, &get_label_from_id, &get_property_from_id, &last_vertex_gid](
const size_t batch_index, const BatchInfo &batch) {
const auto last_vertex_gid_in_batch =
LoadPartialVertices(path, *vertices, batch.offset, batch.count, get_label_from_id, get_property_from_id);
if (batch_index == vertex_batches.size() - 1) {
last_vertex_gid = last_vertex_gid_in_batch;
}
},
vertex_batches);
spdlog::info("Vertices are recovered.");
// Recover vertices (in/out edges).
spdlog::info("Recover connectivity.");
recovery_info.vertex_batches.reserve(vertex_batches.size());
for (const auto batch : vertex_batches) {
recovery_info.vertex_batches.emplace_back(std::make_pair(Gid::FromUint(0), batch.count));
}
std::atomic<uint64_t> highest_edge_gid{0};
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, vertices, edges, edge_count, items = config.items, snapshot_has_edges, &get_edge_type_from_id,
&highest_edge_gid, &recovery_info](const size_t batch_index, const BatchInfo &batch) {
const auto result = LoadPartialConnectivity(path, *vertices, *edges, batch.offset, batch.count, items,
snapshot_has_edges, get_edge_type_from_id);
edge_count->fetch_add(result.edge_count);
auto known_highest_edge_gid = highest_edge_gid.load();
while (known_highest_edge_gid < result.highest_edge_id) {
highest_edge_gid.compare_exchange_weak(known_highest_edge_gid, result.highest_edge_id);
}
recovery_info.vertex_batches[batch_index].first = result.first_vertex_gid;
},
vertex_batches);
spdlog::info("Connectivity is recovered.");
// Set initial values for edge/vertex ID generators.
recovery_info.next_edge_id = highest_edge_gid + 1;
recovery_info.next_vertex_id = last_vertex_gid + 1;
}
// Recover indices.
{
spdlog::info("Recovering metadata of indices.");
if (!snapshot.SetPosition(info.offset_indices)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_INDICES) throw RecoveryFailure("Invalid snapshot data!");
// Recover label indices.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label indices.", *size);
for (uint64_t i = 0; i < *size; ++i) {
const auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.indices.label, get_label_from_id(*label),
"The label index already exists!");
SPDLOG_TRACE("Recovered metadata of label index for :{}", name_id_mapper->IdToName(snapshot_id_map.at(*label)));
}
spdlog::info("Metadata of label indices are recovered.");
}
// Recover label indices statistics.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label indices statistics.", *size);
for (uint64_t i = 0; i < *size; ++i) {
const auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
const auto count = snapshot.ReadUint();
if (!count) throw RecoveryFailure("Invalid snapshot data!");
const auto avg_degree = snapshot.ReadDouble();
if (!avg_degree) throw RecoveryFailure("Invalid snapshot data!");
const auto label_id = get_label_from_id(*label);
indices_constraints.indices.label_stats.emplace_back(label_id, LabelIndexStats{*count, *avg_degree});
SPDLOG_TRACE("Recovered metadata of label index statistics for :{}",
name_id_mapper->IdToName(snapshot_id_map.at(*label)));
}
spdlog::info("Metadata of label indices are recovered.");
}
// Recover label+property indices.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label+property indices.", *size);
for (uint64_t i = 0; i < *size; ++i) {
const auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
const auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.indices.label_property,
{get_label_from_id(*label), get_property_from_id(*property)},
"The label+property index already exists!");
SPDLOG_TRACE("Recovered metadata of label+property index for :{}({})",
name_id_mapper->IdToName(snapshot_id_map.at(*label)),
name_id_mapper->IdToName(snapshot_id_map.at(*property)));
}
spdlog::info("Metadata of label+property indices are recovered.");
}
// Recover label+property indices statistics.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label+property indices statistics.", *size);
for (uint64_t i = 0; i < *size; ++i) {
const auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
const auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
const auto count = snapshot.ReadUint();
if (!count) throw RecoveryFailure("Invalid snapshot data!");
const auto distinct_values_count = snapshot.ReadUint();
if (!distinct_values_count) throw RecoveryFailure("Invalid snapshot data!");
const auto statistic = snapshot.ReadDouble();
if (!statistic) throw RecoveryFailure("Invalid snapshot data!");
const auto avg_group_size = snapshot.ReadDouble();
if (!avg_group_size) throw RecoveryFailure("Invalid snapshot data!");
const auto avg_degree = snapshot.ReadDouble();
if (!avg_degree) throw RecoveryFailure("Invalid snapshot data!");
const auto label_id = get_label_from_id(*label);
const auto property_id = get_property_from_id(*property);
indices_constraints.indices.label_property_stats.emplace_back(
label_id, std::make_pair(property_id, LabelPropertyIndexStats{*count, *distinct_values_count, *statistic,
*avg_group_size, *avg_degree}));
SPDLOG_TRACE("Recovered metadata of label+property index statistics for :{}({})",
name_id_mapper->IdToName(snapshot_id_map.at(*label)),
name_id_mapper->IdToName(snapshot_id_map.at(*property)));
}
spdlog::info("Metadata of label+property indices are recovered.");
}
spdlog::info("Metadata of indices are recovered.");
}
// Recover constraints.
{
spdlog::info("Recovering metadata of constraints.");
if (!snapshot.SetPosition(info.offset_constraints)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_CONSTRAINTS) throw RecoveryFailure("Invalid snapshot data!");
// Recover existence constraints.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} existence constraints.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.constraints.existence,
{get_label_from_id(*label), get_property_from_id(*property)},
"The existence constraint already exists!");
SPDLOG_TRACE("Recovered metadata of existence constraint for :{}({})",
name_id_mapper->IdToName(snapshot_id_map.at(*label)),
name_id_mapper->IdToName(snapshot_id_map.at(*property)));
}
spdlog::info("Metadata of existence constraints are recovered.");
}
// Recover unique constraints.
// Snapshot version should be checked since unique constraints were
// implemented in later versions of snapshot.
if (*version >= kUniqueConstraintVersion) {
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} unique constraints.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
auto properties_count = snapshot.ReadUint();
if (!properties_count) throw RecoveryFailure("Invalid snapshot data!");
std::set<PropertyId> properties;
for (uint64_t j = 0; j < *properties_count; ++j) {
auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
properties.insert(get_property_from_id(*property));
}
AddRecoveredIndexConstraint(&indices_constraints.constraints.unique, {get_label_from_id(*label), properties},
"The unique constraint already exists!");
SPDLOG_TRACE("Recovered metadata of unique constraints for :{}",
name_id_mapper->IdToName(snapshot_id_map.at(*label)));
}
spdlog::info("Metadata of unique constraints are recovered.");
}
spdlog::info("Metadata of constraints are recovered.");
}
spdlog::info("Recovering metadata.");
// Recover epoch history
{
if (!snapshot.SetPosition(info.offset_epoch_history)) throw RecoveryFailure("Couldn't read data from snapshot!");
const auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_EPOCH_HISTORY) throw RecoveryFailure("Invalid snapshot data!");
const auto history_size = snapshot.ReadUint();
if (!history_size) {
throw RecoveryFailure("Invalid snapshot data!");
}
for (int i = 0; i < *history_size; ++i) {
auto maybe_epoch_id = snapshot.ReadString();
if (!maybe_epoch_id) {
throw RecoveryFailure("Invalid snapshot data!");
}
const auto maybe_last_commit_timestamp = snapshot.ReadUint();
if (!maybe_last_commit_timestamp) {
throw RecoveryFailure("Invalid snapshot data!");
}
epoch_history->emplace_back(std::move(*maybe_epoch_id), *maybe_last_commit_timestamp);
}
}
spdlog::info("Metadata recovered.");
// Recover timestamp.
recovery_info.next_timestamp = info.start_timestamp + 1;
// Set success flag (to disable cleanup).
success = true;
return {info, recovery_info, std::move(indices_constraints)};
}
void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snapshot_directory,
const std::filesystem::path &wal_directory, uint64_t snapshot_retention_count,
utils::SkipList<Vertex> *vertices, utils::SkipList<Edge> *edges, NameIdMapper *name_id_mapper,
@ -1577,6 +1923,31 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
}
}
// Write label indices statistics.
{
// NOTE: On-disk does not support snapshots
auto *inmem_index = static_cast<InMemoryLabelIndex *>(indices->label_index_.get());
auto label = inmem_index->ListIndices();
const auto size_pos = snapshot.GetPosition();
snapshot.WriteUint(0); // Just a place holder
unsigned i = 0;
for (const auto &item : label) {
auto stats = inmem_index->GetIndexStats(item);
if (stats) {
snapshot.WriteUint(item.AsUint());
snapshot.WriteUint(stats->count);
snapshot.WriteDouble(stats->avg_degree);
++i;
}
}
if (i != 0) {
const auto last_pos = snapshot.GetPosition();
snapshot.SetPosition(size_pos);
snapshot.WriteUint(i); // Write real size
snapshot.SetPosition(last_pos);
}
}
// Write label+property indices.
{
auto label_property = indices->label_property_index_->ListIndices();
@ -1586,6 +1957,35 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
write_mapping(item.second);
}
}
// Write label+property indices statistics.
{
// NOTE: On-disk does not support snapshots
auto *inmem_index = static_cast<InMemoryLabelPropertyIndex *>(indices->label_property_index_.get());
auto label = inmem_index->ListIndices();
const auto size_pos = snapshot.GetPosition();
snapshot.WriteUint(0); // Just a place holder
unsigned i = 0;
for (const auto &item : label) {
auto stats = inmem_index->GetIndexStats(item);
if (stats) {
snapshot.WriteUint(item.first.AsUint());
snapshot.WriteUint(item.second.AsUint());
snapshot.WriteUint(stats->count);
snapshot.WriteUint(stats->distinct_values_count);
snapshot.WriteDouble(stats->statistic);
snapshot.WriteDouble(stats->avg_group_size);
snapshot.WriteDouble(stats->avg_degree);
++i;
}
}
if (i != 0) {
const auto last_pos = snapshot.GetPosition();
snapshot.SetPosition(size_pos);
snapshot.WriteUint(i); // Write real size
snapshot.SetPosition(last_pos);
}
}
}
// Write constraints.

View File

@ -14,11 +14,15 @@
namespace memgraph::storage::durability {
/// Enum used to indicate a global database operation that isn't transactional.
enum class StorageGlobalOperation {
enum class StorageMetadataOperation {
LABEL_INDEX_CREATE,
LABEL_INDEX_DROP,
LABEL_INDEX_STATS_SET,
LABEL_INDEX_STATS_CLEAR,
LABEL_PROPERTY_INDEX_CREATE,
LABEL_PROPERTY_INDEX_DROP,
LABEL_PROPERTY_INDEX_STATS_SET,
LABEL_PROPERTY_INDEX_STATS_CLEAR,
EXISTENCE_CONSTRAINT_CREATE,
EXISTENCE_CONSTRAINT_DROP,
UNIQUE_CONSTRAINT_CREATE,

View File

@ -20,7 +20,7 @@ namespace memgraph::storage::durability {
// The current version of snapshot and WAL encoding / decoding.
// IMPORTANT: Please bump this version for every snapshot and/or WAL format
// change!!!
const uint64_t kVersion{15};
const uint64_t kVersion{16};
const uint64_t kOldestSupportedVersion{14};
const uint64_t kUniqueConstraintVersion{13};

View File

@ -13,9 +13,11 @@
#include "storage/v2/delta.hpp"
#include "storage/v2/durability/exceptions.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/file_locker.hpp"
#include "utils/logging.hpp"
@ -75,23 +77,31 @@ namespace memgraph::storage::durability {
namespace {
Marker OperationToMarker(StorageGlobalOperation operation) {
Marker OperationToMarker(StorageMetadataOperation operation) {
switch (operation) {
case StorageGlobalOperation::LABEL_INDEX_CREATE:
case StorageMetadataOperation::LABEL_INDEX_CREATE:
return Marker::DELTA_LABEL_INDEX_CREATE;
case StorageGlobalOperation::LABEL_INDEX_DROP:
case StorageMetadataOperation::LABEL_INDEX_DROP:
return Marker::DELTA_LABEL_INDEX_DROP;
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
case StorageMetadataOperation::LABEL_INDEX_STATS_SET:
return Marker::DELTA_LABEL_INDEX_STATS_SET;
case StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR:
return Marker::DELTA_LABEL_INDEX_STATS_CLEAR;
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE:
return Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE;
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP:
return Marker::DELTA_LABEL_PROPERTY_INDEX_DROP;
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_SET:
return Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET;
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR:
return Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR;
case StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE:
return Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE;
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP:
case StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP:
return Marker::DELTA_EXISTENCE_CONSTRAINT_DROP;
case StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
case StorageMetadataOperation::UNIQUE_CONSTRAINT_CREATE:
return Marker::DELTA_UNIQUE_CONSTRAINT_CREATE;
case StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP:
case StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP:
return Marker::DELTA_UNIQUE_CONSTRAINT_DROP;
}
}
@ -150,10 +160,18 @@ WalDeltaData::Type MarkerToWalDeltaDataType(Marker marker) {
return WalDeltaData::Type::LABEL_INDEX_CREATE;
case Marker::DELTA_LABEL_INDEX_DROP:
return WalDeltaData::Type::LABEL_INDEX_DROP;
case Marker::DELTA_LABEL_INDEX_STATS_SET:
return WalDeltaData::Type::LABEL_INDEX_STATS_SET;
case Marker::DELTA_LABEL_INDEX_STATS_CLEAR:
return WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR;
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE;
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP;
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_SET:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET;
case Marker::DELTA_LABEL_PROPERTY_INDEX_STATS_CLEAR:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR;
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
return WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE;
case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP:
@ -263,7 +281,11 @@ WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) {
case WalDeltaData::Type::TRANSACTION_END:
break;
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP: {
case WalDeltaData::Type::LABEL_INDEX_DROP:
case WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR: /* Special case, this clear is done on all label/property
pairs that contain the defined label */
{
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
@ -273,6 +295,18 @@ WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) {
}
break;
}
case WalDeltaData::Type::LABEL_INDEX_STATS_SET: {
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_stats.label = std::move(*label);
auto stats = decoder->ReadString();
if (!stats) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_stats.stats = std::move(*stats);
} else {
if (!decoder->SkipString() || !decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
} break;
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
@ -289,6 +323,23 @@ WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) {
}
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET: {
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_property_stats.label = std::move(*label);
auto property = decoder->ReadString();
if (!property) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_property_stats.property = std::move(*property);
auto stats = decoder->ReadString();
if (!stats) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_property_stats.stats = std::move(*stats);
} else {
if (!decoder->SkipString() || !decoder->SkipString() || !decoder->SkipString())
throw RecoveryFailure("Invalid WAL data!");
}
break;
}
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
if constexpr (read_data) {
@ -371,11 +422,11 @@ WalInfo ReadWalInfo(const std::filesystem::path &path) {
// Read deltas.
info.num_deltas = 0;
auto validate_delta = [&wal]() -> std::optional<std::pair<uint64_t, bool>> {
auto validate_delta = [&wal, version = *version]() -> std::optional<std::pair<uint64_t, bool>> {
try {
auto timestamp = ReadWalDeltaHeader(&wal);
auto type = SkipWalDeltaData(&wal);
return {{timestamp, IsWalDeltaDataTypeTransactionEnd(type)}};
return {{timestamp, IsWalDeltaDataTypeTransactionEnd(type, version)}};
} catch (const RecoveryFailure &) {
return std::nullopt;
}
@ -445,14 +496,28 @@ bool operator==(const WalDeltaData &a, const WalDeltaData &b) {
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP:
case WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR: /* Special case, label property index stats clear just
passes the label and all label/property pairs with the
label get cleared */
return a.operation_label.label == b.operation_label.label;
case WalDeltaData::Type::LABEL_INDEX_STATS_SET:
return a.operation_label_stats.label == b.operation_label_stats.label &&
a.operation_label_stats.stats == b.operation_label_stats.stats;
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP:
return a.operation_label_property.label == b.operation_label_property.label &&
a.operation_label_property.property == b.operation_label_property.property;
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET:
return a.operation_label_property_stats.label == b.operation_label_property_stats.label &&
a.operation_label_property_stats.property == b.operation_label_property_stats.property &&
a.operation_label_property_stats.stats == b.operation_label_property_stats.stats;
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP:
return a.operation_label_properties.label == b.operation_label_properties.label &&
@ -585,30 +650,50 @@ void EncodeTransactionEnd(BaseEncoder *encoder, uint64_t timestamp) {
encoder->WriteMarker(Marker::DELTA_TRANSACTION_END);
}
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties, uint64_t timestamp) {
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageMetadataOperation operation,
LabelId label, const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats, uint64_t timestamp) {
encoder->WriteMarker(Marker::SECTION_DELTA);
encoder->WriteUint(timestamp);
switch (operation) {
case StorageGlobalOperation::LABEL_INDEX_CREATE:
case StorageGlobalOperation::LABEL_INDEX_DROP: {
case StorageMetadataOperation::LABEL_INDEX_CREATE:
case StorageMetadataOperation::LABEL_INDEX_DROP:
case StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR:
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR: /* Special case, this clear is done on all
label/property pairs that contain the defined
label */
{
MG_ASSERT(properties.empty(), "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
break;
}
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP: {
case StorageMetadataOperation::LABEL_INDEX_STATS_SET: {
MG_ASSERT(properties.empty(), "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
encoder->WriteString(ToJson(stats));
break;
}
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE:
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP:
case StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE:
case StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP: {
MG_ASSERT(properties.size() == 1, "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
encoder->WriteString(name_id_mapper->IdToName((*properties.begin()).AsUint()));
break;
}
case StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
case StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP: {
case StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_SET: {
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
encoder->WriteString(name_id_mapper->IdToName((*properties.begin()).AsUint()));
encoder->WriteString(ToJson(property_stats));
break;
}
case StorageMetadataOperation::UNIQUE_CONSTRAINT_CREATE:
case StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP: {
MG_ASSERT(!properties.empty(), "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
@ -802,6 +887,21 @@ RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConst
"The label index doesn't exist!");
break;
}
case WalDeltaData::Type::LABEL_INDEX_STATS_SET: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_stats.label));
LabelIndexStats stats{};
if (!FromJson(delta.operation_label_stats.stats, stats)) {
throw RecoveryFailure("Failed to read statistics!");
}
indices_constraints->indices.label_stats.emplace_back(label_id, stats);
break;
}
case WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label.label));
RemoveRecoveredIndexStats(&indices_constraints->indices.label_stats, label_id,
"The label index stats doesn't exist!");
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
@ -816,6 +916,23 @@ RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConst
"The label property index doesn't exist!");
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET: {
auto &info = delta.operation_label_property_stats;
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(info.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(info.property));
LabelPropertyIndexStats stats{};
if (!FromJson(info.stats, stats)) {
throw RecoveryFailure("Failed to read statistics!");
}
indices_constraints->indices.label_property_stats.emplace_back(label_id, std::make_pair(property_id, stats));
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label.label));
RemoveRecoveredIndexStats(&indices_constraints->indices.label_property_stats, label_id,
"The label index stats doesn't exist!");
break;
}
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
@ -964,9 +1081,10 @@ void WalFile::AppendTransactionEnd(uint64_t timestamp) {
UpdateStats(timestamp);
}
void WalFile::AppendOperation(StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
void WalFile::AppendOperation(StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties,
const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats,
uint64_t timestamp) {
EncodeOperation(&wal_, name_id_mapper_, operation, label, properties, timestamp);
EncodeOperation(&wal_, name_id_mapper_, operation, label, properties, stats, property_stats, timestamp);
UpdateStats(timestamp);
}

View File

@ -21,8 +21,11 @@
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/serialization.hpp"
#include "storage/v2/durability/storage_global_operation.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex.hpp"
@ -58,8 +61,12 @@ struct WalDeltaData {
TRANSACTION_END,
LABEL_INDEX_CREATE,
LABEL_INDEX_DROP,
LABEL_INDEX_STATS_SET,
LABEL_INDEX_STATS_CLEAR,
LABEL_PROPERTY_INDEX_CREATE,
LABEL_PROPERTY_INDEX_DROP,
LABEL_PROPERTY_INDEX_STATS_SET,
LABEL_PROPERTY_INDEX_STATS_CLEAR,
EXISTENCE_CONSTRAINT_CREATE,
EXISTENCE_CONSTRAINT_DROP,
UNIQUE_CONSTRAINT_CREATE,
@ -103,12 +110,23 @@ struct WalDeltaData {
std::string label;
std::set<std::string> properties;
} operation_label_properties;
struct {
std::string label;
std::string stats;
} operation_label_stats;
struct {
std::string label;
std::string property;
std::string stats;
} operation_label_property_stats;
};
bool operator==(const WalDeltaData &a, const WalDeltaData &b);
bool operator!=(const WalDeltaData &a, const WalDeltaData &b);
constexpr bool IsWalDeltaDataTypeTransactionEnd(const WalDeltaData::Type type) {
constexpr bool IsWalDeltaDataTypeTransactionEndVersion15(const WalDeltaData::Type type) {
switch (type) {
// These delta actions are all found inside transactions so they don't
// indicate a transaction end.
@ -131,16 +149,28 @@ constexpr bool IsWalDeltaDataTypeTransactionEnd(const WalDeltaData::Type type) {
// 'transaction'.
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP:
case WalDeltaData::Type::LABEL_INDEX_STATS_SET:
case WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP:
return true;
return true; // TODO: Still true?
}
}
constexpr bool IsWalDeltaDataTypeTransactionEnd(const WalDeltaData::Type type, const uint64_t version = kVersion) {
if (version < 16U) {
return IsWalDeltaDataTypeTransactionEndVersion15(type);
}
// All deltas are now handled in a transactional scope
return type == WalDeltaData::Type::TRANSACTION_END;
}
/// Function used to read information about the WAL file.
/// @throw RecoveryFailure
WalInfo ReadWalInfo(const std::filesystem::path &path);
@ -174,8 +204,9 @@ void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, const Delta
void EncodeTransactionEnd(BaseEncoder *encoder, uint64_t timestamp);
/// Function used to encode non-transactional operation.
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties, uint64_t timestamp);
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageMetadataOperation operation,
LabelId label, const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats, uint64_t timestamp);
/// Function used to load the WAL data into the storage.
/// @throw RecoveryFailure
@ -204,8 +235,8 @@ class WalFile {
void AppendTransactionEnd(uint64_t timestamp);
void AppendOperation(StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
uint64_t timestamp);
void AppendOperation(StorageMetadataOperation operation, LabelId label, const std::set<PropertyId> &properties,
const LabelIndexStats &stats, const LabelPropertyIndexStats &property_stats, uint64_t timestamp);
void Sync();

View File

@ -0,0 +1,35 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <fmt/core.h>
#include "utils/simple_json.hpp"
namespace memgraph::storage {
struct LabelIndexStats {
uint64_t count;
double avg_degree;
};
static inline std::string ToJson(const LabelIndexStats &in) {
return fmt::format(R"({{"count":{}, "avg_degree":{}}})", in.count, in.avg_degree);
}
static inline bool FromJson(const std::string &json, LabelIndexStats &out) {
bool res = true;
res &= utils::GetJsonValue(json, "count", out.count);
res &= utils::GetJsonValue(json, "avg_degree", out.avg_degree);
return res;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,40 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <fmt/core.h>
#include "utils/simple_json.hpp"
namespace memgraph::storage {
struct LabelPropertyIndexStats {
uint64_t count, distinct_values_count;
double statistic, avg_group_size, avg_degree;
};
static inline std::string ToJson(const LabelPropertyIndexStats &in) {
return fmt::format(
R"({{"count":{}, "distinct_values_count":{}, "statistic":{}, "avg_group_size":{} "avg_degree":{}}})", in.count,
in.distinct_values_count, in.statistic, in.avg_group_size, in.avg_degree);
}
static inline bool FromJson(const std::string &json, LabelPropertyIndexStats &out) {
bool res = true;
res &= utils::GetJsonValue(json, "count", out.count);
res &= utils::GetJsonValue(json, "distinct_values_count", out.distinct_values_count);
res &= utils::GetJsonValue(json, "statistic", out.statistic);
res &= utils::GetJsonValue(json, "avg_group_size", out.avg_group_size);
res &= utils::GetJsonValue(json, "avg_degree", out.avg_degree);
return res;
}
} // namespace memgraph::storage

View File

@ -159,11 +159,13 @@ InMemoryLabelIndex::Iterable InMemoryLabelIndex::Vertices(LabelId label, View vi
}
void InMemoryLabelIndex::SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats) {
stats_[label] = stats;
auto locked_stats = stats_.Lock();
locked_stats->insert_or_assign(label, stats);
}
std::optional<LabelIndexStats> InMemoryLabelIndex::GetIndexStats(const storage::LabelId &label) const {
if (auto it = stats_.find(label); it != stats_.end()) {
auto locked_stats = stats_.ReadLock();
if (auto it = locked_stats->find(label); it != locked_stats->end()) {
return it->second;
}
return {};
@ -171,25 +173,24 @@ std::optional<LabelIndexStats> InMemoryLabelIndex::GetIndexStats(const storage::
std::vector<LabelId> InMemoryLabelIndex::ClearIndexStats() {
std::vector<LabelId> deleted_indexes;
deleted_indexes.reserve(stats_.size());
std::transform(stats_.begin(), stats_.end(), std::back_inserter(deleted_indexes),
auto locked_stats = stats_.Lock();
deleted_indexes.reserve(locked_stats->size());
std::transform(locked_stats->begin(), locked_stats->end(), std::back_inserter(deleted_indexes),
[](const auto &elem) { return elem.first; });
stats_.clear();
locked_stats->clear();
return deleted_indexes;
}
std::vector<LabelId> InMemoryLabelIndex::DeleteIndexStats(const storage::LabelId &label) {
std::vector<LabelId> deleted_indexes;
for (auto it = stats_.cbegin(); it != stats_.cend();) {
// stats_ is a map with label as the key, so only one can exist at a time
bool InMemoryLabelIndex::DeleteIndexStats(const storage::LabelId &label) {
auto locked_stats = stats_.Lock();
for (auto it = locked_stats->cbegin(); it != locked_stats->cend(); ++it) {
if (it->first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
} else {
++it;
locked_stats->erase(it);
return true;
}
}
return deleted_indexes;
return false;
}
} // namespace memgraph::storage

View File

@ -13,15 +13,12 @@
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/indices/label_index.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/rw_lock.hpp"
namespace memgraph::storage {
struct LabelIndexStats {
uint64_t count;
double avg_degree;
};
using ParallelizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
@ -108,11 +105,11 @@ class InMemoryLabelIndex : public storage::LabelIndex {
std::vector<LabelId> ClearIndexStats();
std::vector<LabelId> DeleteIndexStats(const storage::LabelId &label);
bool DeleteIndexStats(const storage::LabelId &label);
private:
std::map<LabelId, utils::SkipList<Entry>> index_;
std::map<LabelId, storage::LabelIndexStats> stats_;
utils::Synchronized<std::map<LabelId, storage::LabelIndexStats>, utils::ReadPrioritizedRWLock> stats_;
};
} // namespace memgraph::storage

View File

@ -70,7 +70,7 @@ bool InMemoryLabelPropertyIndex::CreateIndex(LabelId label, PropertyId property,
auto [it, emplaced] =
index_.emplace(std::piecewise_construct, std::forward_as_tuple(label, property), std::forward_as_tuple());
indices_by_property_[property].insert({label, &index_.at({label, property})});
indices_by_property_[property].insert({label, &it->second});
if (!emplaced) {
// Index already exists.
@ -386,20 +386,23 @@ uint64_t InMemoryLabelPropertyIndex::ApproximateVertexCount(
std::vector<std::pair<LabelId, PropertyId>> InMemoryLabelPropertyIndex::ClearIndexStats() {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
deleted_indexes.reserve(stats_.size());
std::transform(stats_.begin(), stats_.end(), std::back_inserter(deleted_indexes),
auto locked_stats = stats_.Lock();
deleted_indexes.reserve(locked_stats->size());
std::transform(locked_stats->begin(), locked_stats->end(), std::back_inserter(deleted_indexes),
[](const auto &elem) { return elem.first; });
stats_.clear();
locked_stats->clear();
return deleted_indexes;
}
// stats_ is a map where the key is a pair of label and property, so for one label many pairs can be deleted
std::vector<std::pair<LabelId, PropertyId>> InMemoryLabelPropertyIndex::DeleteIndexStats(
const storage::LabelId &label) {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
for (auto it = stats_.cbegin(); it != stats_.cend();) {
auto locked_stats = stats_.Lock();
for (auto it = locked_stats->cbegin(); it != locked_stats->cend();) {
if (it->first.first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
it = locked_stats->erase(it);
} else {
++it;
}
@ -409,12 +412,14 @@ std::vector<std::pair<LabelId, PropertyId>> InMemoryLabelPropertyIndex::DeleteIn
void InMemoryLabelPropertyIndex::SetIndexStats(const std::pair<storage::LabelId, storage::PropertyId> &key,
const LabelPropertyIndexStats &stats) {
stats_[key] = stats;
auto locked_stats = stats_.Lock();
locked_stats->insert_or_assign(key, stats);
}
std::optional<LabelPropertyIndexStats> InMemoryLabelPropertyIndex::GetIndexStats(
const std::pair<storage::LabelId, storage::PropertyId> &key) const {
if (auto it = stats_.find(key); it != stats_.end()) {
auto locked_stats = stats_.ReadLock();
if (auto it = locked_stats->find(key); it != locked_stats->end()) {
return it->second;
}
return {};

View File

@ -13,14 +13,11 @@
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/indices/label_property_index.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "utils/rw_lock.hpp"
namespace memgraph::storage {
struct LabelPropertyIndexStats {
uint64_t count, distinct_values_count;
double statistic, avg_group_size, avg_degree;
};
/// TODO: andi. Too many copies, extract at one place
using ParallelizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
@ -138,7 +135,9 @@ class InMemoryLabelPropertyIndex : public storage::LabelPropertyIndex {
private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
std::unordered_map<PropertyId, std::unordered_map<LabelId, utils::SkipList<Entry> *>> indices_by_property_;
std::map<std::pair<LabelId, PropertyId>, storage::LabelPropertyIndexStats> stats_;
utils::Synchronized<std::map<std::pair<LabelId, PropertyId>, storage::LabelPropertyIndexStats>,
utils::ReadPrioritizedRWLock>
stats_;
};
} // namespace memgraph::storage

View File

@ -13,6 +13,7 @@
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
@ -104,7 +105,8 @@ void InMemoryReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk
while (!transaction_complete) {
SPDLOG_INFO("Skipping delta");
const auto [timestamp, delta] = ReadDelta(&decoder);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(
delta.type, durability::kVersion); // TODO: Check if we are always using the latest version when replicating
}
replication::AppendDeltasRes res{false, storage_->replication_state_.last_commit_timestamp_.load()};
@ -112,7 +114,8 @@ void InMemoryReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk
return;
}
ReadAndApplyDelta(storage_, &decoder);
ReadAndApplyDelta(storage_, &decoder,
durability::kVersion); // TODO: Check if we are always using the latest version when replicating
replication::AppendDeltasRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
@ -264,7 +267,7 @@ void InMemoryReplicationServer::LoadWal(InMemoryStorage *storage, replication::D
wal.SetPosition(wal_info.offset_deltas);
for (size_t i = 0; i < wal_info.num_deltas;) {
i += ReadAndApplyDelta(storage, &wal);
i += ReadAndApplyDelta(storage, &wal, *version);
}
spdlog::debug("Replication from current WAL successful!");
@ -281,14 +284,23 @@ void InMemoryReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::B
slk::Save(res, res_builder);
}
uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder) {
uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder,
const uint64_t version) {
auto edge_acc = storage->edges_.access();
auto vertex_acc = storage->vertices_.access();
constexpr bool kUniqueAccess = true;
std::optional<std::pair<uint64_t, InMemoryStorage::ReplicationAccessor>> commit_timestamp_and_accessor;
auto get_transaction = [storage, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
auto get_transaction = [storage, &commit_timestamp_and_accessor](uint64_t commit_timestamp,
bool unique = !kUniqueAccess) {
if (!commit_timestamp_and_accessor) {
auto acc = storage->Access(std::nullopt);
std::unique_ptr<Storage::Accessor> acc = nullptr;
if (unique) {
acc = storage->UniqueAccess(std::nullopt);
} else {
acc = storage->Access(std::nullopt);
}
auto inmem_acc = std::unique_ptr<InMemoryStorage::InMemoryAccessor>(
static_cast<InMemoryStorage::InMemoryAccessor *>(acc.release()));
commit_timestamp_and_accessor.emplace(commit_timestamp, std::move(*inmem_acc));
@ -307,7 +319,7 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
max_commit_timestamp = timestamp;
}
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type, version);
if (timestamp < storage->timestamp_) {
continue;
@ -466,7 +478,7 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
case durability::WalDeltaData::Type::TRANSACTION_END: {
spdlog::trace(" Transaction end");
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
throw utils::BasicException("Invalid data!");
throw utils::BasicException("Invalid commit data!");
auto ret = commit_timestamp_and_accessor->second.Commit(commit_timestamp_and_accessor->first);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
commit_timestamp_and_accessor = std::nullopt;
@ -476,25 +488,45 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
case durability::WalDeltaData::Type::LABEL_INDEX_CREATE: {
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).HasError())
auto *transaction = get_transaction(timestamp, kUniqueAccess);
if (transaction->CreateIndex(storage->NameToLabel(delta.operation_label.label)).HasError())
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).HasError())
auto *transaction = get_transaction(timestamp, kUniqueAccess);
if (transaction->DropIndex(storage->NameToLabel(delta.operation_label.label)).HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_STATS_SET: {
spdlog::trace(" Set label index statistics on :{}", delta.operation_label_stats.label);
// Need to send the timestamp
auto *transaction = get_transaction(timestamp);
const auto label = storage->NameToLabel(delta.operation_label_stats.label);
LabelIndexStats stats{};
if (!FromJson(delta.operation_label_stats.stats, stats)) {
throw utils::BasicException("Failed to read statistics!");
}
transaction->SetIndexStats(label, stats);
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_STATS_CLEAR: {
const auto &info = delta.operation_label;
spdlog::trace(" Clear label index statistics on :{}", info.label);
// Need to send the timestamp
auto *transaction = get_transaction(timestamp);
transaction->DeleteLabelIndexStats(storage->NameToLabel(info.label));
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
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
auto *transaction = get_transaction(timestamp, kUniqueAccess);
if (transaction
->CreateIndex(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
storage->NameToProperty(delta.operation_label_property.property))
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
@ -502,31 +534,53 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: {
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
auto *transaction = get_transaction(timestamp, kUniqueAccess);
if (transaction
->DropIndex(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
storage->NameToProperty(delta.operation_label_property.property))
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_SET: {
const auto &info = delta.operation_label_property_stats;
spdlog::trace(" Set label-property index statistics on :{}", info.label);
// Need to send the timestamp
auto *transaction = get_transaction(timestamp);
const auto label = storage->NameToLabel(info.label);
const auto property = storage->NameToProperty(info.property);
LabelPropertyIndexStats stats{};
if (!FromJson(info.stats, stats)) {
throw utils::BasicException("Failed to read statistics!");
}
transaction->SetIndexStats(label, property, stats);
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_STATS_CLEAR: {
const auto &info = delta.operation_label;
spdlog::trace(" Clear label-property index statistics on :{}", info.label);
// Need to send the timestamp
auto *transaction = get_transaction(timestamp);
transaction->DeleteLabelPropertyIndexStats(storage->NameToLabel(info.label));
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
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 *transaction = get_transaction(timestamp, kUniqueAccess);
auto ret =
transaction->CreateExistenceConstraint(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property));
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
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
auto *transaction = get_transaction(timestamp, kUniqueAccess);
if (transaction
->DropExistenceConstraint(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
storage->NameToProperty(delta.operation_label_property.property))
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
@ -535,13 +589,13 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Create unique constraint on :{} ({})", delta.operation_label_properties.label, 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));
}
auto ret = storage->CreateUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
auto *transaction = get_transaction(timestamp, kUniqueAccess);
auto ret = transaction->CreateUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label),
properties);
if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
@ -550,21 +604,22 @@ uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage,
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Drop unique constraint on :{} ({})", delta.operation_label_properties.label, 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));
}
auto ret = storage->DropUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (ret.HasError() || ret.GetValue() != UniqueConstraints::DeletionStatus::SUCCESS)
auto *transaction = get_transaction(timestamp, kUniqueAccess);
auto ret =
transaction->DropUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label), properties);
if (ret != UniqueConstraints::DeletionStatus::SUCCESS) {
throw utils::BasicException("Invalid transaction!");
}
break;
}
}
}
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!");
if (commit_timestamp_and_accessor) throw utils::BasicException("Did not finish the transaction!");
storage->replication_state_.last_commit_timestamp_ = max_commit_timestamp;

View File

@ -38,7 +38,7 @@ class InMemoryReplicationServer : public ReplicationServer {
static void LoadWal(InMemoryStorage *storage, replication::Decoder *decoder);
static uint64_t ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder);
static uint64_t ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder, uint64_t version);
InMemoryStorage *storage_;
};

View File

@ -12,11 +12,13 @@
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/metadata_delta.hpp"
/// REPLICATION ///
#include "storage/v2/inmemory/replication/replication_client.hpp"
#include "storage/v2/inmemory/replication/replication_server.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
#include "utils/resource_lock.hpp"
namespace memgraph::storage {
@ -187,9 +189,9 @@ InMemoryStorage::~InMemoryStorage() {
}
}
InMemoryStorage::InMemoryAccessor::InMemoryAccessor(InMemoryStorage *storage, IsolationLevel isolation_level,
InMemoryStorage::InMemoryAccessor::InMemoryAccessor(auto tag, InMemoryStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode)
: Accessor(storage, isolation_level, storage_mode), config_(storage->config_.items) {}
: Accessor(tag, storage, isolation_level, storage_mode), config_(storage->config_.items) {}
InMemoryStorage::InMemoryAccessor::InMemoryAccessor(InMemoryAccessor &&other) noexcept
: Accessor(std::move(other)), config_(other.config_) {}
@ -642,7 +644,7 @@ Result<EdgeAccessor> InMemoryStorage::InMemoryAccessor::EdgeSetTo(EdgeAccessor *
}
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> InMemoryStorage::InMemoryAccessor::Commit(
utils::BasicResult<StorageManipulationError, void> InMemoryStorage::InMemoryAccessor::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!");
@ -651,7 +653,51 @@ utils::BasicResult<StorageDataManipulationError, void> InMemoryStorage::InMemory
auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
if (transaction_.deltas.use().empty()) {
if (!transaction_.md_deltas.empty()) {
// This is usually done by the MVCC, but it does not handle the metadata deltas
transaction_.EnsureCommitTimestampExists();
// Save these so we can mark them used in the commit log.
uint64_t start_timestamp = transaction_.start_timestamp;
std::unique_lock<utils::SpinLock> engine_guard(storage_->engine_lock_);
commit_timestamp_.emplace(mem_storage->CommitTimestamp(desired_commit_timestamp));
// 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 (mem_storage->replication_state_.GetRole() == replication::ReplicationRole::MAIN ||
desired_commit_timestamp.has_value()) {
could_replicate_all_sync_replicas = mem_storage->AppendToWalDataDefinition(transaction_, *commit_timestamp_);
// Take committed_transactions lock while holding the engine lock to
// make sure that committed transactions are sorted by the commit
// timestamp in the list.
mem_storage->committed_transactions_.WithLock([&](auto & /*committed_transactions*/) {
// TODO: release lock, and 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 (mem_storage->replication_state_.GetRole() == replication::ReplicationRole::MAIN ||
desired_commit_timestamp.has_value()) {
// Update the last commit timestamp
mem_storage->replication_state_.last_commit_timestamp_.store(*commit_timestamp_);
}
// Release engine lock because we don't have to hold it anymore
// and emplace back could take a long time.
engine_guard.unlock();
});
mem_storage->commit_log_->MarkFinished(start_timestamp);
}
} else if (transaction_.deltas.use().empty()) {
// We don't have to update the commit timestamp here because no one reads
// it.
mem_storage->commit_log_->MarkFinished(transaction_.start_timestamp);
@ -669,7 +715,7 @@ utils::BasicResult<StorageDataManipulationError, void> InMemoryStorage::InMemory
auto validation_result = storage_->constraints_.existence_constraints_->Validate(*prev.vertex);
if (validation_result) {
Abort();
return StorageDataManipulationError{*validation_result};
return StorageManipulationError{*validation_result};
}
}
@ -758,14 +804,14 @@ utils::BasicResult<StorageDataManipulationError, void> InMemoryStorage::InMemory
if (unique_constraint_violation) {
Abort();
return StorageDataManipulationError{*unique_constraint_violation};
return StorageManipulationError{*unique_constraint_violation};
}
}
is_transaction_active_ = false;
if (!could_replicate_all_sync_replicas) {
return StorageDataManipulationError{ReplicationError{}};
return StorageManipulationError{ReplicationError{}};
}
return {};
@ -951,189 +997,121 @@ void InMemoryStorage::InMemoryAccessor::FinalizeTransaction() {
}
}
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::CreateIndex(
LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(indices_.label_index_.get());
if (!mem_label_index->CreateIndex(label, vertices_.access(), std::nullopt)) {
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::CreateIndex(LabelId label) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(in_memory->indices_.label_index_.get());
if (!mem_label_index->CreateIndex(label, in_memory->vertices_.access(), std::nullopt)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
const auto success =
AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_create, label);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelIndices);
if (success) {
return {};
}
return StorageIndexDefinitionError{ReplicationError{}};
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::CreateIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *mem_label_property_index = static_cast<InMemoryLabelPropertyIndex *>(indices_.label_property_index_.get());
if (!mem_label_property_index->CreateIndex(label, property, vertices_.access(), std::nullopt)) {
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::CreateIndex(
LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_label_property_index =
static_cast<InMemoryLabelPropertyIndex *>(in_memory->indices_.label_property_index_.get());
if (!mem_label_property_index->CreateIndex(label, property, in_memory->vertices_.access(), std::nullopt)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_create, label, property);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
if (success) {
return {};
}
return StorageIndexDefinitionError{ReplicationError{}};
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::DropIndex(
LabelId label, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (!indices_.label_index_->DropIndex(label)) {
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::DropIndex(LabelId label) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(in_memory->indices_.label_index_.get());
if (!mem_label_index->DropIndex(label)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success =
AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_drop, label);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelIndices);
if (success) {
return {};
}
return StorageIndexDefinitionError{ReplicationError{}};
return {};
}
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::DropIndex(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (!indices_.label_property_index_->DropIndex(label, property)) {
utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::InMemoryAccessor::DropIndex(
LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_label_property_index =
static_cast<InMemoryLabelPropertyIndex *>(in_memory->indices_.label_property_index_.get());
if (!mem_label_property_index->DropIndex(label, property)) {
return StorageIndexDefinitionError{IndexDefinitionError{}};
}
// For a description why using `timestamp_` is correct, see
// `CreateIndex(LabelId label)`.
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_drop, label, property);
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
if (success) {
return {};
}
return StorageIndexDefinitionError{ReplicationError{}};
return {};
}
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> InMemoryStorage::CreateExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (constraints_.existence_constraints_->ConstraintExists(label, property)) {
utils::BasicResult<StorageExistenceConstraintDefinitionError, void>
InMemoryStorage::InMemoryAccessor::CreateExistenceConstraint(LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *existence_constraints = in_memory->constraints_.existence_constraints_.get();
if (existence_constraints->ConstraintExists(label, property)) {
return StorageExistenceConstraintDefinitionError{ConstraintDefinitionError{}};
}
if (auto violation = ExistenceConstraints::ValidateVerticesOnConstraint(vertices_.access(), label, property);
if (auto violation =
ExistenceConstraints::ValidateVerticesOnConstraint(in_memory->vertices_.access(), label, property);
violation.has_value()) {
return StorageExistenceConstraintDefinitionError{violation.value()};
}
constraints_.existence_constraints_->InsertConstraint(label, property);
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return {};
}
return StorageExistenceConstraintDefinitionError{ReplicationError{}};
existence_constraints->InsertConstraint(label, property);
transaction_.md_deltas.emplace_back(MetadataDelta::existence_constraint_create, label, property);
return {};
}
utils::BasicResult<StorageExistenceConstraintDroppingError, void> InMemoryStorage::DropExistenceConstraint(
LabelId label, PropertyId property, const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
if (!constraints_.existence_constraints_->DropConstraint(label, property)) {
utils::BasicResult<StorageExistenceConstraintDroppingError, void>
InMemoryStorage::InMemoryAccessor::DropExistenceConstraint(LabelId label, PropertyId property) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *existence_constraints = in_memory->constraints_.existence_constraints_.get();
if (!existence_constraints->DropConstraint(label, property)) {
return StorageExistenceConstraintDroppingError{ConstraintDefinitionError{}};
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return {};
}
return StorageExistenceConstraintDroppingError{ReplicationError{}};
transaction_.md_deltas.emplace_back(MetadataDelta::existence_constraint_drop, label, property);
return {};
}
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
InMemoryStorage::CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto *mem_unique_constraints = static_cast<InMemoryUniqueConstraints *>(constraints_.unique_constraints_.get());
auto ret = mem_unique_constraints->CreateConstraint(label, properties, vertices_.access());
InMemoryStorage::InMemoryAccessor::CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_unique_constraints =
static_cast<InMemoryUniqueConstraints *>(in_memory->constraints_.unique_constraints_.get());
auto ret = mem_unique_constraints->CreateConstraint(label, properties, in_memory->vertices_.access());
if (ret.HasError()) {
return StorageUniqueConstraintDefinitionError{ret.GetError()};
}
if (ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS) {
return ret.GetValue();
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label,
properties, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return UniqueConstraints::CreationStatus::SUCCESS;
}
return StorageUniqueConstraintDefinitionError{ReplicationError{}};
transaction_.md_deltas.emplace_back(MetadataDelta::unique_constraint_create, label, properties);
return UniqueConstraints::CreationStatus::SUCCESS;
}
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
InMemoryStorage::DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
const std::optional<uint64_t> desired_commit_timestamp) {
std::unique_lock<utils::RWLock> storage_guard(main_lock_);
auto ret = constraints_.unique_constraints_->DropConstraint(label, properties);
UniqueConstraints::DeletionStatus InMemoryStorage::InMemoryAccessor::DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties) {
MG_ASSERT(unique_guard_.owns_lock(), "Create index requires a unique access to the storage!");
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_unique_constraints =
static_cast<InMemoryUniqueConstraints *>(in_memory->constraints_.unique_constraints_.get());
auto ret = mem_unique_constraints->DropConstraint(label, properties);
if (ret != UniqueConstraints::DeletionStatus::SUCCESS) {
return ret;
}
const auto commit_timestamp = CommitTimestamp(desired_commit_timestamp);
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label,
properties, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return UniqueConstraints::DeletionStatus::SUCCESS;
}
return StorageUniqueConstraintDroppingError{ReplicationError{}};
transaction_.md_deltas.emplace_back(MetadataDelta::unique_constraint_drop, label, properties);
return UniqueConstraints::DeletionStatus::SUCCESS;
}
VerticesIterable InMemoryStorage::InMemoryAccessor::Vertices(LabelId label, View view) {
@ -1191,7 +1169,7 @@ Transaction InMemoryStorage::CreateTransaction(IsolationLevel isolation_level, S
}
template <bool force>
void InMemoryStorage::CollectGarbage(std::unique_lock<utils::RWLock> main_guard) {
void InMemoryStorage::CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard) {
// NOTE: You do not need to consider cleanup of deleted object that occurred in
// different storage modes within the same CollectGarbage call. This is because
// SetStorageMode will ensure CollectGarbage is called before any new transactions
@ -1501,8 +1479,8 @@ void InMemoryStorage::CollectGarbage(std::unique_lock<utils::RWLock> main_guard)
}
// tell the linker he can find the CollectGarbage definitions here
template void InMemoryStorage::CollectGarbage<true>(std::unique_lock<utils::RWLock>);
template void InMemoryStorage::CollectGarbage<false>(std::unique_lock<utils::RWLock>);
template void InMemoryStorage::CollectGarbage<true>(std::unique_lock<utils::ResourceLock>);
template void InMemoryStorage::CollectGarbage<false>(std::unique_lock<utils::ResourceLock>);
StorageInfo InMemoryStorage::GetInfo() const {
auto vertex_count = vertices_.size();
@ -1715,17 +1693,114 @@ bool InMemoryStorage::AppendToWalDataManipulation(const Transaction &transaction
return replication_state_.FinalizeTransaction(final_commit_timestamp);
}
bool InMemoryStorage::AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties,
uint64_t final_commit_timestamp) {
bool InMemoryStorage::AppendToWalDataDefinition(const Transaction &transaction, uint64_t final_commit_timestamp) {
if (!InitializeWalFile()) {
return true;
}
wal_file_->AppendOperation(operation, label, properties, final_commit_timestamp);
replication_state_.InitializeTransaction(wal_file_->SequenceNumber());
for (const auto &md_delta : transaction.md_deltas) {
switch (md_delta.action) {
case MetadataDelta::Action::LABEL_INDEX_CREATE: {
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_CREATE, md_delta.label,
final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_CREATE: {
const auto &info = md_delta.label_property;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_CREATE, info.label,
{info.property}, final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_INDEX_DROP: {
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_DROP, md_delta.label,
final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_DROP: {
const auto &info = md_delta.label_property;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_DROP, info.label,
{info.property}, final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_INDEX_STATS_SET: {
const auto &info = md_delta.label_stats;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_STATS_SET, info.label, info.stats,
final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_INDEX_STATS_CLEAR: {
const auto &info = md_delta.label_stats;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_INDEX_STATS_CLEAR, info.label,
final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_STATS_SET: {
const auto &info = md_delta.label_property_stats;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_SET, info.label,
{info.property}, info.stats, final_commit_timestamp);
} break;
case MetadataDelta::Action::LABEL_PROPERTY_INDEX_STATS_CLEAR: /* Special case we clear all label/property
pairs with the defined label */
{
const auto &info = md_delta.label_stats;
AppendToWalDataDefinition(durability::StorageMetadataOperation::LABEL_PROPERTY_INDEX_STATS_CLEAR, info.label,
final_commit_timestamp);
} break;
case MetadataDelta::Action::EXISTENCE_CONSTRAINT_CREATE: {
const auto &info = md_delta.label_property;
AppendToWalDataDefinition(durability::StorageMetadataOperation::EXISTENCE_CONSTRAINT_CREATE, info.label,
{info.property}, final_commit_timestamp);
} break;
case MetadataDelta::Action::EXISTENCE_CONSTRAINT_DROP: {
const auto &info = md_delta.label_property;
AppendToWalDataDefinition(durability::StorageMetadataOperation::EXISTENCE_CONSTRAINT_DROP, info.label,
{info.property}, final_commit_timestamp);
} break;
case MetadataDelta::Action::UNIQUE_CONSTRAINT_CREATE: {
const auto &info = md_delta.label_properties;
AppendToWalDataDefinition(durability::StorageMetadataOperation::UNIQUE_CONSTRAINT_CREATE, info.label,
info.properties, final_commit_timestamp);
} break;
case MetadataDelta::Action::UNIQUE_CONSTRAINT_DROP: {
const auto &info = md_delta.label_properties;
AppendToWalDataDefinition(durability::StorageMetadataOperation::UNIQUE_CONSTRAINT_DROP, info.label,
info.properties, final_commit_timestamp);
} break;
}
}
// Add a delta that indicates that the transaction is fully written to the WAL
wal_file_->AppendTransactionEnd(final_commit_timestamp);
FinalizeWalFile();
return replication_state_.AppendOperation(wal_file_->SequenceNumber(), operation, label, properties,
final_commit_timestamp);
return replication_state_.FinalizeTransaction(final_commit_timestamp);
}
void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, LabelIndexStats stats,
LabelPropertyIndexStats property_stats,
uint64_t final_commit_timestamp) {
wal_file_->AppendOperation(operation, label, properties, stats, property_stats, final_commit_timestamp);
replication_state_.AppendOperation(operation, label, properties, stats, property_stats, final_commit_timestamp);
}
void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties,
LabelPropertyIndexStats property_stats,
uint64_t final_commit_timestamp) {
return AppendToWalDataDefinition(operation, label, properties, {}, property_stats, final_commit_timestamp);
}
void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
LabelIndexStats stats, uint64_t final_commit_timestamp) {
return AppendToWalDataDefinition(operation, label, {}, stats, {}, final_commit_timestamp);
}
void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties,
uint64_t final_commit_timestamp) {
return AppendToWalDataDefinition(operation, label, properties, {}, final_commit_timestamp);
}
void InMemoryStorage::AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
uint64_t final_commit_timestamp) {
return AppendToWalDataDefinition(operation, label, {}, {}, final_commit_timestamp);
}
utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::CreateSnapshot(
@ -1756,7 +1831,7 @@ utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::Create
auto max_num_tries{10};
while (max_num_tries) {
if (should_try_shared) {
std::shared_lock<utils::RWLock> storage_guard(main_lock_);
std::shared_lock storage_guard(main_lock_);
if (storage_mode_ == memgraph::storage::StorageMode::IN_MEMORY_TRANSACTIONAL) {
snapshot_creator();
return {};
@ -1778,7 +1853,7 @@ utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::Create
return CreateSnapshotError::ReachedMaxNumTries;
}
void InMemoryStorage::FreeMemory(std::unique_lock<utils::RWLock> main_guard) {
void InMemoryStorage::FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) {
CollectGarbage<true>(std::move(main_guard));
// SkipList is already threadsafe
@ -1842,4 +1917,54 @@ std::unique_ptr<ReplicationServer> InMemoryStorage::CreateReplicationServer(
return std::make_unique<InMemoryReplicationServer>(this, config);
}
std::unique_ptr<Storage::Accessor> InMemoryStorage::Access(std::optional<IsolationLevel> override_isolation_level) {
return std::unique_ptr<InMemoryAccessor>(new InMemoryAccessor{
Storage::Accessor::shared_access, this, override_isolation_level.value_or(isolation_level_), storage_mode_});
}
std::unique_ptr<Storage::Accessor> InMemoryStorage::UniqueAccess(
std::optional<IsolationLevel> override_isolation_level) {
return std::unique_ptr<InMemoryAccessor>(new InMemoryAccessor{
Storage::Accessor::unique_access, this, override_isolation_level.value_or(isolation_level_), storage_mode_});
}
IndicesInfo InMemoryStorage::InMemoryAccessor::ListAllIndices() const {
auto *in_memory = static_cast<InMemoryStorage *>(storage_);
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(in_memory->indices_.label_index_.get());
auto *mem_label_property_index =
static_cast<InMemoryLabelPropertyIndex *>(in_memory->indices_.label_property_index_.get());
return {mem_label_index->ListIndices(), mem_label_property_index->ListIndices()};
}
ConstraintsInfo InMemoryStorage::InMemoryAccessor::ListAllConstraints() const {
const auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return {mem_storage->constraints_.existence_constraints_->ListConstraints(),
mem_storage->constraints_.unique_constraints_->ListConstraints()};
}
void InMemoryStorage::InMemoryAccessor::SetIndexStats(const storage::LabelId &label, const LabelIndexStats &stats) {
SetIndexStatsForIndex(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()), label, stats);
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_stats_set, label, stats);
}
void InMemoryStorage::InMemoryAccessor::SetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property,
const LabelPropertyIndexStats &stats) {
SetIndexStatsForIndex(static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()),
std::make_pair(label, property), stats);
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_stats_set, label, property, stats);
}
bool InMemoryStorage::InMemoryAccessor::DeleteLabelIndexStats(const storage::LabelId &label) {
const auto res =
DeleteIndexStatsForIndex<bool>(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()), label);
transaction_.md_deltas.emplace_back(MetadataDelta::label_index_stats_clear, label);
return res;
}
std::vector<std::pair<LabelId, PropertyId>> InMemoryStorage::InMemoryAccessor::DeleteLabelPropertyIndexStats(
const storage::LabelId &label) {
const auto &res = DeleteIndexStatsForIndex<std::vector<std::pair<LabelId, PropertyId>>>(
static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()), label);
transaction_.md_deltas.emplace_back(MetadataDelta::label_property_index_stats_clear, label);
return res;
}
} // namespace memgraph::storage

View File

@ -15,6 +15,7 @@
#include <cstdint>
#include <memory>
#include <utility>
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/storage.hpp"
@ -28,6 +29,7 @@
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/transaction.hpp"
#include "utils/memory.hpp"
#include "utils/resource_lock.hpp"
#include "utils/synchronized.hpp"
namespace memgraph::storage {
@ -64,7 +66,8 @@ class InMemoryStorage final : public Storage {
private:
friend class InMemoryStorage;
explicit InMemoryAccessor(InMemoryStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
explicit InMemoryAccessor(auto tag, InMemoryStorage *storage, IsolationLevel isolation_level,
StorageMode storage_mode);
public:
InMemoryAccessor(const InMemoryAccessor &) = delete;
@ -159,52 +162,19 @@ class InMemoryStorage final : public Storage {
index->SetIndexStats(key, stats);
}
void SetIndexStats(const storage::LabelId &label, const LabelIndexStats &stats) override {
SetIndexStatsForIndex(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()), label, stats);
}
void SetIndexStats(const storage::LabelId &label, const LabelIndexStats &stats) override;
void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const LabelPropertyIndexStats &stats) override {
SetIndexStatsForIndex(static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()),
std::make_pair(label, property), stats);
}
const LabelPropertyIndexStats &stats) override;
template <typename TResult, typename TIndex>
std::vector<TResult> ClearIndexStatsForIndex(TIndex *index) const {
return index->ClearIndexStats();
TResult DeleteIndexStatsForIndex(TIndex *index, const storage::LabelId &label) {
return index->DeleteIndexStats(label);
}
std::vector<LabelId> ClearLabelIndexStats() override {
return ClearIndexStatsForIndex<LabelId>(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()));
}
std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(const storage::LabelId &label) override;
std::vector<std::pair<LabelId, PropertyId>> ClearLabelPropertyIndexStats() override {
return ClearIndexStatsForIndex<std::pair<LabelId, PropertyId>>(
static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()));
}
template <typename TResult, typename TIndex>
std::vector<TResult> DeleteIndexStatsForIndex(TIndex *index, const std::span<std::string> labels) {
std::vector<TResult> deleted_indexes;
for (const auto &label : labels) {
std::vector<TResult> loc_results = index->DeleteIndexStats(NameToLabel(label));
deleted_indexes.insert(deleted_indexes.end(), std::make_move_iterator(loc_results.begin()),
std::make_move_iterator(loc_results.end()));
}
return deleted_indexes;
}
std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(
const std::span<std::string> labels) override {
return DeleteIndexStatsForIndex<std::pair<LabelId, PropertyId>>(
static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()), labels);
}
std::vector<LabelId> DeleteLabelIndexStats(const std::span<std::string> labels) override {
return DeleteIndexStatsForIndex<LabelId>(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()),
labels);
}
bool DeleteLabelIndexStats(const storage::LabelId &label) override;
Result<std::optional<std::pair<std::vector<VertexAccessor>, std::vector<EdgeAccessor>>>> DetachDelete(
std::vector<VertexAccessor *> nodes, std::vector<EdgeAccessor *> edges, bool detach) override;
@ -228,15 +198,9 @@ class InMemoryStorage final : public Storage {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_property_index_->IndexExists(label, property);
}
IndicesInfo ListAllIndices() const override {
const auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return mem_storage->ListAllIndices();
}
IndicesInfo ListAllIndices() const override;
ConstraintsInfo ListAllConstraints() const override {
const auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return mem_storage->ListAllConstraints();
}
ConstraintsInfo ListAllConstraints() const override;
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
@ -245,7 +209,7 @@ class InMemoryStorage final : public Storage {
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> Commit(
utils::BasicResult<StorageManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
/// @throw std::bad_alloc
@ -253,6 +217,78 @@ class InMemoryStorage final : public Storage {
void FinalizeTransaction() override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) override;
/// Returns void if the existence constraint has been created.
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there is already a vertex existing that would break this new constraint.
/// * `ConstraintDefinitionError`: the constraint already exists.
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property) override;
/// Drop an existing existence constraint.
/// Returns void if the existence constraint has been dropped.
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintDefinitionError`: the constraint did not exists.
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property) override;
/// Create an unique constraint.
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there are already vertices violating the constraint.
/// Returns `UniqueConstraints::CreationStatus` otherwise. Value can be:
/// * `SUCCESS` if the constraint was successfully created,
/// * `ALREADY_EXISTS` if the constraint already existed,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
/// @throw std::bad_alloc
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) override;
/// Removes an existing unique constraint.
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// Returns `UniqueConstraints::DeletionStatus` otherwise. Value can be:
/// * `SUCCESS` if constraint was successfully removed,
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) override;
protected:
// TODO Better naming
/// @throw std::bad_alloc
@ -280,88 +316,11 @@ class InMemoryStorage final : public Storage {
Transaction &GetTransaction() { return transaction_; }
};
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override {
return std::unique_ptr<InMemoryAccessor>(
new InMemoryAccessor{this, override_isolation_level.value_or(isolation_level_), storage_mode_});
}
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
std::unique_ptr<Storage::Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Returns void if the existence constraint has been created.
/// Returns `StorageExistenceConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there is already a vertex existing that would break this new constraint.
/// * `ConstraintDefinitionError`: the constraint already exists.
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing existence constraint.
/// Returns void if the existence constraint has been dropped.
/// Returns `StorageExistenceConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintDefinitionError`: the constraint did not exists.
utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an unique constraint.
/// Returns `StorageUniqueConstraintDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: there are already vertices violating the constraint.
/// Returns `UniqueConstraints::CreationStatus` otherwise. Value can be:
/// * `SUCCESS` if the constraint was successfully created,
/// * `ALREADY_EXISTS` if the constraint already existed,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
/// @throw std::bad_alloc
utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
/// Removes an existing unique constraint.
/// Returns `StorageUniqueConstraintDroppingError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// Returns `UniqueConstraints::DeletionStatus` otherwise. Value can be:
/// * `SUCCESS` if constraint was successfully removed,
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the limit of maximum number of properties.
utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus> DropUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp) override;
void FreeMemory(std::unique_lock<utils::RWLock> main_guard) override;
void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) override;
utils::FileRetainer::FileLockerAccessor::ret_type IsPathLocked();
utils::FileRetainer::FileLockerAccessor::ret_type LockPath();
@ -390,18 +349,33 @@ class InMemoryStorage final : public Storage {
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage(std::unique_lock<utils::RWLock> main_guard = {});
void CollectGarbage(std::unique_lock<utils::ResourceLock> main_guard = {});
bool InitializeWalFile();
void FinalizeWalFile();
StorageInfo GetInfo() const override;
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
[[nodiscard]] bool AppendToWalDataDefinition(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label, LabelIndexStats stats,
uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, LabelPropertyIndexStats property_stats,
uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
void AppendToWalDataDefinition(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, LabelIndexStats stats,
LabelPropertyIndexStats property_stats, uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});

View File

@ -0,0 +1,157 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <cstdint>
#include <set>
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage {
// NOTE: MetadataDeltas are not supported in a multi command transaction (single query only)
struct MetadataDelta {
enum class Action {
LABEL_INDEX_CREATE,
LABEL_INDEX_DROP,
LABEL_INDEX_STATS_SET,
LABEL_INDEX_STATS_CLEAR,
LABEL_PROPERTY_INDEX_CREATE,
LABEL_PROPERTY_INDEX_DROP,
LABEL_PROPERTY_INDEX_STATS_SET,
LABEL_PROPERTY_INDEX_STATS_CLEAR,
EXISTENCE_CONSTRAINT_CREATE,
EXISTENCE_CONSTRAINT_DROP,
UNIQUE_CONSTRAINT_CREATE,
UNIQUE_CONSTRAINT_DROP,
};
static constexpr struct LabelIndexCreate {
} label_index_create;
static constexpr struct LabelIndexDrop {
} label_index_drop;
static constexpr struct LabelIndexStatsSet {
} label_index_stats_set;
static constexpr struct LabelIndexStatsClear {
} label_index_stats_clear;
static constexpr struct LabelPropertyIndexCreate {
} label_property_index_create;
static constexpr struct LabelPropertyIndexDrop {
} label_property_index_drop;
static constexpr struct LabelPropertyIndexStatsSet {
} label_property_index_stats_set;
static constexpr struct LabelPropertyIndexStatsClear {
} label_property_index_stats_clear;
static constexpr struct ExistenceConstraintCreate {
} existence_constraint_create;
static constexpr struct ExistenceConstraintDrop {
} existence_constraint_drop;
static constexpr struct UniqueConstraintCreate {
} unique_constraint_create;
static constexpr struct UniqueConstraintDrop {
} unique_constraint_drop;
MetadataDelta(LabelIndexCreate /*tag*/, LabelId label) : action(Action::LABEL_INDEX_CREATE), label(label) {}
MetadataDelta(LabelIndexDrop /*tag*/, LabelId label) : action(Action::LABEL_INDEX_DROP), label(label) {}
MetadataDelta(LabelIndexStatsSet /*tag*/, LabelId label, LabelIndexStats stats)
: action(Action::LABEL_INDEX_STATS_SET), label_stats{label, stats} {}
MetadataDelta(LabelIndexStatsClear /*tag*/, LabelId label) : action(Action::LABEL_INDEX_STATS_CLEAR), label{label} {}
MetadataDelta(LabelPropertyIndexCreate /*tag*/, LabelId label, PropertyId property)
: action(Action::LABEL_PROPERTY_INDEX_CREATE), label_property{label, property} {}
MetadataDelta(LabelPropertyIndexDrop /*tag*/, LabelId label, PropertyId property)
: action(Action::LABEL_PROPERTY_INDEX_DROP), label_property{label, property} {}
MetadataDelta(LabelPropertyIndexStatsSet /*tag*/, LabelId label, PropertyId property, LabelPropertyIndexStats stats)
: action(Action::LABEL_PROPERTY_INDEX_STATS_SET), label_property_stats{label, property, stats} {}
MetadataDelta(LabelPropertyIndexStatsClear /*tag*/, LabelId label)
: action(Action::LABEL_PROPERTY_INDEX_STATS_CLEAR), label{label} {}
MetadataDelta(ExistenceConstraintCreate /*tag*/, LabelId label, PropertyId property)
: action(Action::EXISTENCE_CONSTRAINT_CREATE), label_property{label, property} {}
MetadataDelta(ExistenceConstraintDrop /*tag*/, LabelId label, PropertyId property)
: action(Action::EXISTENCE_CONSTRAINT_DROP), label_property{label, property} {}
MetadataDelta(UniqueConstraintCreate /*tag*/, LabelId label, std::set<PropertyId> properties)
: action(Action::UNIQUE_CONSTRAINT_CREATE), label_properties{label, std::move(properties)} {}
MetadataDelta(UniqueConstraintDrop /*tag*/, LabelId label, std::set<PropertyId> properties)
: action(Action::UNIQUE_CONSTRAINT_DROP), label_properties{label, std::move(properties)} {}
MetadataDelta(const MetadataDelta &) = delete;
MetadataDelta(MetadataDelta &&) = delete;
MetadataDelta &operator=(const MetadataDelta &) = delete;
MetadataDelta &operator=(MetadataDelta &&) = delete;
~MetadataDelta() {
switch (action) {
case Action::LABEL_INDEX_CREATE:
case Action::LABEL_INDEX_DROP:
case Action::LABEL_INDEX_STATS_SET:
case Action::LABEL_INDEX_STATS_CLEAR:
case Action::LABEL_PROPERTY_INDEX_CREATE:
case Action::LABEL_PROPERTY_INDEX_DROP:
case Action::LABEL_PROPERTY_INDEX_STATS_SET:
case Action::LABEL_PROPERTY_INDEX_STATS_CLEAR:
case Action::EXISTENCE_CONSTRAINT_CREATE:
case Action::EXISTENCE_CONSTRAINT_DROP:
break;
case Action::UNIQUE_CONSTRAINT_CREATE:
case Action::UNIQUE_CONSTRAINT_DROP:
label_properties.properties.~set<PropertyId>();
break;
}
}
Action action;
union {
LabelId label;
struct {
LabelId label;
PropertyId property;
} label_property;
struct {
LabelId label;
std::set<PropertyId> properties;
} label_properties;
struct {
LabelId label;
LabelIndexStats stats;
} label_stats;
struct {
LabelId label;
PropertyId property;
LabelPropertyIndexStats stats;
} label_property_stats;
};
};
static_assert(alignof(MetadataDelta) >= 8, "The Delta should be aligned to at least 8!");
} // namespace memgraph::storage

View File

@ -84,26 +84,19 @@ bool storage::ReplicationState::SetMainReplicationRole(storage::Storage *storage
return true;
}
bool storage::ReplicationState::AppendOperation(const uint64_t seq_num, durability::StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties,
void storage::ReplicationState::AppendOperation(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats,
uint64_t final_commit_timestamp) {
bool finalized_on_all_replicas = true;
// TODO Should we return true if not MAIN?
if (GetRole() == replication::ReplicationRole::MAIN) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->StartTransactionReplication(seq_num);
client->IfStreamingTransaction(
[&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); });
const auto finalized = client->FinalizeTransactionReplication();
if (client->Mode() == replication::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
client->IfStreamingTransaction([&](auto &stream) {
stream.AppendOperation(operation, label, properties, stats, property_stats, final_commit_timestamp);
});
}
});
}
return finalized_on_all_replicas;
}
void storage::ReplicationState::InitializeTransaction(uint64_t seq_num) {

View File

@ -51,8 +51,9 @@ struct ReplicationState {
void RestoreReplicationRole(Storage *storage);
// MAIN actually doing the replication
bool AppendOperation(uint64_t seq_num, durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
void AppendOperation(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats, uint64_t final_commit_timestamp);
void InitializeTransaction(uint64_t seq_num);
void AppendDelta(const Delta &delta, const Vertex &parent, uint64_t timestamp);
void AppendDelta(const Delta &delta, const Edge &parent, uint64_t timestamp);

View File

@ -15,6 +15,7 @@
#include <type_traits>
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/storage.hpp"
@ -308,10 +309,12 @@ void ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
EncodeTransactionEnd(&encoder, final_commit_timestamp);
}
void ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp) {
void ReplicaStream::AppendOperation(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats, uint64_t timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeOperation(&encoder, self_->GetStorage()->name_id_mapper_.get(), operation, label, properties, timestamp);
EncodeOperation(&encoder, self_->GetStorage()->name_id_mapper_.get(), operation, label, properties, stats,
property_stats, timestamp);
}
replication::AppendDeltasRes ReplicaStream::Finalize() { return stream_.AwaitResponse(); }

View File

@ -14,6 +14,8 @@
#include "rpc/client.hpp"
#include "storage/v2/durability/storage_global_operation.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index_stats.hpp"
#include "storage/v2/indices/label_property_index_stats.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/global.hpp"
@ -50,8 +52,9 @@ class ReplicaStream {
void AppendTransactionEnd(uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp);
void AppendOperation(durability::StorageMetadataOperation operation, LabelId label,
const std::set<PropertyId> &properties, const LabelIndexStats &stats,
const LabelPropertyIndexStats &property_stats, uint64_t timestamp);
/// @throw rpc::RpcFailedException
replication::AppendDeltasRes Finalize();

View File

@ -9,6 +9,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <thread>
#include "absl/container/flat_hash_set.h"
#include "spdlog/spdlog.h"
@ -56,12 +57,26 @@ Storage::Storage(Config config, StorageMode storage_mode)
replication_state_(config_.durability.restore_replication_state_on_startup,
config_.durability.storage_directory) {}
Storage::Accessor::Accessor(Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode)
Storage::Accessor::Accessor(SharedAccess /* tag */, Storage *storage, IsolationLevel isolation_level,
StorageMode storage_mode)
: storage_(storage),
// The lock must be acquired before creating the transaction object to
// prevent freshly created transactions from dangling in an active state
// during exclusive operations.
storage_guard_(storage_->main_lock_),
unique_guard_(storage_->main_lock_, std::defer_lock),
transaction_(storage->CreateTransaction(isolation_level, storage_mode)),
is_transaction_active_(true),
creation_storage_mode_(storage_mode) {}
Storage::Accessor::Accessor(UniqueAccess /* tag */, Storage *storage, IsolationLevel isolation_level,
StorageMode storage_mode)
: storage_(storage),
// The lock must be acquired before creating the transaction object to
// prevent freshly created transactions from dangling in an active state
// during exclusive operations.
storage_guard_(storage_->main_lock_, std::defer_lock),
unique_guard_(storage_->main_lock_),
transaction_(storage->CreateTransaction(isolation_level, storage_mode)),
is_transaction_active_(true),
creation_storage_mode_(storage_mode) {}
@ -69,6 +84,7 @@ Storage::Accessor::Accessor(Storage *storage, IsolationLevel isolation_level, St
Storage::Accessor::Accessor(Accessor &&other) noexcept
: storage_(other.storage_),
storage_guard_(std::move(other.storage_guard_)),
unique_guard_(std::move(other.unique_guard_)),
transaction_(std::move(other.transaction_)),
commit_timestamp_(other.commit_timestamp_),
is_transaction_active_(other.is_transaction_active_),
@ -78,16 +94,6 @@ Storage::Accessor::Accessor(Accessor &&other) noexcept
other.commit_timestamp_.reset();
}
IndicesInfo Storage::ListAllIndices() const {
std::shared_lock<utils::RWLock> storage_guard_(main_lock_);
return {indices_.label_index_->ListIndices(), indices_.label_property_index_->ListIndices()};
}
ConstraintsInfo Storage::ListAllConstraints() const {
std::shared_lock<utils::RWLock> storage_guard_(main_lock_);
return {constraints_.existence_constraints_->ListConstraints(), constraints_.unique_constraints_->ListConstraints()};
}
/// Main lock is taken by the caller.
void Storage::SetStorageMode(StorageMode storage_mode) {
std::unique_lock main_guard{main_lock_};

View File

@ -11,7 +11,10 @@
#pragma once
#include <chrono>
#include <semaphore>
#include <span>
#include <thread>
#include "io/network/endpoint.hpp"
#include "kvstore/kvstore.hpp"
@ -35,6 +38,7 @@
#include "storage/v2/vertices_iterable.hpp"
#include "utils/event_counter.hpp"
#include "utils/event_histogram.hpp"
#include "utils/resource_lock.hpp"
#include "utils/scheduler.hpp"
#include "utils/timer.hpp"
#include "utils/uuid.hpp"
@ -94,7 +98,13 @@ class Storage {
class Accessor {
public:
Accessor(Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
static constexpr struct SharedAccess {
} shared_access;
static constexpr struct UniqueAccess {
} unique_access;
Accessor(SharedAccess /* tag */, Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
Accessor(UniqueAccess /* tag */, Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
Accessor(const Accessor &) = delete;
Accessor &operator=(const Accessor &) = delete;
Accessor &operator=(Accessor &&other) = delete;
@ -149,14 +159,10 @@ class Storage {
virtual void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const LabelPropertyIndexStats &stats) = 0;
virtual std::vector<std::pair<LabelId, PropertyId>> ClearLabelPropertyIndexStats() = 0;
virtual std::vector<LabelId> ClearLabelIndexStats() = 0;
virtual std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(
std::span<std::string> labels) = 0;
const storage::LabelId &label) = 0;
virtual std::vector<LabelId> DeleteLabelIndexStats(std::span<std::string> labels) = 0;
virtual bool DeleteLabelIndexStats(const storage::LabelId &label) = 0;
virtual void PrefetchInEdges(const VertexAccessor &vertex_acc) = 0;
@ -179,7 +185,7 @@ class Storage {
virtual ConstraintsInfo ListAllConstraints() const = 0;
// NOLINTNEXTLINE(google-default-arguments)
virtual utils::BasicResult<StorageDataManipulationError, void> Commit(
virtual utils::BasicResult<StorageManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) = 0;
virtual void Abort() = 0;
@ -206,9 +212,30 @@ class Storage {
const std::string &id() const { return storage_->id(); }
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) = 0;
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) = 0;
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) = 0;
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) = 0;
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property) = 0;
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property) = 0;
virtual utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) = 0;
virtual UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label,
const std::set<PropertyId> &properties) = 0;
protected:
Storage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
std::shared_lock<utils::ResourceLock> storage_guard_;
std::unique_lock<utils::ResourceLock> unique_guard_; // TODO: Split the accessor into Shared/Unique
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
@ -251,58 +278,15 @@ class Storage {
StorageMode GetStorageMode() const;
virtual void FreeMemory(std::unique_lock<utils::RWLock> main_guard) = 0;
virtual void FreeMemory(std::unique_lock<utils::ResourceLock> main_guard) = 0;
void FreeMemory() { FreeMemory({}); }
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> Access() { return Access(std::optional<IsolationLevel>{}); }
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label) {
return CreateIndex(label, std::optional<uint64_t>{});
}
virtual utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(LabelId label, PropertyId property) {
return CreateIndex(label, property, std::optional<uint64_t>{});
}
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label) {
return DropIndex(label, std::optional<uint64_t>{});
}
virtual utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(LabelId label, PropertyId property) {
return DropIndex(label, property, std::optional<uint64_t>{});
}
IndicesInfo ListAllIndices() const;
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
virtual utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp) = 0;
virtual utils::BasicResult<StorageUniqueConstraintDroppingError, UniqueConstraints::DeletionStatus>
DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp) = 0;
ConstraintsInfo ListAllConstraints() const;
virtual std::unique_ptr<Accessor> UniqueAccess(std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> UniqueAccess() { return UniqueAccess(std::optional<IsolationLevel>{}); }
enum class SetIsolationLevelError : uint8_t { DisabledForAnalyticalMode };
@ -351,7 +335,7 @@ class Storage {
// creation of new accessors by taking a unique lock. This is used when doing
// operations on storage that affect the global state, for example index
// creation.
mutable utils::RWLock main_lock_{utils::RWLock::Priority::WRITE};
mutable utils::ResourceLock main_lock_;
// 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

View File

@ -19,30 +19,27 @@
namespace memgraph::storage {
struct ReplicationError {};
struct PersistenceError {}; // TODO: Generalize and add to InMemory durability as well (currently durability just
// asserts and terminated if failed)
struct IndexPersistenceError {};
struct IndexDefinitionError {};
struct ConstraintsPersistenceError {};
struct SerializationError {};
inline bool operator==(const SerializationError & /*err1*/, const SerializationError & /*err2*/) { return true; }
using StorageDataManipulationError = std::variant<ConstraintViolation, ReplicationError, SerializationError>;
using StorageManipulationError =
std::variant<ConstraintViolation, ReplicationError, SerializationError, PersistenceError>;
struct IndexDefinitionError {};
using StorageIndexDefinitionError = std::variant<IndexDefinitionError, ReplicationError, IndexPersistenceError>;
using StorageIndexDefinitionError = IndexDefinitionError;
struct ConstraintDefinitionError {};
using StorageExistenceConstraintDefinitionError =
std::variant<ConstraintViolation, ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageExistenceConstraintDefinitionError = std::variant<ConstraintViolation, ConstraintDefinitionError>;
using StorageExistenceConstraintDroppingError =
std::variant<ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageExistenceConstraintDroppingError = ConstraintDefinitionError;
using StorageUniqueConstraintDefinitionError =
std::variant<ConstraintViolation, ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageUniqueConstraintDroppingError = std::variant<ReplicationError, ConstraintsPersistenceError>;
using StorageUniqueConstraintDefinitionError = std::variant<ConstraintViolation, ConstraintDefinitionError>;
} // namespace memgraph::storage

View File

@ -22,6 +22,7 @@
#include "storage/v2/delta.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/metadata_delta.hpp"
#include "storage/v2/modified_edge.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/storage_mode.hpp"
@ -44,6 +45,7 @@ struct Transaction {
start_timestamp(start_timestamp),
command_id(0),
deltas(1024UL),
md_deltas(utils::NewDeleteResource()),
must_abort(false),
isolation_level(isolation_level),
storage_mode(storage_mode),
@ -55,6 +57,7 @@ struct Transaction {
commit_timestamp(std::move(other.commit_timestamp)),
command_id(other.command_id),
deltas(std::move(other.deltas)),
md_deltas(std::move(other.md_deltas)),
must_abort(other.must_abort),
isolation_level(other.isolation_level),
storage_mode(other.storage_mode),
@ -93,6 +96,7 @@ struct Transaction {
uint64_t command_id;
Bond<PmrListDelta> deltas;
utils::pmr::list<MetadataDelta> md_deltas;
bool must_abort;
IsolationLevel isolation_level;
StorageMode storage_mode;

View File

@ -30,7 +30,7 @@ struct Vertex {
"Vertex must be created with an initial DELETE_OBJECT delta!");
}
Gid gid;
const Gid gid;
std::vector<LabelId> labels;
PropertyStore properties;

View File

@ -0,0 +1,81 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <condition_variable>
#include <cstdint>
#include <mutex>
namespace memgraph::utils {
/* A type that complies to the
* C++ named requirements `SharedLockable`
*
* Unlike `std::shared_mutex` it can be locked in one thread
* and unlocked in another.
*/
struct ResourceLock {
private:
enum states { UNLOCKED, UNIQUE, SHARED };
public:
void lock() {
auto lock = std::unique_lock{mtx};
// block until available
cv.wait(lock, [this] { return state == UNLOCKED; });
state = UNIQUE;
}
void lock_shared() {
auto lock = std::unique_lock{mtx};
// block until available
cv.wait(lock, [this] { return state != UNIQUE; });
state = SHARED;
++count;
}
bool try_lock() {
auto lock = std::unique_lock{mtx};
if (state == UNLOCKED) {
state = UNIQUE;
return true;
}
return false;
}
bool try_lock_shared() {
auto lock = std::unique_lock{mtx};
if (state != UNIQUE) {
state = SHARED;
++count;
return true;
}
return false;
}
void unlock() {
auto lock = std::unique_lock{mtx};
state = UNLOCKED;
cv.notify_all(); // multiple lock_shared maybe waiting
}
void unlock_shared() {
auto lock = std::unique_lock{mtx};
--count;
if (count == 0) {
state = UNLOCKED;
cv.notify_one(); // should be 0 waiting in lock_shared, only 1 wait in lock can progress
}
}
private:
std::mutex mtx;
std::condition_variable cv;
states state = UNLOCKED;
uint64_t count = 0;
};
} // namespace memgraph::utils

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -126,4 +126,9 @@ class WritePrioritizedRWLock final : public RWLock {
WritePrioritizedRWLock() : RWLock{Priority::WRITE} {};
};
class ReadPrioritizedRWLock final : public RWLock {
public:
ReadPrioritizedRWLock() : RWLock{Priority::READ} {};
};
} // namespace memgraph::utils

56
src/utils/simple_json.hpp Normal file
View File

@ -0,0 +1,56 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <iostream>
#include <map>
#include <regex>
#include <sstream>
#include <string>
namespace memgraph::utils {
static inline std::regex GetRegexKey(const std::string &key) {
return std::regex("\"" + key + "\"\\s*:\\s*\"([^\"]+)\"|\"" + key + "\"\\s*:\\s*([^,\\}]+)");
}
// Function to parse JSON string and return a value by key as a string.
static inline bool GetJsonValue(const std::string &jsonString, const std::string &key, std::string &out) {
const auto regex = GetRegexKey(key);
std::smatch match;
if (std::regex_search(jsonString, match, regex)) {
out = match[1].str();
return true;
}
return false;
}
// Overloaded GetJsonValue function for different types.
template <typename T>
static inline bool GetJsonValue(const std::string &jsonString, const std::string &key, T &out) {
const auto regex = GetRegexKey(key);
std::smatch match;
if (std::regex_search(jsonString, match, regex)) {
for (size_t i = 1; i < match.size(); ++i) {
const auto &m = match[i].str();
if (m.empty()) continue;
std::istringstream ss(m);
ss >> out;
return true;
}
}
return false;
}
}; // namespace memgraph::utils

View File

@ -163,7 +163,8 @@ enum class TypeId : uint64_t {
AST_MERGE,
AST_UNWIND,
AST_AUTH_QUERY,
AST_INFO_QUERY,
AST_DATABASE_INFO_QUERY,
AST_SYSTEM_INFO_QUERY,
AST_CONSTRAINT,
AST_CONSTRAINT_QUERY,
AST_DUMP_QUERY,

View File

@ -52,7 +52,10 @@ class ExpansionBenchFixture : public benchmark::Fixture {
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db_acc->storage()->CreateIndex(label).HasError());
{
auto unique_acc = db_acc->UniqueAccess();
MG_ASSERT(!unique_acc->CreateIndex(label).HasError());
}
interpreter.emplace(&*interpreter_context, std::move(db_acc));
}
@ -69,8 +72,8 @@ BENCHMARK_DEFINE_F(ExpansionBenchFixture, Match)(benchmark::State &state) {
auto query = "MATCH (s:Starting) return s";
while (state.KeepRunning()) {
ResultStreamFaker results(interpreter->db_acc_->get()->storage());
interpreter->Prepare(query, {}, nullptr);
ResultStreamFaker results(interpreter->current_db_.db_acc_->get()->storage());
interpreter->Prepare(query, {}, {});
interpreter->PullAll(&results);
}
}
@ -84,8 +87,8 @@ BENCHMARK_DEFINE_F(ExpansionBenchFixture, Expand)(benchmark::State &state) {
auto query = "MATCH (s:Starting) WITH s MATCH (s)--(d) RETURN count(d)";
while (state.KeepRunning()) {
ResultStreamFaker results(interpreter->db_acc_->get()->storage());
interpreter->Prepare(query, {}, nullptr);
ResultStreamFaker results(interpreter->current_db_.db_acc_->get()->storage());
interpreter->Prepare(query, {}, {});
interpreter->PullAll(&results);
}
}

View File

@ -83,7 +83,10 @@ static void AddStarGraph(memgraph::storage::Storage *db, int spoke_count, int de
}
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
{
auto unique_acc = db->UniqueAccess();
MG_ASSERT(!unique_acc->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
}
static void AddTree(memgraph::storage::Storage *db, int vertex_count) {
@ -105,7 +108,10 @@ static void AddTree(memgraph::storage::Storage *db, int vertex_count) {
}
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
{
auto unique_acc = db->UniqueAccess();
MG_ASSERT(!unique_acc->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
}
static memgraph::query::CypherQuery *ParseCypherQuery(const std::string &query_string,

View File

@ -94,7 +94,10 @@ static memgraph::query::CypherQuery *AddIndexedMatches(int num_matches, const st
static auto CreateIndexedVertices(int index_count, int vertex_count, memgraph::storage::Storage *db) {
auto label = db->NameToLabel("label");
auto prop = db->NameToProperty("prop");
[[maybe_unused]] auto _ = db->CreateIndex(label, prop);
{
auto unique_acc = db->UniqueAccess();
[[maybe_unused]] auto _ = unique_acc->CreateIndex(label, prop);
}
auto dba = db->Access();
for (int vi = 0; vi < vertex_count; ++vi) {
for (int index = 0; index < index_count; ++index) {

View File

@ -30,7 +30,10 @@ TEST(Storage, LabelIndex) {
std::unique_ptr<memgraph::storage::Storage> store{new memgraph::storage::InMemoryStorage()};
auto label = store->NameToLabel("label");
ASSERT_FALSE(store->CreateIndex(label).HasError());
{
auto unique_acc = store->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateIndex(label).HasError());
}
std::vector<std::thread> verifiers;
verifiers.reserve(kNumVerifiers);
@ -112,7 +115,10 @@ TEST(Storage, LabelPropertyIndex) {
auto label = store->NameToLabel("label");
auto prop = store->NameToProperty("prop");
ASSERT_FALSE(store->CreateIndex(label, prop).HasError());
{
auto unique_acc = store->UniqueAccess();
ASSERT_FALSE(unique_acc->CreateIndex(label, prop).HasError());
}
std::vector<std::thread> verifiers;
verifiers.reserve(kNumVerifiers);

View File

@ -84,9 +84,11 @@ void AddLabel(memgraph::storage::Storage *storage, memgraph::storage::Gid gid, L
TEST_F(StorageUniqueConstraints, ChangeProperties) {
{
auto res = storage->CreateUniqueConstraint(label, {prop1, prop2, prop3}, {});
auto unique_acc = storage->UniqueAccess();
auto res = unique_acc->CreateUniqueConstraint(label, {prop1, prop2, prop3});
ASSERT_TRUE(res.HasValue());
ASSERT_EQ(res.GetValue(), memgraph::storage::UniqueConstraints::CreationStatus::SUCCESS);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
{
@ -166,9 +168,11 @@ TEST_F(StorageUniqueConstraints, ChangeProperties) {
TEST_F(StorageUniqueConstraints, ChangeLabels) {
{
auto res = storage->CreateUniqueConstraint(label, {prop1, prop2, prop3}, {});
auto unique_acc = storage->UniqueAccess();
auto res = unique_acc->CreateUniqueConstraint(label, {prop1, prop2, prop3});
ASSERT_TRUE(res.HasValue());
ASSERT_EQ(res.GetValue(), memgraph::storage::UniqueConstraints::CreationStatus::SUCCESS);
ASSERT_FALSE(unique_acc->Commit().HasError());
}
// In the first part of the test, each transaction tries to add the same label

View File

@ -3,6 +3,9 @@ find_package(gflags REQUIRED)
add_executable(memgraph__e2e__replication__constraints constraints.cpp)
target_link_libraries(memgraph__e2e__replication__constraints gflags mgclient mg-utils mg-io Threads::Threads)
add_executable(memgraph__e2e__replication__indices indices.cpp)
target_link_libraries(memgraph__e2e__replication__indices gflags mgclient mg-utils mg-io Threads::Threads)
add_executable(memgraph__e2e__replication__read_write_benchmark read_write_benchmark.cpp)
target_link_libraries(memgraph__e2e__replication__read_write_benchmark gflags json mgclient mg-utils mg-io Threads::Threads)

View File

@ -0,0 +1,174 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <chrono>
#include <random>
#include <thread>
#include <unordered_set>
#include <fmt/format.h>
#include <gflags/gflags.h>
#include "common.hpp"
#include "utils/logging.hpp"
#include "utils/thread.hpp"
#include "utils/timer.hpp"
int main(int argc, char **argv) {
google::SetUsageMessage("Memgraph E2E Replication Indices Test");
gflags::ParseCommandLineFlags(&argc, &argv, true);
memgraph::logging::RedirectToStderr();
auto database_endpoints = mg::e2e::replication::ParseDatabaseEndpoints(FLAGS_database_endpoints);
mg::Client::Init();
auto check_index = [](const auto &data, size_t i, const auto &label, const std::string &property = "") {
const auto label_name = (*data)[i][1].ValueString();
const auto p_type = (*data)[i][2].type();
const auto property_name = p_type == mg::Value::Type::String ? (*data)[i][2].ValueString() : "";
if (label_name != label || property_name != property) {
LOG_FATAL("Invalid index.");
}
};
auto check_delete_stats = [](const auto &data, size_t i, const auto &label, const std::string &property = "") {
const auto label_name = (*data)[i][0].ValueString();
const auto p_type = (*data)[i][1].type();
const auto property_name = p_type == mg::Value::Type::String ? (*data)[i][1].ValueString() : "";
if (label_name != label || property_name != property) {
LOG_FATAL("Invalid stats.");
}
};
{
auto client = mg::e2e::replication::Connect(database_endpoints[0]);
client->Execute("MATCH (n) DETACH DELETE n;");
client->DiscardAll();
client->Execute("CREATE INDEX ON :Node;");
client->DiscardAll();
client->Execute("CREATE INDEX ON :Node(id);");
client->DiscardAll();
client->Execute("CREATE INDEX ON :Node(id2);");
client->DiscardAll();
client->Execute("CREATE INDEX ON :Node2;");
client->DiscardAll();
// Sleep a bit so the constraints get replicated.
std::this_thread::sleep_for(std::chrono::milliseconds(500));
for (const auto &database_endpoint : database_endpoints) {
auto client = mg::e2e::replication::Connect(database_endpoint);
client->Execute("SHOW INDEX INFO;");
if (const auto data = client->FetchAll()) {
if (data->size() != 4) {
LOG_FATAL("Missing indices!");
}
check_index(data, 0, "Node");
check_index(data, 1, "Node2");
check_index(data, 2, "Node", "id");
check_index(data, 3, "Node", "id2");
} else {
LOG_FATAL("Unable to get INDEX INFO from {}", database_endpoint);
}
}
spdlog::info("All indices are in-place.");
for (int i = 0; i < FLAGS_nodes; ++i) {
client->Execute("CREATE (:Node {id:" + std::to_string(i) + "});");
client->DiscardAll();
}
mg::e2e::replication::IntGenerator edge_generator("EdgeCreateGenerator", 0, FLAGS_nodes - 1);
for (int i = 0; i < FLAGS_edges; ++i) {
client->Execute("MATCH (n {id:" + std::to_string(edge_generator.Next()) +
"}), (m {id:" + std::to_string(edge_generator.Next()) + "}) CREATE (n)-[:Edge]->(m);");
client->DiscardAll();
}
client->Execute("ANALYZE GRAPH;");
client->DiscardAll();
}
{
for (int i = 1; i < database_endpoints.size(); ++i) {
auto &database_endpoint = database_endpoints[i];
auto client = mg::e2e::replication::Connect(database_endpoint);
try {
// Hacky way to determine if the statistics were replicated
client->Execute("ANALYZE GRAPH DELETE STATISTICS;");
const auto data = client->FetchAll();
if (data->size() != 4) {
LOG_FATAL("Not all statistics were replicated! Failed endpoint {}:{}", database_endpoint.address,
database_endpoint.port);
}
check_delete_stats(data, 0, "Node");
check_delete_stats(data, 1, "Node2");
check_delete_stats(data, 2, "Node", "id");
check_delete_stats(data, 3, "Node", "id2");
} catch (const std::exception &e) {
LOG_FATAL(e.what());
break;
}
}
}
{
auto client = mg::e2e::replication::Connect(database_endpoints[0]);
client->Execute("ANALYZE GRAPH;"); // Resend stats
client->DiscardAll();
client->Execute("ANALYZE GRAPH DELETE STATISTICS;");
client->DiscardAll();
// Sleep a bit so the drop constraints get replicated.
std::this_thread::sleep_for(std::chrono::milliseconds(500));
for (int i = 1; i < database_endpoints.size(); ++i) {
auto &database_endpoint = database_endpoints[i];
auto client = mg::e2e::replication::Connect(database_endpoints[i]);
client->Execute("ANALYZE GRAPH DELETE STATISTICS;");
if (const auto data = client->FetchAll()) {
// Hacky way to determine if the statistics were replicated
if (data->size() != 0) {
LOG_FATAL("Not all statistics were replicated! Failed endpoint {}:{}", database_endpoint.address,
database_endpoint.port);
}
} else {
LOG_FATAL("Unable to delete statistics from {}", database_endpoints[i]);
}
}
}
{
auto client = mg::e2e::replication::Connect(database_endpoints[0]);
client->Execute("DROP INDEX ON :Node;");
client->DiscardAll();
client->Execute("DROP INDEX ON :Node(id);");
client->DiscardAll();
client->Execute("DROP INDEX ON :Node(id2);");
client->DiscardAll();
client->Execute("DROP INDEX ON :Node2;");
client->DiscardAll();
// Sleep a bit so the constraints get replicated.
std::this_thread::sleep_for(std::chrono::milliseconds(500));
for (const auto &database_endpoint : database_endpoints) {
auto client = mg::e2e::replication::Connect(database_endpoint);
client->Execute("SHOW INDEX INFO;");
if (const auto data = client->FetchAll()) {
if (!data->empty()) {
LOG_FATAL("Undeleted indices!");
}
} else {
LOG_FATAL("Unable to get INDEX INFO from {}", database_endpoint);
}
}
spdlog::info("All indices have been deleted.");
}
return 0;
}

View File

@ -974,7 +974,7 @@ def test_attempt_to_create_indexes_on_main_when_async_replica_is_down():
def test_attempt_to_create_indexes_on_main_when_sync_replica_is_down():
# Goal of this test is to check that main cannot create new indexes/constraints if a sync replica is down.
# Goal of this test is to check creation of new indexes/constraints when a sync replica is down.
# 0/ Start main and sync replicas.
# 1/ Check status of replicas.
# 2/ Add some indexes to main and check it is propagated to the sync_replicas.
@ -1052,9 +1052,10 @@ def test_attempt_to_create_indexes_on_main_when_sync_replica_is_down():
# 5/
expected_data = {
("sync_replica1", "127.0.0.1:10001", "sync", 0, 0, "invalid"),
("sync_replica2", "127.0.0.1:10002", "sync", 2, 0, "ready"),
("sync_replica2", "127.0.0.1:10002", "sync", 5, 0, "ready"),
}
res_from_main = interactive_mg_runner.MEMGRAPH_INSTANCES["main"].query(QUERY_TO_CHECK)
assert res_from_main == interactive_mg_runner.MEMGRAPH_INSTANCES["sync_replica2"].query(QUERY_TO_CHECK)
actual_data = set(interactive_mg_runner.MEMGRAPH_INSTANCES["main"].query("SHOW REPLICAS;"))
assert actual_data == expected_data

View File

@ -41,6 +41,11 @@ workloads:
args: []
<<: *template_cluster
- name: "Index replication"
binary: "tests/e2e/replication/memgraph__e2e__replication__indices"
args: []
<<: *template_cluster
- name: "Read-write benchmark"
binary: "tests/e2e/replication/memgraph__e2e__replication__read_write_benchmark"
args: []

View File

@ -0,0 +1,18 @@
// --storage-items-per-batch is set to 10
CREATE INDEX ON :`label2`(`prop2`);
CREATE INDEX ON :`label2`(`prop`);
CREATE INDEX ON :`label`;
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__:`label2` {__mg_id__: 0, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `ext`: 2, `prop`: "joj"});
CREATE (:__mg_vertex__:`label2`:`label` {__mg_id__: 1, `ext`: 2, `prop`: "joj"});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: 2, `prop`: 1});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 3, `prop2`: 2, `prop`: 2});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 0 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 1 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE;
ANALYZE GRAPH;
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,16 @@
CREATE INDEX ON :`label`;
CREATE INDEX ON :`label2`(`prop2`);
CREATE INDEX ON :`label2`(`prop`);
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__:`label2` {__mg_id__: 0, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `ext`: 2, `prop`: "joj"});
CREATE (:__mg_vertex__:`label`:`label2` {__mg_id__: 1, `ext`: 2, `prop`: "joj"});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: 2, `prop`: 1});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 3, `prop2`: 2, `prop`: 2});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 0 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 1 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`ext`: [false, {`k`: "l"}], `prop`: -1}]->(v);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE;
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,16 @@
CREATE INDEX ON :`label`;
CREATE INDEX ON :`label2`(`prop2`);
CREATE INDEX ON :`label2`(`prop`);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`prop2`, u.`prop` IS UNIQUE;
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__:`label2` {__mg_id__: 0, `prop2`: ["kaj", 2, Null, {`prop4`: -1.341}], `prop`: "joj", `ext`: 2});
CREATE (:__mg_vertex__:`label`:`label2` {__mg_id__: 1, `prop`: "joj", `ext`: 2});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 2, `prop2`: 2, `prop`: 1});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 3, `prop2`: 2, `prop`: 2});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 0 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 1 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 2 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 1 AND v.__mg_id__ = 3 CREATE (u)-[:`link` {`prop`: -1, `ext`: [false, {`k`: "l"}]}]->(v);
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,6 @@
CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`a`, u.`b` IS UNIQUE;

View File

@ -0,0 +1,6 @@
CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`b`, u.`a` IS UNIQUE;

View File

@ -0,0 +1,6 @@
CREATE CONSTRAINT ON (u:`label2`) ASSERT EXISTS (u.`ext2`);
CREATE CONSTRAINT ON (u:`label`) ASSERT EXISTS (u.`ext`);
CREATE CONSTRAINT ON (u:`label2`) ASSERT u.`a`, u.`b` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`a` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`b` IS UNIQUE;
CREATE CONSTRAINT ON (u:`label`) ASSERT u.`c` IS UNIQUE;

View File

@ -0,0 +1,60 @@
// --storage-items-per-batch is set to 7
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__ {__mg_id__: 1});
CREATE (:__mg_vertex__ {__mg_id__: 2});
CREATE (:__mg_vertex__ {__mg_id__: 3});
CREATE (:__mg_vertex__ {__mg_id__: 4});
CREATE (:__mg_vertex__ {__mg_id__: 5});
CREATE (:__mg_vertex__ {__mg_id__: 6});
CREATE (:__mg_vertex__ {__mg_id__: 7});
CREATE (:__mg_vertex__ {__mg_id__: 8});
CREATE (:__mg_vertex__ {__mg_id__: 9});
CREATE (:__mg_vertex__ {__mg_id__: 10});
CREATE (:__mg_vertex__ {__mg_id__: 11});
CREATE (:__mg_vertex__ {__mg_id__: 12});
CREATE (:__mg_vertex__ {__mg_id__: 13});
CREATE (:__mg_vertex__:`label` {__mg_id__: 14});
CREATE (:__mg_vertex__:`label` {__mg_id__: 15});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
ANALYZE GRAPH;
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,58 @@
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__ {__mg_id__: 1});
CREATE (:__mg_vertex__ {__mg_id__: 2});
CREATE (:__mg_vertex__ {__mg_id__: 3});
CREATE (:__mg_vertex__ {__mg_id__: 4});
CREATE (:__mg_vertex__ {__mg_id__: 5});
CREATE (:__mg_vertex__ {__mg_id__: 6});
CREATE (:__mg_vertex__ {__mg_id__: 7});
CREATE (:__mg_vertex__ {__mg_id__: 8});
CREATE (:__mg_vertex__ {__mg_id__: 9});
CREATE (:__mg_vertex__ {__mg_id__: 10});
CREATE (:__mg_vertex__ {__mg_id__: 11});
CREATE (:__mg_vertex__ {__mg_id__: 12});
CREATE (:__mg_vertex__ {__mg_id__: 13});
CREATE (:__mg_vertex__:`label` {__mg_id__: 14});
CREATE (:__mg_vertex__:`label` {__mg_id__: 15});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,58 @@
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__ {__mg_id__: 1});
CREATE (:__mg_vertex__ {__mg_id__: 2});
CREATE (:__mg_vertex__ {__mg_id__: 3});
CREATE (:__mg_vertex__ {__mg_id__: 4});
CREATE (:__mg_vertex__ {__mg_id__: 5});
CREATE (:__mg_vertex__ {__mg_id__: 6});
CREATE (:__mg_vertex__ {__mg_id__: 7});
CREATE (:__mg_vertex__ {__mg_id__: 8});
CREATE (:__mg_vertex__ {__mg_id__: 9});
CREATE (:__mg_vertex__ {__mg_id__: 10});
CREATE (:__mg_vertex__ {__mg_id__: 11});
CREATE (:__mg_vertex__ {__mg_id__: 12});
CREATE (:__mg_vertex__ {__mg_id__: 13});
CREATE (:__mg_vertex__:`label` {__mg_id__: 14});
CREATE (:__mg_vertex__:`label` {__mg_id__: 15});
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 0 AND v.__mg_id__ = 1 CREATE (u)-[:`edge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 2 AND v.__mg_id__ = 3 CREATE (u)-[:`edge` {`prop`: 11}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 4 AND v.__mg_id__ = 5 CREATE (u)-[:`edge` {`prop`: true}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 6 AND v.__mg_id__ = 7 CREATE (u)-[:`edge2`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 8 AND v.__mg_id__ = 9 CREATE (u)-[:`edge2` {`prop`: -3.141}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 10 AND v.__mg_id__ = 11 CREATE (u)-[:`edgelink` {`prop`: {`prop`: 1, `prop2`: {`prop4`: 9}}}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 12 AND v.__mg_id__ = 13 CREATE (u)-[:`edgelink` {`prop`: [1, Null, false, "\n\n\n\n\\\"\"\n\t"]}]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 14 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 0 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 1 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 2 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 3 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 4 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 5 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 6 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 7 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 8 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 9 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 10 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 11 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 12 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 13 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 14 CREATE (u)-[:`testedge`]->(v);
MATCH (u:__mg_vertex__), (v:__mg_vertex__) WHERE u.__mg_id__ = 15 AND v.__mg_id__ = 15 CREATE (u)-[:`testedge`]->(v);
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,5 @@
CREATE INDEX ON :`label2`;
CREATE INDEX ON :`label2`(`prop2`);
CREATE INDEX ON :`label`(`prop2`);
CREATE INDEX ON :`label`(`prop`);
ANALYZE GRAPH;

View File

@ -0,0 +1,4 @@
CREATE INDEX ON :`label2`;
CREATE INDEX ON :`label`(`prop`);
CREATE INDEX ON :`label`(`prop2`);
CREATE INDEX ON :`label2`(`prop2`);

View File

@ -0,0 +1,4 @@
CREATE INDEX ON :`label2`;
CREATE INDEX ON :`label2`(`prop2`);
CREATE INDEX ON :`label`(`prop2`);
CREATE INDEX ON :`label`(`prop`);

View File

@ -0,0 +1,18 @@
// --storage-items-per-batch is set to 5
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__:`label` {__mg_id__: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false});
CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141});
CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop3`: true, `prop2`: -314000000});
CREATE (:__mg_vertex__:`label3`:`label1`:`label2` {__mg_id__: 7});
CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop3`: "str", `prop2`: 2, `prop`: 1});
CREATE (:__mg_vertex__:`label2`:`label1` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}});
CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]});
CREATE (:__mg_vertex__:`label3`:`label` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}});
CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"});
ANALYZE GRAPH;
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,16 @@
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__:`label` {__mg_id__: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false});
CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141});
CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop3`: true, `prop2`: -314000000});
CREATE (:__mg_vertex__:`label2`:`label3`:`label1` {__mg_id__: 7});
CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop3`: "str", `prop2`: 2, `prop`: 1});
CREATE (:__mg_vertex__:`label1`:`label2` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}});
CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]});
CREATE (:__mg_vertex__:`label`:`label3` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}});
CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"});
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -0,0 +1,16 @@
CREATE INDEX ON :__mg_vertex__(__mg_id__);
CREATE (:__mg_vertex__ {__mg_id__: 0});
CREATE (:__mg_vertex__:`label` {__mg_id__: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 2, `prop`: false});
CREATE (:__mg_vertex__:`label` {__mg_id__: 3, `prop`: true});
CREATE (:__mg_vertex__:`label2` {__mg_id__: 4, `prop`: 1});
CREATE (:__mg_vertex__:`label` {__mg_id__: 5, `prop2`: 3.141});
CREATE (:__mg_vertex__:`label6` {__mg_id__: 6, `prop2`: -314000000, `prop3`: true});
CREATE (:__mg_vertex__:`label2`:`label3`:`label1` {__mg_id__: 7});
CREATE (:__mg_vertex__:`label` {__mg_id__: 8, `prop`: 1, `prop2`: 2, `prop3`: "str"});
CREATE (:__mg_vertex__:`label1`:`label2` {__mg_id__: 9, `prop`: {`prop_nes`: "kaj je"}});
CREATE (:__mg_vertex__:`label` {__mg_id__: 10, `prop_array`: [1, false, Null, "str", {`prop2`: 2}]});
CREATE (:__mg_vertex__:`label`:`label3` {__mg_id__: 11, `prop`: {`prop`: [1, false], `prop2`: {}, `prop3`: "test2", `prop4`: "test"}});
CREATE (:__mg_vertex__ {__mg_id__: 12, `prop`: " \n\"\'\t\\%"});
DROP INDEX ON :__mg_vertex__(__mg_id__);
MATCH (u) REMOVE u:__mg_vertex__, u.__mg_id__;

View File

@ -47,7 +47,6 @@ int main(int argc, char **argv) {
MG_ASSERT(res.fields[0] == "setting_name", "Expected \"setting_name\" field in the query result.");
MG_ASSERT(res.fields[1] == "setting_value", "Expected \"setting_value\" field in the query result.");
unsigned i = 0;
for (const auto &record : res.records) {
const auto &settings_name = record[0].ValueString();
if (settings_name == FLAGS_field) {

View File

@ -41,7 +41,7 @@ int main(int argc, char *argv[]) {
memgraph::query::Interpreter interpreter{&interpreter_context, db_acc};
ResultStreamFaker stream(db_acc->storage());
auto [header, _1, qid, _2] = interpreter.Prepare(argv[1], {}, nullptr);
auto [header, _1, qid, _2] = interpreter.Prepare(argv[1], {}, {});
stream.Header(header);
auto summary = interpreter.PullAll(&stream);
stream.Summary(summary);

View File

@ -293,6 +293,9 @@ target_link_libraries(${test_prefix}utils_temporal mg-utils)
add_unit_test(utils_java_string_formatter.cpp)
target_link_libraries(${test_prefix}utils_java_string_formatter mg-utils)
add_unit_test(utils_resource_lock.cpp)
target_link_libraries(${test_prefix}utils_resource_lock mg-utils)
# Test mg-storage-v2
add_unit_test(commit_log_v2.cpp)
target_link_libraries(${test_prefix}commit_log_v2 gflags mg-utils mg-storage-v2)
@ -304,7 +307,7 @@ add_unit_test(storage_v2.cpp)
target_link_libraries(${test_prefix}storage_v2 mg-storage-v2 storage_test_utils)
add_unit_test(storage_v2_constraints.cpp)
target_link_libraries(${test_prefix}storage_v2_constraints mg-storage-v2)
target_link_libraries(${test_prefix}storage_v2_constraints mg-storage-v2 mg-dbms)
add_unit_test(storage_v2_decoder_encoder.cpp)
target_link_libraries(${test_prefix}storage_v2_decoder_encoder mg-storage-v2)
@ -407,3 +410,9 @@ if(MG_ENTERPRISE)
add_unit_test_with_custom_main(dbms_handler.cpp)
target_link_libraries(${test_prefix}dbms_handler mg-query mg-auth mg-glue mg-dbms)
endif()
# Test distributed
add_unit_test(distributed_lamport_clock.cpp)
target_link_libraries(${test_prefix}distributed_lamport_clock mg-distributed)
target_include_directories(${test_prefix}distributed_lamport_clock PRIVATE ${CMAKE_SOURCE_DIR}/include)

View File

@ -117,7 +117,7 @@ class TestSession final : public Session<TestInputStream, TestOutputStream> {
std::optional<std::string> GetServerNameForInit() override { return std::nullopt; }
void Configure(const std::map<std::string, memgraph::communication::bolt::Value> &) override {}
std::string GetDatabaseName() const override { return ""; }
std::string GetCurrentDB() const override { return ""; }
void TestHook_ShouldAbort() { should_abort_ = true; }

View File

@ -2597,23 +2597,23 @@ TEST_P(CypherMainVisitorTest, TestProfileAuthQuery) {
TEST_P(CypherMainVisitorTest, TestShowStorageInfo) {
auto &ast_generator = *GetParam();
auto *query = dynamic_cast<InfoQuery *>(ast_generator.ParseQuery("SHOW STORAGE INFO"));
auto *query = dynamic_cast<SystemInfoQuery *>(ast_generator.ParseQuery("SHOW STORAGE INFO"));
ASSERT_TRUE(query);
EXPECT_EQ(query->info_type_, InfoQuery::InfoType::STORAGE);
EXPECT_EQ(query->info_type_, SystemInfoQuery::InfoType::STORAGE);
}
TEST_P(CypherMainVisitorTest, TestShowIndexInfo) {
auto &ast_generator = *GetParam();
auto *query = dynamic_cast<InfoQuery *>(ast_generator.ParseQuery("SHOW INDEX INFO"));
auto *query = dynamic_cast<DatabaseInfoQuery *>(ast_generator.ParseQuery("SHOW INDEX INFO"));
ASSERT_TRUE(query);
EXPECT_EQ(query->info_type_, InfoQuery::InfoType::INDEX);
EXPECT_EQ(query->info_type_, DatabaseInfoQuery::InfoType::INDEX);
}
TEST_P(CypherMainVisitorTest, TestShowConstraintInfo) {
auto &ast_generator = *GetParam();
auto *query = dynamic_cast<InfoQuery *>(ast_generator.ParseQuery("SHOW CONSTRAINT INFO"));
auto *query = dynamic_cast<DatabaseInfoQuery *>(ast_generator.ParseQuery("SHOW CONSTRAINT INFO"));
ASSERT_TRUE(query);
EXPECT_EQ(query->info_type_, InfoQuery::InfoType::CONSTRAINT);
EXPECT_EQ(query->info_type_, DatabaseInfoQuery::InfoType::CONSTRAINT);
}
TEST_P(CypherMainVisitorTest, CreateConstraintSyntaxError) {

Some files were not shown because too many files have changed in this diff Show More