Add experimental/v1 of ON_DISK_TRANSACTIONAL storage (#850)

Co-authored-by: Andi Skrgat <andi8647@gmail.com>
Co-authored-by: Aidar Samerkhanov <aidar.samerkhanov@memgraph.io>
This commit is contained in:
Marko Budiselić 2023-06-29 11:44:55 +02:00 committed by GitHub
parent aa4f68a37d
commit 9d056e7649
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
182 changed files with 26406 additions and 13468 deletions

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

View File

@ -50,7 +50,7 @@ struct MetricsResponse {
template <typename TSessionData>
class MetricsService {
public:
explicit MetricsService(TSessionData *data) : db_(data->db) {}
explicit MetricsService(TSessionData *data) : db_(data->interpreter_context->db.get()) {}
nlohmann::json GetMetricsJSON() {
auto response = GetMetrics();

View File

@ -56,6 +56,9 @@
#include "query/procedure/module.hpp"
#include "query/procedure/py_module.hpp"
#include "requests/requests.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/view.hpp"
@ -443,21 +446,19 @@ struct SessionData {
// supplied.
#if MG_ENTERPRISE
SessionData(memgraph::storage::Storage *db, memgraph::query::InterpreterContext *interpreter_context,
SessionData(memgraph::query::InterpreterContext *interpreter_context,
memgraph::utils::Synchronized<memgraph::auth::Auth, memgraph::utils::WritePrioritizedRWLock> *auth,
memgraph::audit::Log *audit_log)
: db(db), interpreter_context(interpreter_context), auth(auth), audit_log(audit_log) {}
memgraph::storage::Storage *db;
: interpreter_context(interpreter_context), auth(auth), audit_log(audit_log) {}
memgraph::query::InterpreterContext *interpreter_context;
memgraph::utils::Synchronized<memgraph::auth::Auth, memgraph::utils::WritePrioritizedRWLock> *auth;
memgraph::audit::Log *audit_log;
#else
SessionData(memgraph::storage::Storage *db, memgraph::query::InterpreterContext *interpreter_context,
SessionData(memgraph::query::InterpreterContext *interpreter_context,
memgraph::utils::Synchronized<memgraph::auth::Auth, memgraph::utils::WritePrioritizedRWLock> *auth)
: db(db), interpreter_context(interpreter_context), auth(auth) {}
memgraph::storage::Storage *db;
: interpreter_context(interpreter_context), auth(auth) {}
memgraph::query::InterpreterContext *interpreter_context;
memgraph::utils::Synchronized<memgraph::auth::Auth, memgraph::utils::WritePrioritizedRWLock> *auth;
@ -508,7 +509,6 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
memgraph::communication::v2::OutputStream *output_stream)
: memgraph::communication::bolt::Session<memgraph::communication::v2::InputStream,
memgraph::communication::v2::OutputStream>(input_stream, output_stream),
db_(data->db),
interpreter_context_(data->interpreter_context),
interpreter_(data->interpreter_context),
auth_(data->auth),
@ -583,7 +583,7 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
std::map<std::string, memgraph::communication::bolt::Value> Pull(TEncoder *encoder, std::optional<int> n,
std::optional<int> qid) override {
TypedValueResultStream stream(encoder, db_);
TypedValueResultStream stream(encoder, interpreter_context_->db.get());
return PullResults(stream, n, qid);
}
@ -617,7 +617,8 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
const auto &summary = interpreter_.Pull(&stream, n, qid);
std::map<std::string, memgraph::communication::bolt::Value> decoded_summary;
for (const auto &kv : summary) {
auto maybe_value = memgraph::glue::ToBoltValue(kv.second, *db_, memgraph::storage::View::NEW);
auto maybe_value =
memgraph::glue::ToBoltValue(kv.second, *interpreter_context_->db, memgraph::storage::View::NEW);
if (maybe_value.HasError()) {
switch (maybe_value.GetError()) {
case memgraph::storage::Error::DELETED_OBJECT:
@ -681,7 +682,6 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
};
// NOTE: Needed only for ToBoltValue conversions
const memgraph::storage::Storage *db_;
memgraph::query::InterpreterContext *interpreter_context_;
memgraph::query::Interpreter interpreter_;
memgraph::utils::Synchronized<memgraph::auth::Auth, memgraph::utils::WritePrioritizedRWLock> *auth_;
@ -895,7 +895,15 @@ int main(int argc, char **argv) {
.items_per_batch = FLAGS_storage_items_per_batch,
.recovery_thread_count = FLAGS_storage_recovery_thread_count,
.allow_parallel_index_creation = FLAGS_storage_parallel_index_recovery},
.transaction = {.isolation_level = ParseIsolationLevel()}};
.transaction = {.isolation_level = ParseIsolationLevel()},
.disk = {.main_storage_directory = FLAGS_data_directory + "/rocksdb_main_storage",
.label_index_directory = FLAGS_data_directory + "/rocksdb_label_index",
.label_property_index_directory = FLAGS_data_directory + "/rocksdb_label_property_index",
.unique_constraints_directory = FLAGS_data_directory + "/rocksdb_unique_constraints",
.name_id_mapper_directory = FLAGS_data_directory + "/rocksdb_name_id_mapper",
.id_name_mapper_directory = FLAGS_data_directory + "/rocksdb_id_name_mapper",
.durability_directory = FLAGS_data_directory + "/rocksdb_durability",
.wal_directory = FLAGS_data_directory + "/rocksdb_wal"}};
if (FLAGS_storage_snapshot_interval_sec == 0) {
if (FLAGS_storage_wal_enabled) {
LOG_FATAL(
@ -914,10 +922,9 @@ int main(int argc, char **argv) {
}
db_config.durability.snapshot_interval = std::chrono::seconds(FLAGS_storage_snapshot_interval_sec);
}
memgraph::storage::Storage db(db_config);
memgraph::query::InterpreterContext interpreter_context{
&db,
db_config,
{.query = {.allow_load_csv = FLAGS_allow_load_csv},
.execution_timeout_sec = FLAGS_query_execution_timeout_sec,
.replication_replica_check_frequency = std::chrono::seconds(FLAGS_replication_replica_check_frequency_sec),
@ -927,9 +934,9 @@ int main(int argc, char **argv) {
.stream_transaction_retry_interval = std::chrono::milliseconds(FLAGS_stream_transaction_retry_interval)},
FLAGS_data_directory};
#ifdef MG_ENTERPRISE
SessionData session_data{&db, &interpreter_context, &auth, &audit_log};
SessionData session_data{&interpreter_context, &auth, &audit_log};
#else
SessionData session_data{&db, &interpreter_context, &auth};
SessionData session_data{&interpreter_context, &auth};
#endif
memgraph::query::procedure::gModuleRegistry.SetModulesDirectory(query_modules_directories, FLAGS_data_directory);
@ -969,7 +976,7 @@ int main(int argc, char **argv) {
// Triggers can execute query procedures, so we need to reload the modules first and then
// the triggers
auto storage_accessor = interpreter_context.db->Access();
auto dba = memgraph::query::DbAccessor{&storage_accessor};
auto dba = memgraph::query::DbAccessor{storage_accessor.get()};
interpreter_context.trigger_store.RestoreTriggers(
&interpreter_context.ast_cache, &dba, interpreter_context.config.query, interpreter_context.auth_checker);
}
@ -1002,8 +1009,8 @@ int main(int argc, char **argv) {
std::optional<memgraph::telemetry::Telemetry> telemetry;
if (FLAGS_telemetry_enabled) {
telemetry.emplace(telemetry_server, data_directory / "telemetry", run_id, machine_id, std::chrono::minutes(10));
telemetry->AddCollector("storage", [&db]() -> nlohmann::json {
auto info = db.GetInfo();
telemetry->AddCollector("storage", [db_ = interpreter_context.db.get()]() -> nlohmann::json {
auto info = db_->GetInfo();
return {{"vertices", info.vertex_count}, {"edges", info.edge_count}};
});
telemetry->AddCollector("event_counters", []() -> nlohmann::json {

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
@ -20,7 +20,8 @@
#include <unordered_map>
#include "helpers.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "utils/exceptions.hpp"
#include "utils/logging.hpp"
#include "utils/message.hpp"
@ -421,7 +422,7 @@ void ProcessNodeRow(memgraph::storage::Storage *store, const std::vector<std::st
std::unordered_map<NodeId, memgraph::storage::Gid> *node_id_map) {
std::optional<NodeId> id;
auto acc = store->Access();
auto node = acc.CreateVertex();
auto node = acc->CreateVertex();
for (size_t i = 0; i < row.size(); ++i) {
const auto &field = fields[i];
const auto &value = row[i];
@ -450,29 +451,29 @@ void ProcessNodeRow(memgraph::storage::Storage *store, const std::vector<std::st
} else {
pv_id = memgraph::storage::PropertyValue(node_id.id);
}
auto old_node_property = node.SetProperty(acc.NameToProperty(field.name), pv_id);
auto old_node_property = node.SetProperty(acc->NameToProperty(field.name), pv_id);
if (!old_node_property.HasValue()) throw LoadException("Couldn't add property '{}' to the node", field.name);
if (!old_node_property->IsNull()) throw LoadException("The property '{}' already exists", field.name);
}
id = node_id;
} else if (field.type == "LABEL") {
for (const auto &label : memgraph::utils::Split(value, FLAGS_array_delimiter)) {
auto node_label = node.AddLabel(acc.NameToLabel(label));
auto node_label = node.AddLabel(acc->NameToLabel(label));
if (!node_label.HasValue()) throw LoadException("Couldn't add label '{}' to the node", label);
if (!*node_label) throw LoadException("The label '{}' already exists", label);
}
} else if (field.type != "IGNORE") {
auto old_node_property = node.SetProperty(acc.NameToProperty(field.name), StringToValue(value, field.type));
auto old_node_property = node.SetProperty(acc->NameToProperty(field.name), StringToValue(value, field.type));
if (!old_node_property.HasValue()) throw LoadException("Couldn't add property '{}' to the node", field.name);
if (!old_node_property->IsNull()) throw LoadException("The property '{}' already exists", field.name);
}
}
for (const auto &label : additional_labels) {
auto node_label = node.AddLabel(acc.NameToLabel(label));
auto node_label = node.AddLabel(acc->NameToLabel(label));
if (!node_label.HasValue()) throw LoadException("Couldn't add label '{}' to the node", label);
if (!*node_label) throw LoadException("The label '{}' already exists", label);
}
if (acc.Commit().HasError()) throw LoadException("Couldn't store the node");
if (acc->Commit().HasError()) throw LoadException("Couldn't store the node");
}
void ProcessNodes(memgraph::storage::Storage *store, const std::string &nodes_path,
@ -567,16 +568,16 @@ void ProcessRelationshipsRow(memgraph::storage::Storage *store, const std::vecto
if (!relationship_type) throw LoadException("Relationship TYPE must be set");
auto acc = store->Access();
auto from_node = acc.FindVertex(*start_id, memgraph::storage::View::NEW);
auto from_node = acc->FindVertex(*start_id, memgraph::storage::View::NEW);
if (!from_node) throw LoadException("From node must be in the storage");
auto to_node = acc.FindVertex(*end_id, memgraph::storage::View::NEW);
auto to_node = acc->FindVertex(*end_id, memgraph::storage::View::NEW);
if (!to_node) throw LoadException("To node must be in the storage");
auto relationship = acc.CreateEdge(&*from_node, &*to_node, acc.NameToEdgeType(*relationship_type));
auto relationship = acc->CreateEdge(&from_node.value(), &to_node.value(), acc->NameToEdgeType(*relationship_type));
if (!relationship.HasValue()) throw LoadException("Couldn't create the relationship");
for (const auto &property : properties) {
auto ret = relationship->SetProperty(acc.NameToProperty(property.first), property.second);
auto ret = relationship.GetValue().SetProperty(acc->NameToProperty(property.first), property.second);
if (!ret.HasValue()) {
if (ret.GetError() != memgraph::storage::Error::PROPERTIES_DISABLED) {
throw LoadException("Couldn't add property '{}' to the relationship", property.first);
@ -589,7 +590,7 @@ void ProcessRelationshipsRow(memgraph::storage::Storage *store, const std::vecto
}
}
if (acc.Commit().HasError()) throw LoadException("Couldn't store the relationship");
if (acc->Commit().HasError()) throw LoadException("Couldn't store the relationship");
}
void ProcessRelationships(memgraph::storage::Storage *store, const std::string &relationships_path,
@ -699,13 +700,13 @@ int main(int argc, char *argv[]) {
}
std::unordered_map<NodeId, memgraph::storage::Gid> node_id_map;
memgraph::storage::Storage store{{
std::unique_ptr<memgraph::storage::Storage> store{new memgraph::storage::InMemoryStorage{{
.items = {.properties_on_edges = FLAGS_storage_properties_on_edges},
.durability = {.storage_directory = FLAGS_data_directory,
.recover_on_startup = false,
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::DISABLED,
.snapshot_on_exit = true},
}};
}}};
memgraph::utils::Timer load_timer;
@ -715,7 +716,7 @@ int main(int argc, char *argv[]) {
std::optional<std::vector<Field>> header;
for (const auto &nodes_file : files) {
spdlog::info("Loading {}", nodes_file);
ProcessNodes(&store, nodes_file, &header, &node_id_map, additional_labels);
ProcessNodes(store.get(), nodes_file, &header, &node_id_map, additional_labels);
}
}
@ -725,7 +726,7 @@ int main(int argc, char *argv[]) {
std::optional<std::vector<Field>> header;
for (const auto &relationships_file : files) {
spdlog::info("Loading {}", relationships_file);
ProcessRelationships(&store, relationships_file, type, &header, node_id_map);
ProcessRelationships(store.get(), relationships_file, type, &header, node_id_map);
}
}

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

View File

@ -17,9 +17,11 @@
#include <cppitertools/imap.hpp>
#include "query/exceptions.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/storage_mode.hpp"
#include "utils/pmr/unordered_set.hpp"
#include "utils/variant_helpers.hpp"
@ -347,6 +349,10 @@ class DbAccessor final {
VertexAccessor InsertVertex() { return VertexAccessor(accessor_->CreateVertex()); }
void PrefetchOutEdges(const VertexAccessor &vertex) const { accessor_->PrefetchOutEdges(vertex.impl_); }
void PrefetchInEdges(const VertexAccessor &vertex) const { accessor_->PrefetchInEdges(vertex.impl_); }
storage::Result<EdgeAccessor> InsertEdge(VertexAccessor *from, VertexAccessor *to,
const storage::EdgeTypeId &edge_type) {
auto maybe_edge = accessor_->CreateEdge(&from->impl_, &to->impl_, edge_type);
@ -372,6 +378,8 @@ class DbAccessor final {
VertexAccessor *vertex_accessor) {
using ReturnType = std::pair<VertexAccessor, std::vector<EdgeAccessor>>;
accessor_->PrefetchOutEdges(vertex_accessor->impl_);
accessor_->PrefetchInEdges(vertex_accessor->impl_);
auto res = accessor_->DetachDeleteVertex(&vertex_accessor->impl_);
if (res.HasError()) {
return res.GetError();
@ -424,6 +432,8 @@ class DbAccessor final {
void Abort() { accessor_->Abort(); }
storage::StorageMode GetStorageMode() const { return accessor_->GetCreationStorageMode(); }
bool LabelIndexExists(storage::LabelId label) const { return accessor_->LabelIndexExists(label); }
bool LabelPropertyIndexExists(storage::LabelId label, storage::PropertyId prop) const {
@ -508,6 +518,10 @@ class SubgraphDbAccessor final {
const std::string &EdgeTypeToName(storage::EdgeTypeId type) const;
void PrefetchOutEdges(const SubgraphVertexAccessor &vertex) const { db_accessor_.PrefetchOutEdges(vertex.impl_); }
void PrefetchInEdges(const SubgraphVertexAccessor &vertex) const { db_accessor_.PrefetchInEdges(vertex.impl_); }
storage::Result<std::optional<EdgeAccessor>> RemoveEdge(EdgeAccessor *edge);
storage::Result<EdgeAccessor> InsertEdge(SubgraphVertexAccessor *from, SubgraphVertexAccessor *to,

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
@ -482,6 +482,7 @@ PullPlanDump::PullChunk PullPlanDump::CreateEdgePullChunk() {
// If we have a saved iterable from a previous pull
// we need to use the same iterable
if (!maybe_edge_iterable) {
dba_->PrefetchOutEdges(vertex);
maybe_edge_iterable = std::make_shared<EdgeAccessorIterable>(vertex.OutEdges(storage::View::OLD));
}
auto &maybe_edges = *maybe_edge_iterable;

View File

@ -175,18 +175,34 @@ class ReplicationModificationInMulticommandTxException : public QueryException {
: QueryException("Replication clause not allowed in multicommand transactions.") {}
};
class ReplicationDisabledOnDiskStorage : public QueryException {
public:
ReplicationDisabledOnDiskStorage() : QueryException("Replication not support with disk storage. ") {}
};
class LockPathModificationInMulticommandTxException : public QueryException {
public:
LockPathModificationInMulticommandTxException()
: QueryException("Lock path query not allowed in multicommand transactions.") {}
};
class LockPathDisabledOnDiskStorage : public QueryException {
public:
LockPathDisabledOnDiskStorage()
: QueryException("Lock path disabled on disk storage since all data is already persisted. ") {}
};
class FreeMemoryModificationInMulticommandTxException : public QueryException {
public:
FreeMemoryModificationInMulticommandTxException()
: QueryException("Free memory query not allowed in multicommand transactions.") {}
};
class FreeMemoryDisabledOnDiskStorage : public QueryException {
public:
FreeMemoryDisabledOnDiskStorage() : QueryException("Free memory does nothing when using disk storage. ") {}
};
class ShowConfigModificationInMulticommandTxException : public QueryException {
public:
ShowConfigModificationInMulticommandTxException()
@ -232,6 +248,11 @@ class CreateSnapshotInMulticommandTxException final : public QueryException {
: QueryException("Snapshot cannot be created in multicommand transactions.") {}
};
class CreateSnapshotDisabledOnDiskStorage final : public QueryException {
public:
CreateSnapshotDisabledOnDiskStorage() : QueryException("Data is already persisted when using disk storage. ") {}
};
class SettingConfigInMulticommandTxException final : public QueryException {
public:
SettingConfigInMulticommandTxException()
@ -264,4 +285,14 @@ class TransactionQueueInMulticommandTxException : public QueryException {
: QueryException("Transaction queue queries not allowed in multicommand transactions.") {}
};
class IndexPersistenceException : public QueryException {
public:
IndexPersistenceException() : QueryException("Persisting index on disk failed.") {}
};
class ConstraintsPersistenceException : public QueryException {
public:
ConstraintsPersistenceException() : QueryException("Persisting constraints on disk failed.") {}
};
} // namespace memgraph::query

View File

@ -3143,7 +3143,7 @@ class StorageModeQuery : public memgraph::query::Query {
static const utils::TypeInfo kType;
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
enum class StorageMode { IN_MEMORY_TRANSACTIONAL, IN_MEMORY_ANALYTICAL };
enum class StorageMode { IN_MEMORY_TRANSACTIONAL, IN_MEMORY_ANALYTICAL, ON_DISK_TRANSACTIONAL };
StorageModeQuery() = default;

View File

@ -504,7 +504,10 @@ antlrcpp::Any CypherMainVisitor::visitStorageModeQuery(MemgraphCypher::StorageMo
if (mode->IN_MEMORY_ANALYTICAL()) {
return StorageModeQuery::StorageMode::IN_MEMORY_ANALYTICAL;
}
return StorageModeQuery::StorageMode::IN_MEMORY_TRANSACTIONAL;
if (mode->IN_MEMORY_TRANSACTIONAL()) {
return StorageModeQuery::StorageMode::IN_MEMORY_TRANSACTIONAL;
}
return StorageModeQuery::StorageMode::ON_DISK_TRANSACTIONAL;
});
query_ = storage_mode_query;

View File

@ -365,7 +365,7 @@ isolationLevelScope : GLOBAL | SESSION | NEXT ;
isolationLevelQuery : SET isolationLevelScope TRANSACTION ISOLATION LEVEL isolationLevel ;
storageMode : IN_MEMORY_ANALYTICAL | IN_MEMORY_TRANSACTIONAL ;
storageMode : IN_MEMORY_ANALYTICAL | IN_MEMORY_TRANSACTIONAL | ON_DISK_TRANSACTIONAL ;
storageModeQuery : STORAGE MODE storageMode ;

View File

@ -85,6 +85,7 @@ MODULE_WRITE : M O D U L E UNDERSCORE W R I T E ;
NEXT : N E X T ;
NO : N O ;
NOTHING : N O T H I N G ;
ON_DISK_TRANSACTIONAL : O N UNDERSCORE D I S K UNDERSCORE T R A N S A C T I O N A L ;
NULLIF : N U L L I F ;
PASSWORD : P A S S W O R D ;
PORT : P O R T ;

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
@ -467,6 +467,8 @@ TypedValue Degree(const TypedValue *args, int64_t nargs, const FunctionContext &
FType<Or<Null, Vertex>>("degree", args, nargs);
if (args[0].IsNull()) return TypedValue(ctx.memory);
const auto &vertex = args[0].ValueVertex();
ctx.db_accessor->PrefetchInEdges(vertex);
ctx.db_accessor->PrefetchOutEdges(vertex);
size_t out_degree = UnwrapDegreeResult(vertex.OutDegree(ctx.view));
size_t in_degree = UnwrapDegreeResult(vertex.InDegree(ctx.view));
return TypedValue(static_cast<int64_t>(out_degree + in_degree), ctx.memory);
@ -476,6 +478,7 @@ TypedValue InDegree(const TypedValue *args, int64_t nargs, const FunctionContext
FType<Or<Null, Vertex>>("inDegree", args, nargs);
if (args[0].IsNull()) return TypedValue(ctx.memory);
const auto &vertex = args[0].ValueVertex();
ctx.db_accessor->PrefetchInEdges(vertex);
size_t in_degree = UnwrapDegreeResult(vertex.InDegree(ctx.view));
return TypedValue(static_cast<int64_t>(in_degree), ctx.memory);
}
@ -484,6 +487,7 @@ TypedValue OutDegree(const TypedValue *args, int64_t nargs, const FunctionContex
FType<Or<Null, Vertex>>("outDegree", args, nargs);
if (args[0].IsNull()) return TypedValue(ctx.memory);
const auto &vertex = args[0].ValueVertex();
ctx.db_accessor->PrefetchOutEdges(vertex);
size_t out_degree = UnwrapDegreeResult(vertex.OutDegree(ctx.view));
return TypedValue(static_cast<int64_t>(out_degree), ctx.memory);
}

View File

@ -58,21 +58,25 @@
#include "query/trigger.hpp"
#include "query/typed_value.hpp"
#include "spdlog/spdlog.h"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/storage_mode.hpp"
#include "utils/algorithm.hpp"
#include "utils/build_info.hpp"
#include "utils/event_counter.hpp"
#include "utils/event_histogram.hpp"
#include "utils/exceptions.hpp"
#include "utils/file.hpp"
#include "utils/flag_validation.hpp"
#include "utils/likely.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
#include "utils/memory_tracker.hpp"
#include "utils/message.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/readable_size.hpp"
#include "utils/settings.hpp"
@ -167,8 +171,9 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
/// @throw QueryRuntimeException if an error ocurred.
void SetReplicationRole(ReplicationQuery::ReplicationRole replication_role, std::optional<int64_t> port) override {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(db_);
if (replication_role == ReplicationQuery::ReplicationRole::MAIN) {
if (!db_->SetMainReplicationRole()) {
if (!mem_storage->SetMainReplicationRole()) {
throw QueryRuntimeException("Couldn't set role to main!");
}
}
@ -176,8 +181,9 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
if (!port || *port < 0 || *port > std::numeric_limits<uint16_t>::max()) {
throw QueryRuntimeException("Port number invalid!");
}
if (!db_->SetReplicaRole(
io::network::Endpoint(storage::replication::kDefaultReplicationServerIp, static_cast<uint16_t>(*port)))) {
if (!mem_storage->SetReplicaRole(
io::network::Endpoint(storage::replication::kDefaultReplicationServerIp, static_cast<uint16_t>(*port)),
storage::replication::ReplicationServerConfig{})) {
throw QueryRuntimeException("Couldn't set role to replica!");
}
}
@ -185,7 +191,7 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
/// @throw QueryRuntimeException if an error ocurred.
ReplicationQuery::ReplicationRole ShowReplicationRole() const override {
switch (db_->GetReplicationRole()) {
switch (static_cast<storage::InMemoryStorage *>(db_)->GetReplicationRole()) {
case storage::replication::ReplicationRole::MAIN:
return ReplicationQuery::ReplicationRole::MAIN;
case storage::replication::ReplicationRole::REPLICA:
@ -198,7 +204,8 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
void RegisterReplica(const std::string &name, const std::string &socket_address,
const ReplicationQuery::SyncMode sync_mode,
const std::chrono::seconds replica_check_frequency) override {
if (db_->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(db_);
if (mem_storage->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
// replica can't register another replica
throw QueryRuntimeException("Replica can't register another replica!");
}
@ -223,9 +230,9 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
io::network::Endpoint::ParseSocketOrIpAddress(socket_address, storage::replication::kDefaultReplicationPort);
if (maybe_ip_and_port) {
auto [ip, port] = *maybe_ip_and_port;
auto ret = db_->RegisterReplica(name, {std::move(ip), port}, repl_mode,
storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
{.replica_check_frequency = replica_check_frequency, .ssl = std::nullopt});
auto ret = mem_storage->RegisterReplica(
name, {std::move(ip), port}, repl_mode, storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
{.replica_check_frequency = replica_check_frequency, .ssl = std::nullopt});
if (ret.HasError()) {
throw QueryRuntimeException(fmt::format("Couldn't register replica '{}'!", name));
}
@ -236,23 +243,26 @@ class ReplQueryHandler final : public query::ReplicationQueryHandler {
/// @throw QueryRuntimeException if an error ocurred.
void DropReplica(const std::string &replica_name) override {
if (db_->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(db_);
if (mem_storage->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
// replica can't unregister a replica
throw QueryRuntimeException("Replica can't unregister a replica!");
}
if (!db_->UnregisterReplica(replica_name)) {
if (!mem_storage->UnregisterReplica(replica_name)) {
throw QueryRuntimeException(fmt::format("Couldn't unregister the replica '{}'", replica_name));
}
}
using Replica = ReplicationQueryHandler::Replica;
std::vector<Replica> ShowReplicas() const override {
if (db_->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(db_);
if (mem_storage->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) {
// replica can't show registered replicas (it shouldn't have any)
throw QueryRuntimeException("Replica can't show registered replicas (it shouldn't have any)!");
}
auto repl_infos = db_->ReplicasInfo();
auto repl_infos = mem_storage->ReplicasInfo();
std::vector<Replica> replicas;
replicas.reserve(repl_infos.size());
@ -534,7 +544,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
notifications->emplace_back(SeverityLevel::WARNING, NotificationCode::REPLICA_PORT_WARNING,
"Be careful the replication port must be different from the memgraph port!");
}
callback.fn = [handler = ReplQueryHandler{interpreter_context->db}, role = repl_query->role_,
callback.fn = [handler = ReplQueryHandler{interpreter_context->db.get()}, role = repl_query->role_,
maybe_port]() mutable {
handler.SetReplicationRole(role, maybe_port);
return std::vector<std::vector<TypedValue>>();
@ -547,7 +557,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
}
case ReplicationQuery::Action::SHOW_REPLICATION_ROLE: {
callback.header = {"replication role"};
callback.fn = [handler = ReplQueryHandler{interpreter_context->db}] {
callback.fn = [handler = ReplQueryHandler{interpreter_context->db.get()}] {
auto mode = handler.ShowReplicationRole();
switch (mode) {
case ReplicationQuery::ReplicationRole::MAIN: {
@ -566,7 +576,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
auto socket_address = repl_query->socket_address_->Accept(evaluator);
const auto replica_check_frequency = interpreter_context->config.replication_replica_check_frequency;
callback.fn = [handler = ReplQueryHandler{interpreter_context->db}, name, socket_address, sync_mode,
callback.fn = [handler = ReplQueryHandler{interpreter_context->db.get()}, name, socket_address, sync_mode,
replica_check_frequency]() mutable {
handler.RegisterReplica(name, std::string(socket_address.ValueString()), sync_mode, replica_check_frequency);
return std::vector<std::vector<TypedValue>>();
@ -578,7 +588,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
case ReplicationQuery::Action::DROP_REPLICA: {
const auto &name = repl_query->replica_name_;
callback.fn = [handler = ReplQueryHandler{interpreter_context->db}, name]() mutable {
callback.fn = [handler = ReplQueryHandler{interpreter_context->db.get()}, name]() mutable {
handler.DropReplica(name);
return std::vector<std::vector<TypedValue>>();
};
@ -591,7 +601,8 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
callback.header = {
"name", "socket_address", "sync_mode", "current_timestamp_of_replica", "number_of_timestamp_behind_master",
"state"};
callback.fn = [handler = ReplQueryHandler{interpreter_context->db}, replica_nfields = callback.header.size()] {
callback.fn = [handler = ReplQueryHandler{interpreter_context->db.get()},
replica_nfields = callback.header.size()] {
const auto &replicas = handler.ShowReplicas();
auto typed_replicas = std::vector<std::vector<TypedValue>>{};
typed_replicas.reserve(replicas.size());
@ -1188,11 +1199,38 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *strea
}
using RWType = plan::ReadWriteTypeChecker::RWType;
bool IsWriteQueryOnMainMemoryReplica(storage::Storage *storage,
const query::plan::ReadWriteTypeChecker::RWType query_type) {
if (auto storage_mode = storage->GetStorageMode(); storage_mode == storage::StorageMode::IN_MEMORY_ANALYTICAL ||
storage_mode == storage::StorageMode::IN_MEMORY_TRANSACTIONAL) {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(storage);
return (mem_storage->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA) &&
(query_type == RWType::W || query_type == RWType::RW);
}
return false;
}
} // namespace
InterpreterContext::InterpreterContext(storage::Storage *db, const InterpreterConfig config,
InterpreterContext::InterpreterContext(const storage::Config storage_config, const InterpreterConfig interpreter_config,
const std::filesystem::path &data_directory)
: db(db), trigger_store(data_directory / "triggers"), config(config), streams{this, data_directory / "streams"} {}
: trigger_store(data_directory / "triggers"),
config(interpreter_config),
streams{this, data_directory / "streams"} {
if (utils::DirExists(storage_config.disk.main_storage_directory)) {
db = std::make_unique<storage::DiskStorage>(storage_config);
} else {
db = std::make_unique<storage::InMemoryStorage>(storage_config);
}
}
InterpreterContext::InterpreterContext(std::unique_ptr<storage::Storage> db, InterpreterConfig interpreter_config,
const std::filesystem::path &data_directory)
: db(std::move(db)),
trigger_store(data_directory / "triggers"),
config(interpreter_config),
streams{this, data_directory / "streams"} {}
Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_context_(interpreter_context) {
MG_ASSERT(interpreter_context_, "Interpreter context must not be NULL");
@ -1216,8 +1254,7 @@ PreparedQuery Interpreter::PrepareTransactionQuery(std::string_view query_upper,
expect_rollback_ = false;
metadata_ = GenOptional(metadata);
db_accessor_ =
std::make_unique<storage::Storage::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
db_accessor_ = interpreter_context_->db->Access(GetIsolationLevelOverride());
execution_db_accessor_.emplace(db_accessor_.get());
transaction_status_.store(TransactionStatus::ACTIVE, std::memory_order_release);
@ -1845,6 +1882,8 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans
index_notification.code = NotificationCode::EXISTENT_INDEX;
index_notification.title = fmt::format("Index on label {} on properties {} already exists.",
label_name, properties_stringified);
} else if constexpr (std::is_same_v<ErrorType, storage::IndexPersistenceError>) {
throw IndexPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -1880,6 +1919,8 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans
index_notification.code = NotificationCode::NONEXISTENT_INDEX;
index_notification.title = fmt::format("Index on label {} on properties {} doesn't exist.",
label_name, properties_stringified);
} else if constexpr (std::is_same_v<ErrorType, storage::IndexPersistenceError>) {
throw IndexPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -1947,6 +1988,10 @@ PreparedQuery PrepareReplicationQuery(ParsedQuery parsed_query, bool in_explicit
throw ReplicationModificationInMulticommandTxException();
}
if (interpreter_context->db->GetStorageMode() == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
throw ReplicationDisabledOnDiskStorage();
}
auto *replication_query = utils::Downcast<ReplicationQuery>(parsed_query.query);
auto callback =
HandleReplicationQuery(replication_query, parsed_query.parameters, interpreter_context, dba, notifications);
@ -1969,11 +2014,15 @@ PreparedQuery PrepareReplicationQuery(ParsedQuery parsed_query, bool in_explicit
}
PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
InterpreterContext *interpreter_context, DbAccessor *dba) {
InterpreterContext *interpreter_context) {
if (in_explicit_transaction) {
throw LockPathModificationInMulticommandTxException();
}
if (interpreter_context->db->GetStorageMode() == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
throw LockPathDisabledOnDiskStorage();
}
auto *lock_path_query = utils::Downcast<LockPathQuery>(parsed_query.query);
return PreparedQuery{
@ -1981,12 +2030,13 @@ PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, bool in_explicit_tr
std::move(parsed_query.required_privileges),
[interpreter_context, action = lock_path_query->action_](
AnyStream *stream, std::optional<int> n) -> std::optional<QueryHandlerResult> {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(interpreter_context->db.get());
std::vector<std::vector<TypedValue>> status;
std::string res;
switch (action) {
case LockPathQuery::Action::LOCK_PATH: {
const auto lock_success = interpreter_context->db->LockPath();
const auto lock_success = mem_storage->LockPath();
if (lock_success.HasError()) [[unlikely]] {
throw QueryRuntimeException("Failed to lock the data directory");
}
@ -1994,7 +2044,7 @@ PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, bool in_explicit_tr
break;
}
case LockPathQuery::Action::UNLOCK_PATH: {
const auto unlock_success = interpreter_context->db->UnlockPath();
const auto unlock_success = mem_storage->UnlockPath();
if (unlock_success.HasError()) [[unlikely]] {
throw QueryRuntimeException("Failed to unlock the data directory");
}
@ -2002,7 +2052,7 @@ PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, bool in_explicit_tr
break;
}
case LockPathQuery::Action::STATUS: {
const auto locked_status = interpreter_context->db->IsPathLocked();
const auto locked_status = mem_storage->IsPathLocked();
if (locked_status.HasError()) [[unlikely]] {
throw QueryRuntimeException("Failed to access the data directory");
}
@ -2027,6 +2077,10 @@ PreparedQuery PrepareFreeMemoryQuery(ParsedQuery parsed_query, bool in_explicit_
throw FreeMemoryModificationInMulticommandTxException();
}
if (interpreter_context->db->GetStorageMode() == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
throw FreeMemoryDisabledOnDiskStorage();
}
return PreparedQuery{
{},
std::move(parsed_query.required_privileges),
@ -2240,9 +2294,23 @@ constexpr auto ToStorageMode(const StorageModeQuery::StorageMode storage_mode) n
return storage::StorageMode::IN_MEMORY_TRANSACTIONAL;
case StorageModeQuery::StorageMode::IN_MEMORY_ANALYTICAL:
return storage::StorageMode::IN_MEMORY_ANALYTICAL;
case StorageModeQuery::StorageMode::ON_DISK_TRANSACTIONAL:
return storage::StorageMode::ON_DISK_TRANSACTIONAL;
}
}
bool SwitchingFromInMemoryToDisk(storage::StorageMode current_mode, storage::StorageMode next_mode) {
return (current_mode == storage::StorageMode::IN_MEMORY_TRANSACTIONAL ||
current_mode == storage::StorageMode::IN_MEMORY_ANALYTICAL) &&
next_mode == storage::StorageMode::ON_DISK_TRANSACTIONAL;
}
bool SwitchingFromDiskToInMemory(storage::StorageMode current_mode, storage::StorageMode next_mode) {
return current_mode == storage::StorageMode::ON_DISK_TRANSACTIONAL &&
(next_mode == storage::StorageMode::IN_MEMORY_TRANSACTIONAL ||
next_mode == storage::StorageMode::IN_MEMORY_ANALYTICAL);
}
PreparedQuery PrepareIsolationLevelQuery(ParsedQuery parsed_query, const bool in_explicit_transaction,
InterpreterContext *interpreter_context, Interpreter *interpreter) {
if (in_explicit_transaction) {
@ -2284,6 +2352,59 @@ PreparedQuery PrepareIsolationLevelQuery(ParsedQuery parsed_query, const bool in
RWType::NONE};
}
Callback SwitchMemoryDevice(storage::StorageMode current_mode, storage::StorageMode requested_mode,
InterpreterContext *interpreter_context) {
Callback callback;
callback.fn = [current_mode, requested_mode, interpreter_context]() mutable {
if (current_mode == requested_mode) {
return std::vector<std::vector<TypedValue>>();
}
if (SwitchingFromDiskToInMemory(current_mode, requested_mode)) {
throw utils::BasicException(
"You cannot switch from on-disk storage to in-memory storage while the database is running. "
"Please delete your data directory and restart the database. Once restarted, the Memgraph will automatically "
"start to use in-memory storage.");
}
if (SwitchingFromInMemoryToDisk(current_mode, requested_mode)) {
std::unique_lock main_guard{interpreter_context->db->main_lock_};
if (auto vertex_cnt_approx = interpreter_context->db->GetInfo().vertex_count; vertex_cnt_approx > 0) {
throw utils::BasicException(
"You cannot switch from in-memory storage to on-disk storage when the database "
"contains data. Please delete all entries from the database, run FREE MEMORY and then repeat this "
"query. ");
}
main_guard.unlock();
if (interpreter_context->interpreters->size() > 1) {
throw utils::BasicException(
"You cannot switch from in-memory storage to on-disk storage when there are "
"multiple sessions active. Please close all other sessions and try again. If you are using Memgraph Lab, "
"please start mgconsole "
"and run the STORAGE MODE ON_DISK_TRANSACTIONAL there first. Memgraph Lab is using multiple sessions to "
"run queries in parallel "
"so it is currently impossible to switch to on-disk storage while the Lab is running. After you switch "
"from the mgconsole, you can "
"continue to use Memgraph Lab as you wish.");
}
auto db_config = interpreter_context->db->config_;
interpreter_context->db = std::make_unique<memgraph::storage::DiskStorage>(db_config);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
}
bool ActiveTransactionsExist(InterpreterContext *interpreter_context) {
bool exists_active_transaction = interpreter_context->interpreters.WithLock([](const auto &interpreters_) {
return std::any_of(interpreters_.begin(), interpreters_.end(), [](const auto &interpreter) {
return interpreter->transaction_status_.load() != TransactionStatus::IDLE;
});
});
return exists_active_transaction;
}
PreparedQuery PrepareStorageModeQuery(ParsedQuery parsed_query, const bool in_explicit_transaction,
InterpreterContext *interpreter_context) {
if (in_explicit_transaction) {
@ -2292,23 +2413,26 @@ PreparedQuery PrepareStorageModeQuery(ParsedQuery parsed_query, const bool in_ex
auto *storage_mode_query = utils::Downcast<StorageModeQuery>(parsed_query.query);
MG_ASSERT(storage_mode_query);
const auto storage_mode = ToStorageMode(storage_mode_query->storage_mode_);
const auto requested_mode = ToStorageMode(storage_mode_query->storage_mode_);
auto current_mode = interpreter_context->db->GetStorageMode();
auto exists_active_transaction = interpreter_context->interpreters.WithLock([](const auto &interpreters_) {
return std::any_of(interpreters_.begin(), interpreters_.end(), [](const auto &interpreter) {
return interpreter->transaction_status_.load() != TransactionStatus::IDLE;
});
});
if (exists_active_transaction) {
spdlog::info(
"Storage mode will be modified when there are no other active transactions. Check the status of the "
"transactions using 'SHOW TRANSACTIONS' query and ensure no other transactions are active.");
std::function<void()> callback;
if (current_mode == storage::StorageMode::ON_DISK_TRANSACTIONAL ||
requested_mode == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
callback = SwitchMemoryDevice(current_mode, requested_mode, interpreter_context).fn;
} else {
if (ActiveTransactionsExist(interpreter_context)) {
spdlog::info(
"Storage mode will be modified when there are no other active transactions. Check the status of the "
"transactions using 'SHOW TRANSACTIONS' query and ensure no other transactions are active.");
}
callback = [requested_mode, interpreter_context]() -> std::function<void()> {
return [interpreter_context, requested_mode] { interpreter_context->db->SetStorageMode(requested_mode); };
}();
}
auto callback = [storage_mode, interpreter_context]() -> std::function<void()> {
return [interpreter_context, storage_mode] { interpreter_context->db->SetStorageMode(storage_mode); };
}();
return PreparedQuery{{},
std::move(parsed_query.required_privileges),
[callback = std::move(callback)](AnyStream * /*stream*/,
@ -2325,20 +2449,25 @@ PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_expli
throw CreateSnapshotInMulticommandTxException();
}
if (interpreter_context->db->GetStorageMode() == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
throw CreateSnapshotDisabledOnDiskStorage();
}
return PreparedQuery{
{},
std::move(parsed_query.required_privileges),
[interpreter_context](AnyStream *stream, std::optional<int> n) -> std::optional<QueryHandlerResult> {
if (auto maybe_error = interpreter_context->db->CreateSnapshot({}); maybe_error.HasError()) {
auto *mem_storage = static_cast<storage::InMemoryStorage *>(interpreter_context->db.get());
if (auto maybe_error = mem_storage->CreateSnapshot({}); maybe_error.HasError()) {
switch (maybe_error.GetError()) {
case storage::Storage::CreateSnapshotError::DisabledForReplica:
case storage::InMemoryStorage::CreateSnapshotError::DisabledForReplica:
throw utils::BasicException(
"Failed to create a snapshot. Replica instances are not allowed to create them.");
case storage::Storage::CreateSnapshotError::DisabledForAnalyticsPeriodicCommit:
case storage::InMemoryStorage::CreateSnapshotError::DisabledForAnalyticsPeriodicCommit:
spdlog::warn(utils::MessageWithLink("Periodic snapshots are disabled for analytical mode.",
"https://memgr.ph/replication"));
break;
case storage::Storage::CreateSnapshotError::ReachedMaxNumTries:
case storage::InMemoryStorage::CreateSnapshotError::ReachedMaxNumTries:
spdlog::warn("Failed to create snapshot. Reached max number of tries. Please contact support");
break;
}
@ -2586,7 +2715,7 @@ PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transa
case InfoQuery::InfoType::INDEX:
header = {"index type", "label", "property"};
handler = [interpreter_context] {
auto *db = interpreter_context->db;
auto *db = interpreter_context->db.get();
auto info = db->ListAllIndices();
std::vector<std::vector<TypedValue>> results;
results.reserve(info.label.size() + info.label_property.size());
@ -2603,7 +2732,7 @@ PreparedQuery PrepareInfoQuery(ParsedQuery parsed_query, bool in_explicit_transa
case InfoQuery::InfoType::CONSTRAINT:
header = {"constraint type", "label", "properties"};
handler = [interpreter_context] {
auto *db = interpreter_context->db;
auto *db = interpreter_context->db.get();
auto info = db->ListAllConstraints();
std::vector<std::vector<TypedValue>> results;
results.reserve(info.existence.size() + info.unique.size());
@ -2690,7 +2819,7 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
handler = [interpreter_context, label, label_name = constraint_query->constraint_.label.name,
properties_stringified = std::move(properties_stringified),
properties = std::move(properties)](Notification &constraint_notification) {
auto maybe_constraint_error = interpreter_context->db->CreateExistenceConstraint(label, properties[0]);
auto maybe_constraint_error = interpreter_context->db->CreateExistenceConstraint(label, properties[0], {});
if (maybe_constraint_error.HasError()) {
const auto &error = maybe_constraint_error.GetError();
@ -2713,9 +2842,12 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
properties_stringified);
} else if constexpr (std::is_same_v<ErrorType, storage::ReplicationError>) {
throw ReplicationException(
"At least one SYNC replica has not confirmed the creation of the EXISTS constraint on label "
"At least one SYNC replica has not confirmed the creation of the EXISTS constraint on "
"label "
"{} on properties {}.",
label_name, properties_stringified);
} else if constexpr (std::is_same_v<ErrorType, storage::ConstraintsPersistenceError>) {
throw ConstraintsPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -2738,11 +2870,12 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
handler = [interpreter_context, label, label_name = constraint_query->constraint_.label.name,
properties_stringified = std::move(properties_stringified),
property_set = std::move(property_set)](Notification &constraint_notification) {
auto maybe_constraint_error = interpreter_context->db->CreateUniqueConstraint(label, property_set);
auto maybe_constraint_error = interpreter_context->db->CreateUniqueConstraint(label, property_set, {});
if (maybe_constraint_error.HasError()) {
const auto &error = maybe_constraint_error.GetError();
std::visit(
[&interpreter_context, &label_name, &properties_stringified]<typename T>(T &&arg) {
[&interpreter_context, &label_name, &properties_stringified,
&constraint_notification]<typename T>(T &&arg) {
using ErrorType = std::remove_cvref_t<T>;
if constexpr (std::is_same_v<ErrorType, storage::ConstraintViolation>) {
auto &violation = arg;
@ -2756,10 +2889,18 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
"Unable to create unique constraint :{}({}), because an "
"existing node violates it.",
violation_label_name, property_names_stream.str());
} else if constexpr (std::is_same_v<ErrorType, storage::ConstraintDefinitionError>) {
constraint_notification.code = NotificationCode::EXISTENT_CONSTRAINT;
constraint_notification.title =
fmt::format("Constraint UNIQUE on label {} and properties {} couldn't be created.",
label_name, properties_stringified);
} else if constexpr (std::is_same_v<ErrorType, storage::ReplicationError>) {
throw ReplicationException(fmt::format(
"At least one SYNC replica has not confirmed the creation of the UNIQUE constraint: {}({}).",
label_name, properties_stringified));
throw ReplicationException(
fmt::format("At least one SYNC replica has not confirmed the creation of the UNIQUE "
"constraint: {}({}).",
label_name, properties_stringified));
} else if constexpr (std::is_same_v<ErrorType, storage::ConstraintsPersistenceError>) {
throw ConstraintsPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -2805,7 +2946,7 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
handler = [interpreter_context, label, label_name = constraint_query->constraint_.label.name,
properties_stringified = std::move(properties_stringified),
properties = std::move(properties)](Notification &constraint_notification) {
auto maybe_constraint_error = interpreter_context->db->DropExistenceConstraint(label, properties[0]);
auto maybe_constraint_error = interpreter_context->db->DropExistenceConstraint(label, properties[0], {});
if (maybe_constraint_error.HasError()) {
const auto &error = maybe_constraint_error.GetError();
std::visit(
@ -2821,6 +2962,8 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
fmt::format("At least one SYNC replica has not confirmed the dropping of the EXISTS "
"constraint on label {} on properties {}.",
label_name, properties_stringified));
} else if constexpr (std::is_same_v<ErrorType, storage::ConstraintsPersistenceError>) {
throw ConstraintsPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -2844,7 +2987,7 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
handler = [interpreter_context, label, label_name = constraint_query->constraint_.label.name,
properties_stringified = std::move(properties_stringified),
property_set = std::move(property_set)](Notification &constraint_notification) {
auto maybe_constraint_error = interpreter_context->db->DropUniqueConstraint(label, property_set);
auto maybe_constraint_error = interpreter_context->db->DropUniqueConstraint(label, property_set, {});
if (maybe_constraint_error.HasError()) {
const auto &error = maybe_constraint_error.GetError();
std::visit(
@ -2855,6 +2998,8 @@ PreparedQuery PrepareConstraintQuery(ParsedQuery parsed_query, bool in_explicit_
fmt::format("At least one SYNC replica has not confirmed the dropping of the UNIQUE "
"constraint on label {} on properties {}.",
label_name, properties_stringified));
} else if constexpr (std::is_same_v<ErrorType, storage::ConstraintsPersistenceError>) {
throw ConstraintsPersistenceException();
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -3021,8 +3166,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
utils::Downcast<TriggerQuery>(parsed_query.query) || utils::Downcast<AnalyzeGraphQuery>(parsed_query.query) ||
utils::Downcast<TransactionQueueQuery>(parsed_query.query))) {
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveTransactions);
db_accessor_ =
std::make_unique<storage::Storage::Accessor>(interpreter_context_->db->Access(GetIsolationLevelOverride()));
db_accessor_ = interpreter_context_->db->Access(GetIsolationLevelOverride());
execution_db_accessor_.emplace(db_accessor_.get());
transaction_status_.store(TransactionStatus::ACTIVE, std::memory_order_release);
@ -3066,7 +3210,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
&*execution_db_accessor_, &query_execution->execution_memory_with_exception, username, &transaction_status_);
} else if (utils::Downcast<InfoQuery>(parsed_query.query)) {
prepared_query = PrepareInfoQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->summary,
interpreter_context_, interpreter_context_->db,
interpreter_context_, interpreter_context_->db.get(),
&query_execution->execution_memory_with_exception, interpreter_isolation_level,
next_transaction_isolation_level);
} else if (utils::Downcast<ConstraintQuery>(parsed_query.query)) {
@ -3077,8 +3221,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
PrepareReplicationQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->notifications,
interpreter_context_, &*execution_db_accessor_);
} else if (utils::Downcast<LockPathQuery>(parsed_query.query)) {
prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_,
&*execution_db_accessor_);
prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_);
} else if (utils::Downcast<FreeMemoryQuery>(parsed_query.query)) {
prepared_query = PrepareFreeMemoryQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_);
} else if (utils::Downcast<ShowConfigQuery>(parsed_query.query)) {
@ -3118,9 +3261,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
UpdateTypeCount(rw_type);
if (const auto query_type = query_execution->prepared_query->rw_type;
interpreter_context_->db->GetReplicationRole() == storage::replication::ReplicationRole::REPLICA &&
(query_type == RWType::W || query_type == RWType::RW)) {
if (IsWriteQueryOnMainMemoryReplica(interpreter_context_->db.get(), rw_type)) {
query_execution = nullptr;
throw QueryException("Write query forbidden on the replica!");
}
@ -3165,6 +3306,9 @@ void Interpreter::Abort() {
if (!db_accessor_) return;
db_accessor_->Abort();
for (auto &qe : query_executions_) {
if (qe) qe->CleanRuntimeData();
}
execution_db_accessor_.reset();
db_accessor_.reset();
trigger_context_collector_.reset();
@ -3180,7 +3324,7 @@ void RunTriggersIndividually(const utils::SkipList<Trigger> &triggers, Interpret
// create a new transaction for each trigger
auto storage_acc = interpreter_context->db->Access();
DbAccessor db_accessor{&storage_acc};
DbAccessor db_accessor{storage_acc.get()};
trigger_context.AdaptForAccessor(&db_accessor);
try {
@ -3223,6 +3367,8 @@ void RunTriggersIndividually(const utils::SkipList<Trigger> &triggers, Interpret
trigger.Name(), label_name, property_names_stream.str());
}
}
} else if constexpr (std::is_same_v<ErrorType, storage::SerializationError>) {
throw QueryException("Unable to commit due to serialization error.");
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -3260,6 +3406,14 @@ void Interpreter::Commit() {
utils::OnScopeExit clean_status(
[this]() { transaction_status_.store(TransactionStatus::IDLE, std::memory_order_release); });
auto current_storage_mode = interpreter_context_->db->GetStorageMode();
auto creation_mode = db_accessor_->GetCreationStorageMode();
if (creation_mode != storage::StorageMode::ON_DISK_TRANSACTIONAL &&
current_storage_mode == storage::StorageMode::ON_DISK_TRANSACTIONAL) {
throw QueryException(
"Cannot commit transaction because the storage mode has changed from in-memory storage to on-disk storage.");
}
utils::OnScopeExit update_metrics([]() {
memgraph::metrics::IncrementCounter(memgraph::metrics::CommitedTransactions);
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveTransactions);
@ -3293,6 +3447,9 @@ void Interpreter::Commit() {
}
const auto reset_necessary_members = [this]() {
for (auto &qe : query_executions_) {
if (qe) qe->CleanRuntimeData();
}
execution_db_accessor_.reset();
db_accessor_.reset();
trigger_context_collector_.reset();
@ -3331,6 +3488,8 @@ void Interpreter::Commit() {
property_names_stream.str());
}
}
} else if constexpr (std::is_same_v<ErrorType, storage::SerializationError>) {
throw QueryException("Unable to commit due to serialization error.");
} else {
static_assert(kAlwaysFalse<T>, "Missing type from variant visitor");
}
@ -3338,11 +3497,11 @@ void Interpreter::Commit() {
error);
}
// The ordered execution of after commit triggers is heavily depending on the exclusiveness of db_accessor_->Commit():
// only one of the transactions can be commiting at the same time, so when the commit is finished, that transaction
// probably will schedule its after commit triggers, because the other transactions that want to commit are still
// waiting for commiting or one of them just started commiting its changes.
// This means the ordered execution of after commit triggers are not guaranteed.
// The ordered execution of after commit triggers is heavily depending on the exclusiveness of
// db_accessor_->Commit(): only one of the transactions can be commiting at the same time, so when the commit is
// finished, that transaction probably will schedule its after commit triggers, because the other transactions that
// want to commit are still waiting for commiting or one of them just started commiting its changes. This means the
// ordered execution of after commit triggers are not guaranteed.
if (trigger_context && interpreter_context_->trigger_store.AfterCommitTriggers().size() > 0) {
interpreter_context_->after_commit_trigger_pool.AddTask(
[this, trigger_context = std::move(*trigger_context),

View File

@ -32,7 +32,10 @@
#include "query/stream/streams.hpp"
#include "query/trigger.hpp"
#include "query/typed_value.hpp"
#include "spdlog/spdlog.h"
#include "storage/v2/disk/storage.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/storage.hpp"
#include "utils/event_counter.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
@ -208,11 +211,15 @@ class Interpreter;
* running concurrently).
*
*/
/// TODO: andi decouple in a separate file why here?
struct InterpreterContext {
explicit InterpreterContext(storage::Storage *db, InterpreterConfig config,
explicit InterpreterContext(storage::Config storage_config, InterpreterConfig interpreter_config,
const std::filesystem::path &data_directory);
storage::Storage *db;
InterpreterContext(std::unique_ptr<storage::Storage> db, InterpreterConfig interpreter_config,
const std::filesystem::path &data_directory);
std::unique_ptr<storage::Storage> db;
// ANTLR has singleton instance that is shared between threads. It is
// protected by locks inside of ANTLR. Unfortunately, they are not protected
@ -319,9 +326,6 @@ class Interpreter final {
void BeginTransaction(const std::map<std::string, storage::PropertyValue> &metadata = {});
/*
Returns transaction id or empty if the db_accessor is not initialized.
*/
std::optional<uint64_t> GetTransactionId() const;
void CommitTransaction();
@ -378,6 +382,13 @@ class Interpreter final {
prepared_query.reset();
std::visit([](auto &memory_resource) { memory_resource.Release(); }, execution_memory);
}
void CleanRuntimeData() {
if (prepared_query.has_value()) {
prepared_query.reset();
}
notifications.clear();
}
};
// Interpreter supports multiple prepared queries at the same time.
@ -533,4 +544,5 @@ std::map<std::string, TypedValue> Interpreter::Pull(TStream *result_stream, std:
// don't return the execution summary as it's not finished
return {{"has_more", TypedValue(true)}};
}
} // namespace memgraph::query

View File

@ -54,6 +54,7 @@
#include "utils/likely.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
#include "utils/message.hpp"
#include "utils/pmr/deque.hpp"
#include "utils/pmr/list.hpp"
#include "utils/pmr/unordered_map.hpp"
@ -813,10 +814,12 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame, ExecutionContext &context) {
// old_node_value may be Null when using optional matching
if (!existing_node.IsNull()) {
ExpectType(self_.common_.node_symbol, existing_node, TypedValue::Type::Vertex);
context.db_accessor->PrefetchInEdges(vertex);
in_edges_.emplace(
UnwrapEdgesResult(vertex.InEdges(self_.view_, self_.common_.edge_types, existing_node.ValueVertex())));
}
} else {
context.db_accessor->PrefetchInEdges(vertex);
in_edges_.emplace(UnwrapEdgesResult(vertex.InEdges(self_.view_, self_.common_.edge_types)));
}
if (in_edges_) {
@ -830,10 +833,12 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame, ExecutionContext &context) {
// old_node_value may be Null when using optional matching
if (!existing_node.IsNull()) {
ExpectType(self_.common_.node_symbol, existing_node, TypedValue::Type::Vertex);
context.db_accessor->PrefetchOutEdges(vertex);
out_edges_.emplace(
UnwrapEdgesResult(vertex.OutEdges(self_.view_, self_.common_.edge_types, existing_node.ValueVertex())));
}
} else {
context.db_accessor->PrefetchOutEdges(vertex);
out_edges_.emplace(UnwrapEdgesResult(vertex.OutEdges(self_.view_, self_.common_.edge_types)));
}
if (out_edges_) {
@ -891,7 +896,8 @@ namespace {
* @return See above.
*/
auto ExpandFromVertex(const VertexAccessor &vertex, EdgeAtom::Direction direction,
const std::vector<storage::EdgeTypeId> &edge_types, utils::MemoryResource *memory) {
const std::vector<storage::EdgeTypeId> &edge_types, utils::MemoryResource *memory,
DbAccessor *db_accessor) {
// wraps an EdgeAccessor into a pair <accessor, direction>
auto wrapper = [](EdgeAtom::Direction direction, auto &&edges) {
return iter::imap([direction](const auto &edge) { return std::make_pair(edge, direction); },
@ -902,6 +908,7 @@ auto ExpandFromVertex(const VertexAccessor &vertex, EdgeAtom::Direction directio
utils::pmr::vector<decltype(wrapper(direction, *vertex.InEdges(view, edge_types)))> chain_elements(memory);
if (direction != EdgeAtom::Direction::OUT) {
db_accessor->PrefetchInEdges(vertex);
auto edges = UnwrapEdgesResult(vertex.InEdges(view, edge_types));
if (edges.begin() != edges.end()) {
chain_elements.emplace_back(wrapper(EdgeAtom::Direction::IN, std::move(edges)));
@ -909,6 +916,7 @@ auto ExpandFromVertex(const VertexAccessor &vertex, EdgeAtom::Direction directio
}
if (direction != EdgeAtom::Direction::IN) {
db_accessor->PrefetchOutEdges(vertex);
auto edges = UnwrapEdgesResult(vertex.OutEdges(view, edge_types));
if (edges.begin() != edges.end()) {
chain_elements.emplace_back(wrapper(EdgeAtom::Direction::OUT, std::move(edges)));
@ -974,8 +982,9 @@ class ExpandVariableCursor : public Cursor {
// a stack of edge iterables corresponding to the level/depth of
// the expansion currently being Pulled
using ExpandEdges = decltype(ExpandFromVertex(std::declval<VertexAccessor>(), EdgeAtom::Direction::IN,
self_.common_.edge_types, utils::NewDeleteResource()));
using ExpandEdges =
decltype(ExpandFromVertex(std::declval<VertexAccessor>(), EdgeAtom::Direction::IN, self_.common_.edge_types,
utils::NewDeleteResource(), std::declval<DbAccessor *>()));
utils::pmr::vector<ExpandEdges> edges_;
// an iterator indicating the position in the corresponding edges_ element
@ -1016,7 +1025,8 @@ class ExpandVariableCursor : public Cursor {
if (upper_bound_ > 0) {
auto *memory = edges_.get_allocator().GetMemoryResource();
edges_.emplace_back(ExpandFromVertex(vertex, self_.common_.direction, self_.common_.edge_types, memory));
edges_.emplace_back(
ExpandFromVertex(vertex, self_.common_.direction, self_.common_.edge_types, memory, context.db_accessor));
edges_it_.emplace_back(edges_.back().begin());
}
@ -1122,8 +1132,8 @@ class ExpandVariableCursor : public Cursor {
// edge's expansions onto the stack, if we should continue to expand
if (upper_bound_ > static_cast<int64_t>(edges_.size())) {
auto *memory = edges_.get_allocator().GetMemoryResource();
edges_.emplace_back(
ExpandFromVertex(current_vertex, self_.common_.direction, self_.common_.edge_types, memory));
edges_.emplace_back(ExpandFromVertex(current_vertex, self_.common_.direction, self_.common_.edge_types, memory,
context.db_accessor));
edges_it_.emplace_back(edges_.back().begin());
}
@ -1266,6 +1276,7 @@ class STShortestPathCursor : public query::plan::Cursor {
for (const auto &vertex : source_frontier) {
if (self_.common_.direction != EdgeAtom::Direction::IN) {
context.db_accessor->PrefetchOutEdges(vertex);
auto out_edges = UnwrapEdgesResult(vertex.OutEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : out_edges) {
#ifdef MG_ENTERPRISE
@ -1292,6 +1303,7 @@ class STShortestPathCursor : public query::plan::Cursor {
}
}
if (self_.common_.direction != EdgeAtom::Direction::OUT) {
dba.PrefetchInEdges(vertex);
auto in_edges = UnwrapEdgesResult(vertex.InEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : in_edges) {
#ifdef MG_ENTERPRISE
@ -1332,6 +1344,7 @@ class STShortestPathCursor : public query::plan::Cursor {
// reversed.
for (const auto &vertex : sink_frontier) {
if (self_.common_.direction != EdgeAtom::Direction::OUT) {
context.db_accessor->PrefetchOutEdges(vertex);
auto out_edges = UnwrapEdgesResult(vertex.OutEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : out_edges) {
#ifdef MG_ENTERPRISE
@ -1357,6 +1370,7 @@ class STShortestPathCursor : public query::plan::Cursor {
}
}
if (self_.common_.direction != EdgeAtom::Direction::IN) {
dba.PrefetchInEdges(vertex);
auto in_edges = UnwrapEdgesResult(vertex.InEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : in_edges) {
#ifdef MG_ENTERPRISE
@ -1446,12 +1460,14 @@ class SingleSourceShortestPathCursor : public query::plan::Cursor {
// populates the to_visit_next_ structure with expansions
// from the given vertex. skips expansions that don't satisfy
// the "where" condition.
auto expand_from_vertex = [this, &expand_pair](const auto &vertex) {
auto expand_from_vertex = [this, &expand_pair, &context](const auto &vertex) {
if (self_.common_.direction != EdgeAtom::Direction::IN) {
context.db_accessor->PrefetchOutEdges(vertex);
auto out_edges = UnwrapEdgesResult(vertex.OutEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : out_edges) expand_pair(edge, edge.To());
}
if (self_.common_.direction != EdgeAtom::Direction::OUT) {
context.db_accessor->PrefetchInEdges(vertex);
auto in_edges = UnwrapEdgesResult(vertex.InEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : in_edges) expand_pair(edge, edge.From());
}
@ -1646,15 +1662,17 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
// Populates the priority queue structure with expansions
// from the given vertex. skips expansions that don't satisfy
// the "where" condition.
auto expand_from_vertex = [this, &expand_pair](const VertexAccessor &vertex, const TypedValue &weight,
int64_t depth) {
auto expand_from_vertex = [this, &expand_pair, &context](const VertexAccessor &vertex, const TypedValue &weight,
int64_t depth) {
if (self_.common_.direction != EdgeAtom::Direction::IN) {
context.db_accessor->PrefetchOutEdges(vertex);
auto out_edges = UnwrapEdgesResult(vertex.OutEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : out_edges) {
expand_pair(edge, edge.To(), weight, depth);
}
}
if (self_.common_.direction != EdgeAtom::Direction::OUT) {
context.db_accessor->PrefetchInEdges(vertex);
auto in_edges = UnwrapEdgesResult(vertex.InEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : in_edges) {
expand_pair(edge, edge.From(), weight, depth);
@ -1913,6 +1931,7 @@ class ExpandAllShortestPathsCursor : public query::plan::Cursor {
auto expand_from_vertex = [this, &expand_vertex, &context](const VertexAccessor &vertex, const TypedValue &weight,
int64_t depth) {
if (self_.common_.direction != EdgeAtom::Direction::IN) {
context.db_accessor->PrefetchOutEdges(vertex);
auto out_edges = UnwrapEdgesResult(vertex.OutEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : out_edges) {
#ifdef MG_ENTERPRISE
@ -1927,6 +1946,7 @@ class ExpandAllShortestPathsCursor : public query::plan::Cursor {
}
}
if (self_.common_.direction != EdgeAtom::Direction::OUT) {
context.db_accessor->PrefetchInEdges(vertex);
auto in_edges = UnwrapEdgesResult(vertex.InEdges(storage::View::OLD, self_.common_.edge_types));
for (const auto &edge : in_edges) {
#ifdef MG_ENTERPRISE

View File

@ -28,6 +28,7 @@
#include "utils/fnv.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
#include "utils/synchronized.hpp"
#include "utils/visitor.hpp"
namespace memgraph {

View File

@ -27,7 +27,6 @@
#include "query/plan/operator.hpp"
#include "query/plan/preprocess.hpp"
#include "storage/v2/indices.hpp"
DECLARE_int64(query_vertex_count_to_expand_existing);

View File

@ -21,10 +21,12 @@
#include <stdexcept>
#include <type_traits>
#include <utility>
#include <variant>
#include "license/license.hpp"
#include "mg_procedure.h"
#include "module.hpp"
#include "query/db_accessor.hpp"
#include "query/frontend/ast/ast.hpp"
#include "query/procedure/cypher_types.hpp"
#include "query/procedure/mg_procedure_helpers.hpp"
@ -1950,9 +1952,9 @@ void NextPermittedEdge(mgp_edges_iterator &it, const bool for_in) {
const auto *auth_checker = it.source_vertex.graph->ctx->auth_checker.get();
const auto view = it.source_vertex.graph->view;
while (*impl_it != end) {
if (auth_checker->Has(**impl_it, memgraph::query::AuthQuery::FineGrainedPrivilege::READ)) {
const auto &check_vertex =
it.source_vertex.getImpl() == (*impl_it)->From() ? (*impl_it)->To() : (*impl_it)->From();
auto edgeAcc = **impl_it;
if (auth_checker->Has(edgeAcc, memgraph::query::AuthQuery::FineGrainedPrivilege::READ)) {
const auto &check_vertex = it.source_vertex.getImpl() == edgeAcc.From() ? edgeAcc.To() : edgeAcc.From();
if (auth_checker->Has(check_vertex, view, memgraph::query::AuthQuery::FineGrainedPrivilege::READ)) {
break;
}
@ -1968,6 +1970,14 @@ void NextPermittedEdge(mgp_edges_iterator &it, const bool for_in) {
mgp_error mgp_vertex_iter_in_edges(mgp_vertex *v, mgp_memory *memory, mgp_edges_iterator **result) {
return WrapExceptions(
[v, memory] {
auto dbAccessor = v->graph->impl;
if (std::holds_alternative<memgraph::query::DbAccessor *>(dbAccessor)) {
std::get<memgraph::query::DbAccessor *>(dbAccessor)
->PrefetchInEdges(std::get<memgraph::query::VertexAccessor>(v->impl));
} else {
std::get<memgraph::query::SubgraphDbAccessor *>(dbAccessor)
->PrefetchInEdges(std::get<memgraph::query::SubgraphVertexAccessor>(v->impl));
}
auto it = NewMgpObject<mgp_edges_iterator>(memory, *v);
MG_ASSERT(it != nullptr);
@ -1995,19 +2005,20 @@ mgp_error mgp_vertex_iter_in_edges(mgp_vertex *v, mgp_memory *memory, mgp_edges_
#endif
if (*it->in_it != it->in->end()) {
std::visit(memgraph::utils::Overloaded{
[&](memgraph::query::DbAccessor *) {
it->current_e.emplace(**it->in_it, (**it->in_it).From(), (**it->in_it).To(), v->graph,
it->GetMemoryResource());
},
[&](memgraph::query::SubgraphDbAccessor *impl) {
it->current_e.emplace(
**it->in_it,
memgraph::query::SubgraphVertexAccessor((**it->in_it).From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor((**it->in_it).To(), impl->getGraph()), v->graph,
it->GetMemoryResource());
}},
v->graph->impl);
std::visit(
memgraph::utils::Overloaded{
[&](memgraph::query::DbAccessor *) {
auto edgeAcc = **it->in_it;
it->current_e.emplace(edgeAcc, edgeAcc.From(), edgeAcc.To(), v->graph, it->GetMemoryResource());
},
[&](memgraph::query::SubgraphDbAccessor *impl) {
auto edgeAcc = **it->in_it;
it->current_e.emplace(edgeAcc,
memgraph::query::SubgraphVertexAccessor(edgeAcc.From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor(edgeAcc.To(), impl->getGraph()),
v->graph, it->GetMemoryResource());
}},
v->graph->impl);
}
return it.release();
@ -2018,6 +2029,14 @@ mgp_error mgp_vertex_iter_in_edges(mgp_vertex *v, mgp_memory *memory, mgp_edges_
mgp_error mgp_vertex_iter_out_edges(mgp_vertex *v, mgp_memory *memory, mgp_edges_iterator **result) {
return WrapExceptions(
[v, memory] {
auto dbAccessor = v->graph->impl;
if (std::holds_alternative<memgraph::query::DbAccessor *>(dbAccessor)) {
std::get<memgraph::query::DbAccessor *>(dbAccessor)
->PrefetchOutEdges(std::get<memgraph::query::VertexAccessor>(v->impl));
} else {
std::get<memgraph::query::SubgraphDbAccessor *>(dbAccessor)
->PrefetchOutEdges(std::get<memgraph::query::SubgraphVertexAccessor>(v->impl));
}
auto it = NewMgpObject<mgp_edges_iterator>(memory, *v);
MG_ASSERT(it != nullptr);
auto maybe_edges = std::visit([v](auto &impl) { return impl.OutEdges(v->graph->view); }, v->impl);
@ -2047,19 +2066,20 @@ mgp_error mgp_vertex_iter_out_edges(mgp_vertex *v, mgp_memory *memory, mgp_edges
#endif
if (*it->out_it != it->out->end()) {
std::visit(memgraph::utils::Overloaded{
[&](memgraph::query::DbAccessor *) {
it->current_e.emplace(**it->out_it, (**it->out_it).From(), (**it->out_it).To(), v->graph,
it->GetMemoryResource());
},
[&](memgraph::query::SubgraphDbAccessor *impl) {
it->current_e.emplace(
**it->out_it,
memgraph::query::SubgraphVertexAccessor((**it->out_it).From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor((**it->out_it).To(), impl->getGraph()), v->graph,
it->GetMemoryResource());
}},
v->graph->impl);
std::visit(
memgraph::utils::Overloaded{
[&](memgraph::query::DbAccessor *) {
memgraph::query::EdgeAccessor edgeAcc = **it->out_it;
it->current_e.emplace(edgeAcc, edgeAcc.From(), edgeAcc.To(), v->graph, it->GetMemoryResource());
},
[&](memgraph::query::SubgraphDbAccessor *impl) {
auto edgeAcc = **it->out_it;
it->current_e.emplace(edgeAcc,
memgraph::query::SubgraphVertexAccessor(edgeAcc.From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor(edgeAcc.To(), impl->getGraph()),
v->graph, it->GetMemoryResource());
}},
v->graph->impl);
}
return it.release();
@ -2110,13 +2130,15 @@ mgp_error mgp_edges_iterator_next(mgp_edges_iterator *it, mgp_edge **result) {
}
std::visit(memgraph::utils::Overloaded{
[&](memgraph::query::DbAccessor *) {
it->current_e.emplace(**impl_it, (**impl_it).From(), (**impl_it).To(),
it->source_vertex.graph, it->GetMemoryResource());
auto edgeAcc = **impl_it;
it->current_e.emplace(edgeAcc, edgeAcc.From(), edgeAcc.To(), it->source_vertex.graph,
it->GetMemoryResource());
},
[&](memgraph::query::SubgraphDbAccessor *impl) {
auto edgeAcc = **impl_it;
it->current_e.emplace(
**impl_it, memgraph::query::SubgraphVertexAccessor((**impl_it).From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor((**impl_it).To(), impl->getGraph()),
edgeAcc, memgraph::query::SubgraphVertexAccessor(edgeAcc.From(), impl->getGraph()),
memgraph::query::SubgraphVertexAccessor(edgeAcc.To(), impl->getGraph()),
it->source_vertex.graph, it->GetMemoryResource());
}},
it->source_vertex.graph->impl);

View File

@ -435,28 +435,24 @@ struct mgp_vertex {
/// the allocator which was used to allocate `this`.
using allocator_type = memgraph::utils::Allocator<mgp_vertex>;
// Hopefully VertexAccessor copy constructor remains noexcept, so that we can
// have everything noexcept here.
static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::VertexAccessor>);
mgp_vertex(memgraph::query::VertexAccessor v, mgp_graph *graph, memgraph::utils::MemoryResource *memory) noexcept
mgp_vertex(memgraph::query::VertexAccessor v, mgp_graph *graph, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(v), graph(graph) {}
mgp_vertex(memgraph::query::SubgraphVertexAccessor v, mgp_graph *graph,
memgraph::utils::MemoryResource *memory) noexcept
mgp_vertex(memgraph::query::SubgraphVertexAccessor v, mgp_graph *graph, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(v), graph(graph) {}
mgp_vertex(const mgp_vertex &other, memgraph::utils::MemoryResource *memory) noexcept
mgp_vertex(const mgp_vertex &other, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(other.impl), graph(other.graph) {}
mgp_vertex(mgp_vertex &&other, memgraph::utils::MemoryResource *memory) noexcept
mgp_vertex(mgp_vertex &&other, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(other.impl), graph(other.graph) {}
mgp_vertex(mgp_vertex &&other) noexcept : memory(other.memory), impl(other.impl), graph(other.graph) {}
// NOLINTNEXTLINE(hicpp-noexcept-move, performance-noexcept-move-constructor)
mgp_vertex(mgp_vertex &&other) : memory(other.memory), impl(other.impl), graph(other.graph) {}
memgraph::query::VertexAccessor getImpl() const {
return std::visit(
memgraph::utils::Overloaded{[](memgraph::query::VertexAccessor impl) { return impl; },
memgraph::utils::Overloaded{[](const memgraph::query::VertexAccessor &impl) { return impl; },
[](memgraph::query::SubgraphVertexAccessor impl) { return impl.impl_; }},
this->impl);
}
@ -486,33 +482,28 @@ struct mgp_edge {
/// the allocator which was used to allocate `this`.
using allocator_type = memgraph::utils::Allocator<mgp_edge>;
// Hopefully EdgeAccessor copy constructor remains noexcept, so that we can
// have everything noexcept here.
static_assert(std::is_nothrow_copy_constructible_v<memgraph::query::EdgeAccessor>);
static mgp_edge *Copy(const mgp_edge &edge, mgp_memory &memory);
mgp_edge(const memgraph::query::EdgeAccessor &impl, mgp_graph *graph,
memgraph::utils::MemoryResource *memory) noexcept
mgp_edge(const memgraph::query::EdgeAccessor &impl, mgp_graph *graph, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(impl), from(impl.From(), graph, memory), to(impl.To(), graph, memory) {}
mgp_edge(const memgraph::query::EdgeAccessor &impl, const memgraph::query::VertexAccessor &from_v,
const memgraph::query::VertexAccessor &to_v, mgp_graph *graph,
memgraph::utils::MemoryResource *memory) noexcept
const memgraph::query::VertexAccessor &to_v, mgp_graph *graph, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(impl), from(from_v, graph, memory), to(to_v, graph, memory) {}
mgp_edge(const memgraph::query::EdgeAccessor &impl, const memgraph::query::SubgraphVertexAccessor &from_v,
const memgraph::query::SubgraphVertexAccessor &to_v, mgp_graph *graph,
memgraph::utils::MemoryResource *memory) noexcept
memgraph::utils::MemoryResource *memory)
: memory(memory), impl(impl), from(from_v, graph, memory), to(to_v, graph, memory) {}
mgp_edge(const mgp_edge &other, memgraph::utils::MemoryResource *memory) noexcept
mgp_edge(const mgp_edge &other, memgraph::utils::MemoryResource *memory)
: memory(memory), impl(other.impl), from(other.from, memory), to(other.to, memory) {}
mgp_edge(mgp_edge &&other, memgraph::utils::MemoryResource *memory) noexcept
mgp_edge(mgp_edge &&other, memgraph::utils::MemoryResource *memory)
: memory(other.memory), impl(other.impl), from(std::move(other.from), memory), to(std::move(other.to), memory) {}
mgp_edge(mgp_edge &&other) noexcept
// NOLINTNEXTLINE(hicpp-noexcept-move, performance-noexcept-move-constructor)
mgp_edge(mgp_edge &&other)
: memory(other.memory), impl(other.impl), from(std::move(other.from)), to(std::move(other.to)) {}
/// Copy construction without memgraph::utils::MemoryResource is not allowed.
@ -671,14 +662,12 @@ struct mgp_properties_iterator {
struct mgp_edges_iterator {
using allocator_type = memgraph::utils::Allocator<mgp_edges_iterator>;
// Hopefully mgp_vertex copy constructor remains noexcept, so that we can
// have everything noexcept here.
static_assert(std::is_nothrow_constructible_v<mgp_vertex, const mgp_vertex &, memgraph::utils::MemoryResource *>);
mgp_edges_iterator(const mgp_vertex &v, memgraph::utils::MemoryResource *memory) noexcept
mgp_edges_iterator(const mgp_vertex &v, memgraph::utils::MemoryResource *memory)
: memory(memory), source_vertex(v, memory) {}
mgp_edges_iterator(mgp_edges_iterator &&other) noexcept
// NOLINTNEXTLINE(hicpp-noexcept-move, performance-noexcept-move-constructor)
mgp_edges_iterator(mgp_edges_iterator &&other)
: memory(other.memory),
source_vertex(std::move(other.source_vertex)),
in(std::move(other.in)),

View File

@ -496,7 +496,7 @@ Streams::StreamsMap::iterator Streams::CreateConsumer(StreamsMap &map, const std
[interpreter_context, interpreter]() { interpreter_context->interpreters->erase(interpreter.get()); }};
memgraph::metrics::IncrementCounter(memgraph::metrics::MessagesConsumed, messages.size());
CallCustomTransformation(transformation_name, messages, result, accessor, *memory_resource, stream_name);
CallCustomTransformation(transformation_name, messages, result, *accessor, *memory_resource, stream_name);
DiscardValueResultStream stream;
@ -743,7 +743,7 @@ TransformationResult Streams::Check(const std::string &stream_name, std::optiona
&transformation_name = transformation_name, &result,
&test_result]<typename T>(const std::vector<T> &messages) mutable {
auto accessor = interpreter_context->db->Access();
CallCustomTransformation(transformation_name, messages, result, accessor, *memory_resource, stream_name);
CallCustomTransformation(transformation_name, messages, result, *accessor, *memory_resource, stream_name);
auto result_row = std::vector<TypedValue>();
result_row.reserve(kCheckStreamResultSize);

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
@ -302,6 +302,7 @@ void TriggerContext::AdaptForAccessor(DbAccessor *accessor) {
if (!maybe_from_vertex) {
continue;
}
accessor->PrefetchOutEdges(*maybe_from_vertex);
auto maybe_out_edges = maybe_from_vertex->OutEdges(storage::View::OLD);
MG_ASSERT(maybe_out_edges.HasValue());
const auto edge_gid = created_edge.object.Gid();
@ -323,6 +324,7 @@ void TriggerContext::AdaptForAccessor(DbAccessor *accessor) {
auto it = values->begin();
for (const auto &value : *values) {
if (auto maybe_vertex = accessor->FindVertex(value.object.From().Gid(), storage::View::OLD); maybe_vertex) {
accessor->PrefetchOutEdges(*maybe_vertex);
auto maybe_out_edges = maybe_vertex->OutEdges(storage::View::OLD);
MG_ASSERT(maybe_out_edges.HasValue());
for (const auto &edge : *maybe_out_edges) {

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
@ -160,11 +160,6 @@ enum class TriggerEventType : uint8_t {
const char *TriggerEventTypeToString(TriggerEventType event_type);
static_assert(std::is_trivially_copy_constructible_v<VertexAccessor>,
"VertexAccessor is not trivially copy constructible, move it where possible and remove this assert");
static_assert(std::is_trivially_copy_constructible_v<EdgeAccessor>,
"EdgeAccessor is not trivially copy constructible, move it where possible and remove this asssert");
// Holds the information necessary for triggers
class TriggerContext {
public:

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

3
src/storage/README.md Normal file
View File

@ -0,0 +1,3 @@
# Storage Modes
* `ON_DISK_TRANSACTIONAL`

View File

@ -1,16 +1,27 @@
set(storage_v2_src_files
commit_log.cpp
constraints.cpp
constraints/existence_constraints.cpp
temporal.cpp
durability/durability.cpp
durability/serialization.cpp
durability/snapshot.cpp
durability/wal.cpp
edge_accessor.cpp
indices.cpp
property_store.cpp
vertex_accessor.cpp
storage.cpp
indices/indices.cpp
all_vertices_iterable.cpp
vertices_iterable.cpp
inmemory/storage.cpp
inmemory/label_index.cpp
inmemory/label_property_index.cpp
inmemory/unique_constraints.cpp
disk/storage.cpp
disk/rocksdb_storage.cpp
disk/label_index.cpp
disk/label_property_index.cpp
disk/unique_constraints.cpp
storage_mode.cpp
isolation_level.cpp)

View File

@ -0,0 +1,44 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/all_vertices_iterable.hpp"
namespace memgraph::storage {
auto AdvanceToVisibleVertex(utils::SkipList<Vertex>::Iterator it, utils::SkipList<Vertex>::Iterator end,
std::optional<VertexAccessor> *vertex, Transaction *tx, View view, Indices *indices,
Constraints *constraints, Config::Items config) {
while (it != end) {
*vertex = VertexAccessor::Create(&*it, tx, indices, constraints, config, view);
if (!*vertex) {
++it;
continue;
}
break;
}
return it;
}
AllVerticesIterable::Iterator::Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it)
: self_(self),
it_(AdvanceToVisibleVertex(it, self->vertices_accessor_.end(), &self->vertex_, self->transaction_, self->view_,
self->indices_, self_->constraints_, self->config_)) {}
VertexAccessor AllVerticesIterable::Iterator::operator*() const { return *self_->vertex_; }
AllVerticesIterable::Iterator &AllVerticesIterable::Iterator::operator++() {
++it_;
it_ = AdvanceToVisibleVertex(it_, self_->vertices_accessor_.end(), &self_->vertex_, self_->transaction_, self_->view_,
self_->indices_, self_->constraints_, self_->config_);
return *this;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,58 @@
// 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 "storage/v2/vertex_accessor.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage {
class AllVerticesIterable final {
utils::SkipList<Vertex>::Accessor vertices_accessor_;
Transaction *transaction_;
View view_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
std::optional<VertexAccessor> vertex_;
public:
class Iterator final {
AllVerticesIterable *self_;
utils::SkipList<Vertex>::Iterator it_;
public:
Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it);
VertexAccessor operator*() const;
Iterator &operator++();
bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; }
bool operator!=(const Iterator &other) const { return !(*this == other); }
};
AllVerticesIterable(utils::SkipList<Vertex>::Accessor vertices_accessor, Transaction *transaction, View view,
Indices *indices, Constraints *constraints, Config::Items config)
: vertices_accessor_(std::move(vertices_accessor)),
transaction_(transaction),
view_(view),
indices_(indices),
constraints_(constraints),
config_(config) {}
Iterator begin() { return {this, vertices_accessor_.begin()}; }
Iterator end() { return {this, vertices_accessor_.end()}; }
};
} // namespace memgraph::storage

View File

@ -60,6 +60,17 @@ struct Config {
struct Transaction {
IsolationLevel isolation_level{IsolationLevel::SNAPSHOT_ISOLATION};
} transaction;
struct DiskConfig {
std::filesystem::path main_storage_directory{"rocksdb_main_storage"};
std::filesystem::path label_index_directory{"rocksdb_label_index"};
std::filesystem::path label_property_index_directory{"rocksdb_label_property_index"};
std::filesystem::path unique_constraints_directory{"rocksdb_unique_constraints"};
std::filesystem::path name_id_mapper_directory{"rocksdb_name_id_mapper"};
std::filesystem::path id_name_mapper_directory{"rocksdb_id_name_mapper"};
std::filesystem::path durability_directory{"rocksdb_durability"};
std::filesystem::path wal_directory{"rocksdb_wal"};
} disk;
};
} // namespace memgraph::storage

View File

@ -1,199 +0,0 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <optional>
#include <set>
#include <vector>
#include "storage/v2/id_types.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/logging.hpp"
#include "utils/result.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage {
// NOLINTNEXTLINE(misc-definitions-in-headers)
const size_t kUniqueConstraintsMaxProperties = 32;
/// Utility class to store data in a fixed size array. The array is used
/// instead of `std::vector` to avoid `std::bad_alloc` exception where not
/// necessary.
template <class T>
struct FixedCapacityArray {
size_t size;
T values[kUniqueConstraintsMaxProperties];
explicit FixedCapacityArray(size_t array_size) : size(array_size) {
MG_ASSERT(size <= kUniqueConstraintsMaxProperties, "Invalid array size!");
}
};
using PropertyIdArray = FixedCapacityArray<PropertyId>;
struct ConstraintViolation {
enum class Type {
EXISTENCE,
UNIQUE,
};
Type type;
LabelId label;
// While multiple properties are supported by unique constraints, the
// `properties` set will always have exactly one element in the case of
// existence constraint violation.
std::set<PropertyId> properties;
};
bool operator==(const ConstraintViolation &lhs, const ConstraintViolation &rhs);
class UniqueConstraints {
private:
struct Entry {
std::vector<PropertyValue> values;
const Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs);
bool operator==(const Entry &rhs);
bool operator<(const std::vector<PropertyValue> &rhs);
bool operator==(const std::vector<PropertyValue> &rhs);
};
public:
/// Status for creation of unique constraints.
/// Note that this does not cover the case when the constraint is violated.
enum class CreationStatus {
SUCCESS,
ALREADY_EXISTS,
EMPTY_PROPERTIES,
PROPERTIES_SIZE_LIMIT_EXCEEDED,
};
/// Status for deletion of unique constraints.
enum class DeletionStatus {
SUCCESS,
NOT_FOUND,
EMPTY_PROPERTIES,
PROPERTIES_SIZE_LIMIT_EXCEEDED,
};
/// Indexes the given vertex for relevant labels and properties.
/// This method should be called before committing and validating vertices
/// against unique constraints.
/// @throw std::bad_alloc
void UpdateBeforeCommit(const Vertex *vertex, const Transaction &tx);
/// Creates unique constraint on the given `label` and a list of `properties`.
/// Returns constraint violation if there are multiple vertices with the same
/// label and property values. Returns `CreationStatus::ALREADY_EXISTS` if
/// constraint already existed, `CreationStatus::EMPTY_PROPERTIES` if the
/// given list of properties is empty,
/// `CreationStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED` if the list of properties
/// exceeds the maximum allowed number of properties, and
/// `CreationStatus::SUCCESS` on success.
/// @throw std::bad_alloc
utils::BasicResult<ConstraintViolation, CreationStatus> CreateConstraint(LabelId label,
const std::set<PropertyId> &properties,
utils::SkipList<Vertex>::Accessor vertices);
/// Deletes the specified constraint. Returns `DeletionStatus::NOT_FOUND` if
/// there is not such constraint in the storage,
/// `DeletionStatus::EMPTY_PROPERTIES` if the given set of `properties` is
/// empty, `DeletionStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED` if the given set
/// of `properties` exceeds the maximum allowed number of properties, and
/// `DeletionStatus::SUCCESS` on success.
DeletionStatus DropConstraint(LabelId label, const std::set<PropertyId> &properties);
bool ConstraintExists(LabelId label, const std::set<PropertyId> &properties) {
return constraints_.find({label, properties}) != constraints_.end();
}
/// Validates the given vertex against unique constraints before committing.
/// This method should be called while commit lock is active with
/// `commit_timestamp` being a potential commit timestamp of the transaction.
/// @throw std::bad_alloc
std::optional<ConstraintViolation> Validate(const Vertex &vertex, const Transaction &tx,
uint64_t commit_timestamp) const;
std::vector<std::pair<LabelId, std::set<PropertyId>>> ListConstraints() const;
/// GC method that removes outdated entries from constraints' storages.
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
void Clear() { constraints_.clear(); }
private:
std::map<std::pair<LabelId, std::set<PropertyId>>, utils::SkipList<Entry>> constraints_;
};
struct Constraints {
std::vector<std::pair<LabelId, PropertyId>> existence_constraints;
UniqueConstraints unique_constraints;
};
/// Adds a unique constraint to `constraints`. Returns true if the constraint
/// was successfully added, false if it already exists and a
/// `ConstraintViolation` if there is an existing vertex violating the
/// constraint.
///
/// @throw std::bad_alloc
/// @throw std::length_error
inline utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint(
Constraints *constraints, LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor vertices) {
if (utils::Contains(constraints->existence_constraints, std::make_pair(label, property))) {
return false;
}
for (const auto &vertex : vertices) {
if (!vertex.deleted && utils::Contains(vertex.labels, label) && !vertex.properties.HasProperty(property)) {
return ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label, std::set<PropertyId>{property}};
}
}
constraints->existence_constraints.emplace_back(label, property);
return true;
}
/// Removes a unique constraint from `constraints`. Returns true if the
/// constraint was removed, and false if it doesn't exist.
inline bool DropExistenceConstraint(Constraints *constraints, LabelId label, PropertyId property) {
auto it = std::find(constraints->existence_constraints.begin(), constraints->existence_constraints.end(),
std::make_pair(label, property));
if (it == constraints->existence_constraints.end()) {
return false;
}
constraints->existence_constraints.erase(it);
return true;
}
/// Verifies that the given vertex satisfies all existence constraints. Returns
/// `std::nullopt` if all checks pass, and `ConstraintViolation` describing the
/// violated constraint otherwise.
[[nodiscard]] inline std::optional<ConstraintViolation> ValidateExistenceConstraints(const Vertex &vertex,
const Constraints &constraints) {
for (const auto &[label, property] : constraints.existence_constraints) {
if (!vertex.deleted && utils::Contains(vertex.labels, label) && !vertex.properties.HasProperty(property)) {
return ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label, std::set<PropertyId>{property}};
}
}
return std::nullopt;
}
/// Returns a list of all created existence constraints.
inline std::vector<std::pair<LabelId, PropertyId>> ListExistenceConstraints(const Constraints &constraints) {
return constraints.existence_constraints;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,39 @@
// 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 <set>
#include "storage/v2/id_types.hpp"
namespace memgraph::storage {
struct ConstraintViolation {
enum class Type {
EXISTENCE,
UNIQUE,
};
Type type;
LabelId label;
// While multiple properties are supported by unique constraints, the
// `properties` set will always have exactly one element in the case of
// existence constraint violation.
std::set<PropertyId> properties;
};
inline bool operator==(const ConstraintViolation &lhs, const ConstraintViolation &rhs) {
return lhs.type == rhs.type && lhs.label == rhs.label && lhs.properties == rhs.properties;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,48 @@
// 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 "storage/v2/config.hpp"
#include "storage/v2/constraints/existence_constraints.hpp"
#include "storage/v2/disk/unique_constraints.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
#include "storage/v2/storage_mode.hpp"
namespace memgraph::storage {
struct Constraints {
Constraints(const Config &config, StorageMode storage_mode) {
std::invoke([this, config, storage_mode]() {
existence_constraints_ = std::make_unique<ExistenceConstraints>();
switch (storage_mode) {
case StorageMode::IN_MEMORY_TRANSACTIONAL:
case StorageMode::IN_MEMORY_ANALYTICAL:
unique_constraints_ = std::make_unique<InMemoryUniqueConstraints>();
break;
case StorageMode::ON_DISK_TRANSACTIONAL:
unique_constraints_ = std::make_unique<DiskUniqueConstraints>(config);
break;
};
});
}
Constraints(const Constraints &) = delete;
Constraints(Constraints &&) = delete;
Constraints &operator=(const Constraints &) = delete;
Constraints &operator=(Constraints &&) = delete;
~Constraints() = default;
std::unique_ptr<ExistenceConstraints> existence_constraints_;
std::unique_ptr<UniqueConstraints> unique_constraints_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,58 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/constraints/existence_constraints.hpp"
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/mvcc.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage {
bool ExistenceConstraints::ConstraintExists(LabelId label, PropertyId property) const {
return utils::Contains(constraints_, std::make_pair(label, property));
}
void ExistenceConstraints::InsertConstraint(LabelId label, PropertyId property) {
if (ConstraintExists(label, property)) {
return;
}
constraints_.emplace_back(label, property);
}
bool ExistenceConstraints::DropConstraint(LabelId label, PropertyId property) {
auto it = std::find(constraints_.begin(), constraints_.end(), std::make_pair(label, property));
if (it == constraints_.end()) {
return false;
}
constraints_.erase(it);
return true;
}
std::vector<std::pair<LabelId, PropertyId>> ExistenceConstraints::ListConstraints() const { return constraints_; }
[[nodiscard]] std::optional<ConstraintViolation> ExistenceConstraints::Validate(const Vertex &vertex) {
for (const auto &[label, property] : constraints_) {
if (auto violation = ValidateVertexOnConstraint(vertex, label, property); violation.has_value()) {
return violation;
}
}
return std::nullopt;
}
void ExistenceConstraints::LoadExistenceConstraints(const std::vector<std::string> &keys) {
for (const auto &key : keys) {
const std::vector<std::string> parts = utils::Split(key, ",");
constraints_.emplace_back(LabelId::FromUint(std::stoull(parts[0])), PropertyId::FromUint(std::stoull(parts[1])));
}
}
} // namespace memgraph::storage

View File

@ -0,0 +1,62 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <optional>
#include "storage/v2/constraints/constraint_violation.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage {
class ExistenceConstraints {
public:
[[nodiscard]] static std::optional<ConstraintViolation> ValidateVertexOnConstraint(const Vertex &vertex,
LabelId label,
PropertyId property) {
if (!vertex.deleted && utils::Contains(vertex.labels, label) && !vertex.properties.HasProperty(property)) {
return ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label, std::set<PropertyId>{property}};
}
return std::nullopt;
}
[[nodiscard]] static std::optional<ConstraintViolation> ValidateVerticesOnConstraint(
utils::SkipList<Vertex>::Accessor vertices, LabelId label, PropertyId property) {
for (const auto &vertex : vertices) {
if (auto violation = ValidateVertexOnConstraint(vertex, label, property); violation.has_value()) {
return violation;
}
}
return std::nullopt;
}
bool ConstraintExists(LabelId label, PropertyId property) const;
void InsertConstraint(LabelId label, PropertyId property);
/// Returns true if the constraint was removed, and false if it doesn't exist.
bool DropConstraint(LabelId label, PropertyId property);
/// Returns `std::nullopt` if all checks pass, and `ConstraintViolation` describing the violated constraint
/// otherwise.
[[nodiscard]] std::optional<ConstraintViolation> Validate(const Vertex &vertex);
std::vector<std::pair<LabelId, PropertyId>> ListConstraints() const;
void LoadExistenceConstraints(const std::vector<std::string> &keys);
private:
std::vector<std::pair<LabelId, PropertyId>> constraints_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,75 @@
// 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 <set>
#include "storage/v2/constraints/constraint_violation.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/result.hpp"
namespace memgraph::storage {
// NOLINTNEXTLINE(misc-definitions-in-headers)
const size_t kUniqueConstraintsMaxProperties = 32;
class UniqueConstraints {
public:
UniqueConstraints() = default;
UniqueConstraints(const UniqueConstraints &) = delete;
UniqueConstraints(UniqueConstraints &&) = delete;
UniqueConstraints &operator=(const UniqueConstraints &) = delete;
UniqueConstraints &operator=(UniqueConstraints &&) = delete;
virtual ~UniqueConstraints() = default;
enum class CreationStatus {
SUCCESS,
ALREADY_EXISTS,
EMPTY_PROPERTIES,
PROPERTIES_SIZE_LIMIT_EXCEEDED,
};
enum class DeletionStatus {
SUCCESS,
NOT_FOUND,
EMPTY_PROPERTIES,
PROPERTIES_SIZE_LIMIT_EXCEEDED,
};
virtual DeletionStatus DropConstraint(LabelId label, const std::set<PropertyId> &properties) = 0;
virtual bool ConstraintExists(LabelId label, const std::set<PropertyId> &properties) const = 0;
virtual void UpdateOnRemoveLabel(LabelId removed_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) = 0;
virtual void UpdateOnAddLabel(LabelId added_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) = 0;
virtual std::vector<std::pair<LabelId, std::set<PropertyId>>> ListConstraints() const = 0;
virtual void Clear() = 0;
protected:
static DeletionStatus CheckPropertiesBeforeDeletion(const std::set<PropertyId> &properties) {
if (properties.empty()) {
return UniqueConstraints::DeletionStatus::EMPTY_PROPERTIES;
}
if (properties.size() > kUniqueConstraintsMaxProperties) {
return UniqueConstraints::DeletionStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED;
}
return UniqueConstraints::DeletionStatus::SUCCESS;
}
};
} // namespace memgraph::storage

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
@ -12,6 +12,7 @@
#pragma once
#include <atomic>
#include <cstdint>
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/id_types.hpp"
@ -122,7 +123,10 @@ inline bool operator!=(const PreviousPtr::Pointer &a, const PreviousPtr::Pointer
struct Delta {
enum class Action {
// Used for both Vertex and Edge
/// Use for Vertex and Edge
/// Used for disk storage for modifying MVCC logic and storing old key. Storing old key is necessary for
/// deleting old-data (compaction).
DELETE_DESERIALIZED_OBJECT,
DELETE_OBJECT,
RECREATE_OBJECT,
SET_PROPERTY,
@ -137,6 +141,7 @@ struct Delta {
};
// Used for both Vertex and Edge
struct DeleteDeserializedObjectTag {};
struct DeleteObjectTag {};
struct RecreateObjectTag {};
struct SetPropertyTag {};
@ -149,44 +154,48 @@ struct Delta {
struct RemoveInEdgeTag {};
struct RemoveOutEdgeTag {};
Delta(DeleteObjectTag, std::atomic<uint64_t> *timestamp, uint64_t command_id)
Delta(DeleteDeserializedObjectTag /*tag*/, std::atomic<uint64_t> *timestamp,
const std::optional<std::string> &old_disk_key)
: action(Action::DELETE_DESERIALIZED_OBJECT), timestamp(timestamp), command_id(0), old_disk_key(old_disk_key) {}
Delta(DeleteObjectTag /*tag*/, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::DELETE_OBJECT), timestamp(timestamp), command_id(command_id) {}
Delta(RecreateObjectTag, std::atomic<uint64_t> *timestamp, uint64_t command_id)
Delta(RecreateObjectTag /*tag*/, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::RECREATE_OBJECT), timestamp(timestamp), command_id(command_id) {}
Delta(AddLabelTag, LabelId label, std::atomic<uint64_t> *timestamp, uint64_t command_id)
Delta(AddLabelTag /*tag*/, LabelId label, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::ADD_LABEL), timestamp(timestamp), command_id(command_id), label(label) {}
Delta(RemoveLabelTag, LabelId label, std::atomic<uint64_t> *timestamp, uint64_t command_id)
Delta(RemoveLabelTag /*tag*/, LabelId label, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::REMOVE_LABEL), timestamp(timestamp), command_id(command_id), label(label) {}
Delta(SetPropertyTag, PropertyId key, const PropertyValue &value, std::atomic<uint64_t> *timestamp,
Delta(SetPropertyTag /*tag*/, PropertyId key, const PropertyValue &value, std::atomic<uint64_t> *timestamp,
uint64_t command_id)
: action(Action::SET_PROPERTY), timestamp(timestamp), command_id(command_id), property({key, value}) {}
Delta(AddInEdgeTag, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
Delta(AddInEdgeTag /*tag*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
uint64_t command_id)
: action(Action::ADD_IN_EDGE),
timestamp(timestamp),
command_id(command_id),
vertex_edge({edge_type, vertex, edge}) {}
Delta(AddOutEdgeTag, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
Delta(AddOutEdgeTag /*tag*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
uint64_t command_id)
: action(Action::ADD_OUT_EDGE),
timestamp(timestamp),
command_id(command_id),
vertex_edge({edge_type, vertex, edge}) {}
Delta(RemoveInEdgeTag, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
Delta(RemoveInEdgeTag /*tag*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
uint64_t command_id)
: action(Action::REMOVE_IN_EDGE),
timestamp(timestamp),
command_id(command_id),
vertex_edge({edge_type, vertex, edge}) {}
Delta(RemoveOutEdgeTag, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
Delta(RemoveOutEdgeTag /*tag*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge, std::atomic<uint64_t> *timestamp,
uint64_t command_id)
: action(Action::REMOVE_OUT_EDGE),
timestamp(timestamp),
@ -209,6 +218,9 @@ struct Delta {
case Action::REMOVE_IN_EDGE:
case Action::REMOVE_OUT_EDGE:
break;
case Action::DELETE_DESERIALIZED_OBJECT:
old_disk_key.reset();
break;
case Action::SET_PROPERTY:
property.value.~PropertyValue();
break;
@ -224,6 +236,7 @@ struct Delta {
std::atomic<Delta *> next{nullptr};
union {
std::optional<std::string> old_disk_key;
LabelId label;
struct {
PropertyId key;

View File

@ -0,0 +1,215 @@
// 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 <rocksdb/options.h>
#include <rocksdb/utilities/transaction.h>
#include "storage/v2/disk/label_index.hpp"
#include "utils/disk_utils.hpp"
#include "utils/rocksdb_serialization.hpp"
namespace memgraph::storage {
namespace {
[[nodiscard]] bool ClearTransactionEntriesWithRemovedIndexingLabel(
rocksdb::Transaction &disk_transaction, const std::map<Gid, std::vector<LabelId>> &transaction_entries) {
for (const auto &[vertex_gid, labels] : transaction_entries) {
for (const auto &indexing_label : labels) {
if (auto status = disk_transaction.Delete(utils::SerializeVertexAsKeyForLabelIndex(indexing_label, vertex_gid));
!status.ok()) {
return false;
}
}
}
return true;
}
/// TODO: duplication with label_property_index.cpp
bool CommitWithTimestamp(rocksdb::Transaction *disk_transaction, uint64_t commit_ts) {
disk_transaction->SetCommitTimestamp(commit_ts);
const auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
} // namespace
DiskLabelIndex::DiskLabelIndex(Indices *indices, Constraints *constraints, const Config &config)
: LabelIndex(indices, constraints, config) {
utils::EnsureDirOrDie(config.disk.label_index_directory);
kvstore_ = std::make_unique<RocksDBStorage>();
kvstore_->options_.create_if_missing = true;
kvstore_->options_.comparator = new ComparatorWithU64TsImpl();
logging::AssertRocksDBStatus(rocksdb::TransactionDB::Open(kvstore_->options_, rocksdb::TransactionDBOptions(),
config.disk.label_index_directory, &kvstore_->db_));
}
bool DiskLabelIndex::CreateIndex(LabelId label, const std::vector<std::pair<std::string, std::string>> &vertices) {
if (!index_.emplace(label).second) {
return false;
}
auto disk_transaction = CreateRocksDBTransaction();
for (const auto &[key, value] : vertices) {
disk_transaction->Put(key, value);
}
return CommitWithTimestamp(disk_transaction.get(), 0);
}
std::unique_ptr<rocksdb::Transaction> DiskLabelIndex::CreateRocksDBTransaction() const {
return std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
}
std::unique_ptr<rocksdb::Transaction> DiskLabelIndex::CreateAllReadingRocksDBTransaction() const {
auto tx = CreateRocksDBTransaction();
tx->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
return tx;
}
bool DiskLabelIndex::SyncVertexToLabelIndexStorage(const Vertex &vertex, uint64_t commit_timestamp) const {
auto disk_transaction = CreateRocksDBTransaction();
if (auto maybe_old_disk_key = utils::GetOldDiskKeyOrNull(vertex.delta); maybe_old_disk_key.has_value()) {
if (!disk_transaction->Delete(maybe_old_disk_key.value()).ok()) {
return false;
}
}
for (const LabelId index_label : index_) {
if (!utils::Contains(vertex.labels, index_label)) {
continue;
}
if (!disk_transaction
->Put(utils::SerializeVertexAsKeyForLabelIndex(index_label, vertex.gid),
utils::SerializeVertexAsValueForLabelIndex(index_label, vertex.labels, vertex.properties))
.ok()) {
return false;
}
}
return CommitWithTimestamp(disk_transaction.get(), commit_timestamp);
}
/// TODO: this can probably be optimized
bool DiskLabelIndex::ClearDeletedVertex(std::string_view gid, uint64_t transaction_commit_timestamp) const {
auto disk_transaction = CreateAllReadingRocksDBTransaction();
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
if (std::string key = it->key().ToString(); gid == utils::ExtractGidFromLabelIndexStorage(key)) {
if (!disk_transaction->Delete(key).ok()) {
return false;
}
}
}
return CommitWithTimestamp(disk_transaction.get(), transaction_commit_timestamp);
}
bool DiskLabelIndex::DeleteVerticesWithRemovedIndexingLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp) {
auto disk_transaction = CreateAllReadingRocksDBTransaction();
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
bool deletion_success = entries_for_deletion.WithLock(
[transaction_start_timestamp, disk_transaction_ptr = disk_transaction.get()](auto &tx_to_entries_for_deletion) {
if (auto tx_it = tx_to_entries_for_deletion.find(transaction_start_timestamp);
tx_it != tx_to_entries_for_deletion.end()) {
bool res = ClearTransactionEntriesWithRemovedIndexingLabel(*disk_transaction_ptr, tx_it->second);
tx_to_entries_for_deletion.erase(tx_it);
return res;
}
return true;
});
if (deletion_success) {
return CommitWithTimestamp(disk_transaction.get(), transaction_commit_timestamp);
}
return false;
}
void DiskLabelIndex::UpdateOnAddLabel(LabelId added_label, Vertex *vertex_before_update, const Transaction &tx) {
entries_for_deletion.WithLock([added_label, vertex_before_update, &tx](auto &tx_to_entries_for_deletion) {
auto tx_it = tx_to_entries_for_deletion.find(tx.start_timestamp);
if (tx_it == tx_to_entries_for_deletion.end()) {
return;
}
auto vertex_label_index_it = tx_it->second.find(vertex_before_update->gid);
if (vertex_label_index_it == tx_it->second.end()) {
return;
}
std::erase_if(vertex_label_index_it->second,
[added_label](const LabelId &indexed_label) { return indexed_label == added_label; });
});
}
void DiskLabelIndex::UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_before_update, const Transaction &tx) {
if (!IndexExists(removed_label)) {
return;
}
entries_for_deletion.WithLock([&removed_label, &tx, vertex_before_update](auto &tx_to_entries_for_deletion) {
auto [it, _] = tx_to_entries_for_deletion.emplace(
std::piecewise_construct, std::forward_as_tuple(tx.start_timestamp), std::forward_as_tuple());
auto &vertex_map_store = it->second;
auto [it_vertex_map_store, emplaced] = vertex_map_store.emplace(
std::piecewise_construct, std::forward_as_tuple(vertex_before_update->gid), std::forward_as_tuple());
it_vertex_map_store->second.emplace_back(removed_label);
});
}
/// TODO: andi Here will come Bloom filter deletion
bool DiskLabelIndex::DropIndex(LabelId label) {
if (!(index_.erase(label) > 0)) {
return false;
}
auto disk_transaction = CreateAllReadingRocksDBTransaction();
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
std::string key = it->key().ToString();
if (key.starts_with(utils::SerializeIdType(label))) {
disk_transaction->Delete(it->key().ToString());
}
}
return CommitWithTimestamp(disk_transaction.get(), 0);
}
bool DiskLabelIndex::IndexExists(LabelId label) const { return index_.find(label) != index_.end(); }
std::vector<LabelId> DiskLabelIndex::ListIndices() const { return {index_.begin(), index_.end()}; }
uint64_t DiskLabelIndex::ApproximateVertexCount(LabelId /*label*/) const { return 10; }
void DiskLabelIndex::LoadIndexInfo(const std::vector<std::string> &labels) {
for (const std::string &label : labels) {
LabelId label_id = LabelId::FromUint(std::stoull(label));
index_.insert(label_id);
}
}
RocksDBStorage *DiskLabelIndex::GetRocksDBStorage() const { return kvstore_.get(); }
} // namespace memgraph::storage

View File

@ -0,0 +1,62 @@
// 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 <rocksdb/iterator.h>
#include <rocksdb/utilities/transaction.h>
#include "storage/v2/disk/rocksdb_storage.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices/label_index.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/rocksdb_serialization.hpp"
namespace memgraph::storage {
class DiskLabelIndex : public storage::LabelIndex {
public:
DiskLabelIndex(Indices *indices, Constraints *constraints, const Config &config);
[[nodiscard]] bool CreateIndex(LabelId label, const std::vector<std::pair<std::string, std::string>> &vertices);
std::unique_ptr<rocksdb::Transaction> CreateRocksDBTransaction() const;
std::unique_ptr<rocksdb::Transaction> CreateAllReadingRocksDBTransaction() const;
[[nodiscard]] bool SyncVertexToLabelIndexStorage(const Vertex &vertex, uint64_t commit_timestamp) const;
[[nodiscard]] bool ClearDeletedVertex(std::string_view gid, uint64_t transaction_commit_timestamp) const;
[[nodiscard]] bool DeleteVerticesWithRemovedIndexingLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp);
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_before_update, const Transaction &tx) override;
void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_before_update, const Transaction &tx) override;
/// Returns false if there was no index to drop
bool DropIndex(LabelId label) override;
bool IndexExists(LabelId label) const override;
std::vector<LabelId> ListIndices() const override;
uint64_t ApproximateVertexCount(LabelId label) const override;
RocksDBStorage *GetRocksDBStorage() const;
void LoadIndexInfo(const std::vector<std::string> &labels);
private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<LabelId>>>> entries_for_deletion;
std::unordered_set<LabelId> index_;
std::unique_ptr<RocksDBStorage> kvstore_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,228 @@
// 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.
/// TODO: clear dependencies
#include "storage/v2/disk/label_property_index.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/inmemory/indices_utils.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/disk_utils.hpp"
#include "utils/exceptions.hpp"
#include "utils/file.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage {
namespace {
bool IsVertexIndexedByLabelProperty(const Vertex &vertex, LabelId label, PropertyId property) {
return utils::Contains(vertex.labels, label) && vertex.properties.HasProperty(property);
}
[[nodiscard]] bool ClearTransactionEntriesWithRemovedIndexingLabel(
rocksdb::Transaction &disk_transaction,
const std::map<Gid, std::vector<std::pair<LabelId, PropertyId>>> &transaction_entries) {
for (const auto &[vertex_gid, index] : transaction_entries) {
for (const auto &[indexing_label, indexing_property] : index) {
if (auto status = disk_transaction.Delete(
utils::SerializeVertexAsKeyForLabelPropertyIndex(indexing_label, indexing_property, vertex_gid));
!status.ok()) {
return false;
}
}
}
return true;
}
bool CommitWithTimestamp(rocksdb::Transaction *disk_transaction, uint64_t commit_ts) {
disk_transaction->SetCommitTimestamp(commit_ts);
const auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
} // namespace
DiskLabelPropertyIndex::DiskLabelPropertyIndex(Indices *indices, Constraints *constraints, const Config &config)
: LabelPropertyIndex(indices, constraints, config) {
utils::EnsureDirOrDie(config.disk.label_property_index_directory);
kvstore_ = std::make_unique<RocksDBStorage>();
kvstore_->options_.create_if_missing = true;
kvstore_->options_.comparator = new ComparatorWithU64TsImpl();
logging::AssertRocksDBStatus(rocksdb::TransactionDB::Open(
kvstore_->options_, rocksdb::TransactionDBOptions(), config.disk.label_property_index_directory, &kvstore_->db_));
}
bool DiskLabelPropertyIndex::CreateIndex(LabelId label, PropertyId property,
const std::vector<std::pair<std::string, std::string>> &vertices) {
if (!index_.emplace(label, property).second) {
return false;
}
auto disk_transaction = CreateRocksDBTransaction();
for (const auto &[key, value] : vertices) {
disk_transaction->Put(key, value);
}
return CommitWithTimestamp(disk_transaction.get(), 0);
}
std::unique_ptr<rocksdb::Transaction> DiskLabelPropertyIndex::CreateRocksDBTransaction() const {
return std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
}
std::unique_ptr<rocksdb::Transaction> DiskLabelPropertyIndex::CreateAllReadingRocksDBTransaction() const {
auto tx = CreateRocksDBTransaction();
tx->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
return tx;
}
bool DiskLabelPropertyIndex::SyncVertexToLabelPropertyIndexStorage(const Vertex &vertex,
uint64_t commit_timestamp) const {
auto disk_transaction = CreateRocksDBTransaction();
if (auto maybe_old_disk_key = utils::GetOldDiskKeyOrNull(vertex.delta); maybe_old_disk_key.has_value()) {
if (!disk_transaction->Delete(maybe_old_disk_key.value()).ok()) {
return false;
}
}
for (const auto &[index_label, index_property] : index_) {
if (IsVertexIndexedByLabelProperty(vertex, index_label, index_property)) {
if (!disk_transaction
->Put(utils::SerializeVertexAsKeyForLabelPropertyIndex(index_label, index_property, vertex.gid),
utils::SerializeVertexAsValueForLabelPropertyIndex(index_label, vertex.labels, vertex.properties))
.ok()) {
return false;
}
}
}
return CommitWithTimestamp(disk_transaction.get(), commit_timestamp);
}
bool DiskLabelPropertyIndex::ClearDeletedVertex(std::string_view gid, uint64_t transaction_commit_timestamp) const {
auto disk_transaction = CreateAllReadingRocksDBTransaction();
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
if (std::string key = it->key().ToString(); gid == utils::ExtractGidFromLabelPropertyIndexStorage(key)) {
if (!disk_transaction->Delete(key).ok()) {
return false;
}
}
}
return CommitWithTimestamp(disk_transaction.get(), transaction_commit_timestamp);
}
bool DiskLabelPropertyIndex::DeleteVerticesWithRemovedIndexingLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp) {
auto disk_transaction = CreateAllReadingRocksDBTransaction();
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
bool deletion_success = entries_for_deletion.WithLock(
[transaction_start_timestamp, disk_transaction_ptr = disk_transaction.get()](auto &tx_to_entries_for_deletion) {
if (auto tx_it = tx_to_entries_for_deletion.find(transaction_start_timestamp);
tx_it != tx_to_entries_for_deletion.end()) {
bool res = ClearTransactionEntriesWithRemovedIndexingLabel(*disk_transaction_ptr, tx_it->second);
tx_to_entries_for_deletion.erase(tx_it);
return res;
}
return true;
});
if (deletion_success) {
return CommitWithTimestamp(disk_transaction.get(), transaction_commit_timestamp);
}
return false;
}
void DiskLabelPropertyIndex::UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) {
entries_for_deletion.WithLock([added_label, vertex_after_update, &tx](auto &tx_to_entries_for_deletion) {
auto tx_it = tx_to_entries_for_deletion.find(tx.start_timestamp);
if (tx_it == tx_to_entries_for_deletion.end()) {
return;
}
auto vertex_label_index_it = tx_it->second.find(vertex_after_update->gid);
if (vertex_label_index_it == tx_it->second.end()) {
return;
}
std::erase_if(vertex_label_index_it->second,
[added_label](const std::pair<LabelId, PropertyId> &index) { return index.first == added_label; });
});
}
void DiskLabelPropertyIndex::UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_after_update,
const Transaction &tx) {
for (const auto &index_entry : index_) {
if (index_entry.first != removed_label) {
continue;
}
entries_for_deletion.WithLock([&index_entry, &tx, vertex_after_update](auto &tx_to_entries_for_deletion) {
const auto &[indexing_label, indexing_property] = index_entry;
auto [it, _] = tx_to_entries_for_deletion.emplace(
std::piecewise_construct, std::forward_as_tuple(tx.start_timestamp), std::forward_as_tuple());
auto &vertex_map_store = it->second;
auto [it_vertex_map_store, emplaced] = vertex_map_store.emplace(
std::piecewise_construct, std::forward_as_tuple(vertex_after_update->gid), std::forward_as_tuple());
it_vertex_map_store->second.emplace_back(indexing_label, indexing_property);
});
}
}
/// TODO: andi If stays the same, move it to the hpp
void DiskLabelPropertyIndex::UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) {}
bool DiskLabelPropertyIndex::DropIndex(LabelId label, PropertyId property) {
return index_.erase({label, property}) > 0;
}
bool DiskLabelPropertyIndex::IndexExists(LabelId label, PropertyId property) const {
return utils::Contains(index_, std::make_pair(label, property));
}
std::vector<std::pair<LabelId, PropertyId>> DiskLabelPropertyIndex::ListIndices() const {
return {index_.begin(), index_.end()};
}
uint64_t DiskLabelPropertyIndex::ApproximateVertexCount(LabelId /*label*/, PropertyId /*property*/) const { return 10; }
uint64_t DiskLabelPropertyIndex::ApproximateVertexCount(LabelId /*label*/, PropertyId /*property*/,
const PropertyValue & /*value*/) const {
return 10;
}
uint64_t DiskLabelPropertyIndex::ApproximateVertexCount(
LabelId /*label*/, PropertyId /*property*/, const std::optional<utils::Bound<PropertyValue>> & /*lower*/,
const std::optional<utils::Bound<PropertyValue>> & /*upper*/) const {
return 10;
}
void DiskLabelPropertyIndex::LoadIndexInfo(const std::vector<std::string> &keys) {
for (const auto &label_property : keys) {
std::vector<std::string> label_property_split = utils::Split(label_property, ",");
index_.emplace(std::make_pair(LabelId::FromUint(std::stoull(label_property_split[0])),
PropertyId::FromUint(std::stoull(label_property_split[1]))));
}
}
RocksDBStorage *DiskLabelPropertyIndex::GetRocksDBStorage() const { return kvstore_.get(); }
} // namespace memgraph::storage

View File

@ -0,0 +1,71 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/disk/rocksdb_storage.hpp"
#include "storage/v2/indices/label_property_index.hpp"
namespace memgraph::storage {
/// TODO: andi. Too many copies, extract at one place
using ParalellizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
class DiskLabelPropertyIndex : public storage::LabelPropertyIndex {
public:
DiskLabelPropertyIndex(Indices *indices, Constraints *constraints, const Config &config);
bool CreateIndex(LabelId label, PropertyId property,
const std::vector<std::pair<std::string, std::string>> &vertices);
std::unique_ptr<rocksdb::Transaction> CreateRocksDBTransaction() const;
std::unique_ptr<rocksdb::Transaction> CreateAllReadingRocksDBTransaction() const;
[[nodiscard]] bool SyncVertexToLabelPropertyIndexStorage(const Vertex &vertex, uint64_t commit_timestamp) const;
[[nodiscard]] bool ClearDeletedVertex(std::string_view gid, uint64_t transaction_commit_timestamp) const;
[[nodiscard]] bool DeleteVerticesWithRemovedIndexingLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp);
void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) override;
void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_after_update, const Transaction &tx) override;
void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) override;
bool DropIndex(LabelId label, PropertyId property) override;
bool IndexExists(LabelId label, PropertyId property) const override;
std::vector<std::pair<LabelId, PropertyId>> ListIndices() const override;
uint64_t ApproximateVertexCount(LabelId label, PropertyId property) const override;
uint64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const override;
uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const override;
RocksDBStorage *GetRocksDBStorage() const;
void LoadIndexInfo(const std::vector<std::string> &keys);
private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::vector<std::pair<LabelId, PropertyId>>>>>
entries_for_deletion;
std::set<std::pair<LabelId, PropertyId>> index_;
std::unique_ptr<RocksDBStorage> kvstore_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,111 @@
// 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 <cstddef>
#include <filesystem>
#include <memory>
#include <string>
#include <string_view>
#include "kvstore/kvstore.hpp"
#include "spdlog/spdlog.h"
#include "storage/v2/name_id_mapper.hpp"
#include "utils/logging.hpp"
#include "utils/string.hpp"
namespace memgraph::storage {
/// Implements class adapter. Object adapters are usually better but here we need access to protected members
/// of base class and we don't want to make them public. Also, from the performance perspective, it doesn't matter
/// since we either have dynamic virtual dispatch here or on storage level.
class DiskNameIdMapper final : public NameIdMapper {
public:
explicit DiskNameIdMapper(std::filesystem::path name_to_id_path, std::filesystem::path id_to_name_path)
: name_to_id_storage_(std::make_unique<kvstore::KVStore>(name_to_id_path)),
id_to_name_storage_(std::make_unique<kvstore::KVStore>(id_to_name_path)) {
InitializeFromDisk();
}
uint64_t NameToId(const std::string_view name) override {
if (auto maybe_id = MaybeNameToId(name); maybe_id.has_value()) {
return maybe_id.value();
}
uint64_t res_id = 0;
if (auto maybe_id_from_disk = name_to_id_storage_->Get(std::string(name)); maybe_id_from_disk.has_value()) {
res_id = std::stoull(maybe_id_from_disk.value());
InsertNameIdEntryToCache(std::string(name), res_id);
InsertIdNameEntryToCache(res_id, std::string(name));
} else {
res_id = NameIdMapper::NameToId(name);
MG_ASSERT(id_to_name_storage_->Put(std::to_string(res_id), std::string(name)),
"Failed to store id to name to disk!");
MG_ASSERT(name_to_id_storage_->Put(std::string(name), std::to_string(res_id)),
"Failed to store id to name to disk!");
}
return res_id;
}
const std::string &IdToName(uint64_t id) override {
auto maybe_name = NameIdMapper::MaybeIdToName(id);
if (maybe_name.has_value()) {
return maybe_name.value();
}
auto maybe_name_from_disk = id_to_name_storage_->Get(std::to_string(id));
MG_ASSERT(maybe_name_from_disk.has_value(), "Trying to get a name from disk for an invalid ID!");
InsertIdNameEntryToCache(id, maybe_name_from_disk.value());
return InsertNameIdEntryToCache(maybe_name_from_disk.value(), id);
}
private:
std::optional<std::reference_wrapper<const uint64_t>> MaybeNameToId(const std::string_view name) const {
auto name_to_id_acc = name_to_id_.access();
auto result = name_to_id_acc.find(name);
if (result == name_to_id_acc.end()) {
return std::nullopt;
}
return result->id;
}
const std::string &InsertNameIdEntryToCache(const std::string &name, uint64_t id) {
auto name_to_id_acc = name_to_id_.access();
return name_to_id_acc.insert({std::string(name), id}).first->name;
}
const std::string &InsertIdNameEntryToCache(uint64_t id, const std::string &name) {
auto id_to_name_acc = id_to_name_.access();
return id_to_name_acc.insert({id, std::string(name)}).first->name;
}
void InitializeFromDisk() {
for (auto itr = name_to_id_storage_->begin(); itr != name_to_id_storage_->end(); ++itr) {
auto name = itr->first;
auto id = std::stoull(itr->second);
InsertNameIdEntryToCache(name, id);
counter_.fetch_add(1, std::memory_order_release);
}
for (auto itr = id_to_name_storage_->begin(); itr != id_to_name_storage_->end(); ++itr) {
auto id = std::stoull(itr->first);
auto name = itr->second;
InsertIdNameEntryToCache(id, name);
}
}
std::unique_ptr<kvstore::KVStore> name_to_id_storage_;
std::unique_ptr<kvstore::KVStore> id_to_name_storage_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,83 @@
// 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 "rocksdb_storage.hpp"
#include <string_view>
#include "utils/rocksdb_serialization.hpp"
namespace memgraph::storage {
namespace {
inline rocksdb::Slice StripTimestampFromUserKey(const rocksdb::Slice &user_key, size_t ts_sz) {
rocksdb::Slice ret = user_key;
ret.remove_suffix(ts_sz);
return ret;
}
/// NOTE: Timestamp is encoded as last 8B in user key.
inline rocksdb::Slice ExtractTimestampFromUserKey(const rocksdb::Slice &user_key) {
assert(user_key.size() >= sizeof(uint64_t));
return {user_key.data() + user_key.size() - sizeof(uint64_t), sizeof(uint64_t)};
}
// Extracts global id from user key. User key must be without timestamp.
std::string_view ExtractGidFromUserKey(const rocksdb::Slice &key) {
assert(key.size() >= 2);
auto keyStrView = key.ToStringView();
return keyStrView.substr(keyStrView.find_last_of('|') + 1);
}
} // namespace
ComparatorWithU64TsImpl::ComparatorWithU64TsImpl()
: Comparator(/*ts_sz=*/sizeof(uint64_t)), cmp_without_ts_(rocksdb::BytewiseComparator()) {
assert(cmp_without_ts_->timestamp_size() == 0);
}
int ComparatorWithU64TsImpl::Compare(const rocksdb::Slice &a, const rocksdb::Slice &b) const {
int ret = CompareWithoutTimestamp(a, b);
if (ret != 0) {
return ret;
}
// Compare timestamp.
// For the same user key with different timestamps, larger (newer) timestamp
// comes first.
return CompareTimestamp(ExtractTimestampFromUserKey(b), ExtractTimestampFromUserKey(a));
}
int ComparatorWithU64TsImpl::CompareWithoutTimestamp(const rocksdb::Slice &a, bool a_has_ts, const rocksdb::Slice &b,
bool b_has_ts) const {
const size_t ts_sz = timestamp_size();
assert(!a_has_ts || a.size() >= ts_sz);
assert(!b_has_ts || b.size() >= ts_sz);
rocksdb::Slice lhsUserKey = a_has_ts ? StripTimestampFromUserKey(a, ts_sz) : a;
rocksdb::Slice rhsUserKey = b_has_ts ? StripTimestampFromUserKey(b, ts_sz) : b;
rocksdb::Slice lhsGid = ExtractGidFromUserKey(lhsUserKey);
rocksdb::Slice rhsGid = ExtractGidFromUserKey(rhsUserKey);
return cmp_without_ts_->Compare(lhsGid, rhsGid);
}
int ComparatorWithU64TsImpl::CompareTimestamp(const rocksdb::Slice &ts1, const rocksdb::Slice &ts2) const {
assert(ts1.size() == sizeof(uint64_t));
assert(ts2.size() == sizeof(uint64_t));
uint64_t lhs = utils::DecodeFixed64(ts1.data());
uint64_t rhs = utils::DecodeFixed64(ts2.data());
if (lhs < rhs) {
return -1;
}
if (lhs > rhs) {
return 1;
}
return 0;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,89 @@
// 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 <rocksdb/comparator.h>
#include <rocksdb/db.h>
#include <rocksdb/iterator.h>
#include <rocksdb/options.h>
#include <rocksdb/status.h>
#include <rocksdb/utilities/transaction_db.h>
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_store.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage {
/// TODO: this should be somehow more wrapped inside the storage class so from the software engineering perspective
/// it isn't great to have this here. But for now it is ok.
/// Wraps RocksDB objects inside a struct. Vertex_chandle and edge_chandle are column family handles that may be
/// nullptr. In that case client should take care about them.
struct RocksDBStorage {
explicit RocksDBStorage() {}
RocksDBStorage(const RocksDBStorage &) = delete;
RocksDBStorage &operator=(const RocksDBStorage &) = delete;
RocksDBStorage(RocksDBStorage &&) = delete;
RocksDBStorage &operator=(RocksDBStorage &&) = delete;
~RocksDBStorage() {
delete db_;
db_ = nullptr;
delete options_.comparator;
options_.comparator = nullptr;
}
rocksdb::Options options_;
rocksdb::TransactionDB *db_;
rocksdb::ColumnFamilyHandle *vertex_chandle = nullptr;
rocksdb::ColumnFamilyHandle *edge_chandle = nullptr;
rocksdb::ColumnFamilyHandle *default_chandle = nullptr;
uint64_t ApproximateVertexCount() const {
uint64_t estimate_num_keys = 0;
db_->GetIntProperty(vertex_chandle, "rocksdb.estimate-num-keys", &estimate_num_keys);
return estimate_num_keys;
}
uint64_t ApproximateEdgeCount() const {
uint64_t estimate_num_keys = 0;
db_->GetIntProperty(edge_chandle, "rocksdb.estimate-num-keys", &estimate_num_keys);
return estimate_num_keys;
}
};
/// RocksDB comparator that compares keys with timestamps.
class ComparatorWithU64TsImpl : public rocksdb::Comparator {
public:
explicit ComparatorWithU64TsImpl();
static const char *kClassName() { return "be"; }
const char *Name() const override { return kClassName(); }
void FindShortSuccessor(std::string * /*key*/) const override {}
void FindShortestSeparator(std::string * /*start*/, const rocksdb::Slice & /*limit*/) const override {}
int Compare(const rocksdb::Slice &a, const rocksdb::Slice &b) const override;
using Comparator::CompareWithoutTimestamp;
int CompareWithoutTimestamp(const rocksdb::Slice &a, bool a_has_ts, const rocksdb::Slice &b,
bool b_has_ts) const override;
int CompareTimestamp(const rocksdb::Slice &ts1, const rocksdb::Slice &ts2) const override;
private:
const Comparator *cmp_without_ts_{nullptr};
};
} // namespace memgraph::storage

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,303 @@
// 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 "kvstore/kvstore.hpp"
#include "storage/v2/constraints/constraint_violation.hpp"
#include "storage/v2/disk/rocksdb_storage.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/property_store.hpp"
#include "storage/v2/storage.hpp"
#include "utils/rw_lock.hpp"
#include <rocksdb/db.h>
namespace memgraph::storage {
class DiskStorage final : public Storage {
public:
explicit DiskStorage(Config config = Config());
DiskStorage(const DiskStorage &) = delete;
DiskStorage(DiskStorage &&) = delete;
DiskStorage &operator=(const DiskStorage &) = delete;
DiskStorage &operator=(DiskStorage &&) = delete;
~DiskStorage() override;
class DiskAccessor final : public Storage::Accessor {
private:
friend class DiskStorage;
explicit DiskAccessor(DiskStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
public:
DiskAccessor(const DiskAccessor &) = delete;
DiskAccessor &operator=(const DiskAccessor &) = delete;
DiskAccessor &operator=(DiskAccessor &&other) = delete;
DiskAccessor(DiskAccessor &&other) noexcept;
~DiskAccessor() override;
VertexAccessor CreateVertex() override;
std::optional<VertexAccessor> FindVertex(Gid gid, View view) override;
VerticesIterable Vertices(View view) override;
VerticesIterable Vertices(LabelId label, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override;
uint64_t ApproximateVertexCount() const override;
uint64_t ApproximateVertexCount(LabelId /*label*/) const override { return 10; }
uint64_t ApproximateVertexCount(LabelId /*label*/, PropertyId /*property*/) const override { return 10; }
uint64_t ApproximateVertexCount(LabelId /*label*/, PropertyId /*property*/,
const PropertyValue & /*value*/) const override {
return 10;
}
uint64_t ApproximateVertexCount(LabelId /*label*/, PropertyId /*property*/,
const std::optional<utils::Bound<PropertyValue>> & /*lower*/,
const std::optional<utils::Bound<PropertyValue>> & /*upper*/) const override {
return 10;
}
std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId & /*label*/) const override {
return {};
}
std::optional<storage::LabelPropertyIndexStats> GetIndexStats(
const storage::LabelId & /*label*/, const storage::PropertyId & /*property*/) const override {
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 {
throw utils::NotYetImplemented("DeleteIndexStatsForLabels(labels) is not implemented for DiskStorage.");
}
std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(
const std::span<std::string> /*labels*/) override {
throw utils::NotYetImplemented("DeleteIndexStatsForLabels(labels) is not implemented for DiskStorage.");
}
void SetIndexStats(const storage::LabelId & /*label*/, const LabelIndexStats & /*stats*/) override {
throw utils::NotYetImplemented("SetIndexStats(stats) is not implemented for DiskStorage.");
}
void SetIndexStats(const storage::LabelId & /*label*/, const storage::PropertyId & /*property*/,
const LabelPropertyIndexStats & /*stats*/) override {
throw utils::NotYetImplemented("SetIndexStats(stats) is not implemented for DiskStorage.");
}
/// TODO: It is just marked as deleted but the memory isn't reclaimed because of the in-memory storage
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) override;
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex) override;
void PrefetchInEdges(const VertexAccessor &vertex_acc) override;
void PrefetchOutEdges(const VertexAccessor &vertex_acc) override;
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) override;
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) override;
bool LabelIndexExists(LabelId label) const override {
auto *disk_storage = static_cast<DiskStorage *>(storage_);
return disk_storage->indices_.label_index_->IndexExists(label);
}
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const override {
auto *disk_storage = static_cast<DiskStorage *>(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();
}
ConstraintsInfo ListAllConstraints() const override {
auto *disk_storage = static_cast<DiskStorage *>(storage_);
return disk_storage->ListAllConstraints();
}
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
void Abort() override;
void FinalizeTransaction() override;
std::optional<storage::VertexAccessor> LoadVertexToLabelIndexCache(
const rocksdb::Slice &key, const rocksdb::Slice &value, Delta *index_delta,
utils::SkipList<storage::Vertex>::Accessor index_accessor);
std::optional<storage::VertexAccessor> LoadVertexToMainMemoryCache(const rocksdb::Slice &key,
const rocksdb::Slice &value);
std::optional<storage::VertexAccessor> LoadVertexToLabelPropertyIndexCache(
const rocksdb::Slice &key, const rocksdb::Slice &value, Delta *index_delta,
utils::SkipList<storage::Vertex>::Accessor index_accessor);
std::optional<storage::EdgeAccessor> DeserializeEdge(const rocksdb::Slice &key, const rocksdb::Slice &value);
private:
VertexAccessor CreateVertex(utils::SkipList<Vertex>::Accessor &accessor, storage::Gid gid,
const std::vector<LabelId> &label_ids, PropertyStore &&properties, Delta *delta);
void PrefetchEdges(const auto &prefetch_edge_filter);
Result<EdgeAccessor> CreateEdge(const VertexAccessor *from, const VertexAccessor *to, EdgeTypeId edge_type,
storage::Gid gid, std::string_view properties, const std::string &old_disk_key);
/// Flushes vertices and edges to the disk with the commit timestamp.
/// 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> FlushMainMemoryCache();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> FlushIndexCache();
[[nodiscard]] utils::BasicResult<StorageDataManipulationError, void> CheckVertexConstraintsBeforeCommit(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage) const;
bool WriteVertexToDisk(const Vertex &vertex);
bool WriteEdgeToDisk(EdgeRef edge, const std::string &serializedEdgeKey);
bool DeleteVertexFromDisk(const std::string &vertex);
bool DeleteEdgeFromDisk(const std::string &edge);
/// Main storage
utils::SkipList<storage::Vertex> vertices_;
std::vector<std::unique_ptr<utils::SkipList<storage::Vertex>>> index_storage_;
/// We need them because query context for indexed reading is cleared after the query is done not after the
/// transaction is done
std::vector<std::list<Delta>> index_deltas_storage_;
utils::SkipList<storage::Edge> edges_;
Config::Items config_;
std::vector<std::string> edges_to_delete_;
std::vector<std::pair<std::string, std::string>> vertices_to_delete_;
rocksdb::Transaction *disk_transaction_;
};
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_});
}
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;
private:
void LoadIndexInfoIfExists() const;
/// TODO (andi): Maybe good to separate these methods and durability kvstore into a separate class
bool PersistLabelIndexCreation(LabelId label) const;
bool PersistLabelIndexDeletion(LabelId label) const;
void LoadLabelIndexInfoIfExists() const;
bool PersistLabelPropertyIndexAndExistenceConstraintCreation(LabelId label, PropertyId property,
const char *key) const;
bool PersistLabelPropertyIndexAndExistenceConstraintDeletion(LabelId label, PropertyId property,
const char *key) const;
void LoadLabelPropertyIndexInfoIfExists() const;
void LoadConstraintsInfoIfExists() const;
void LoadExistenceConstraintInfoIfExists() const;
bool PersistUniqueConstraintCreation(LabelId label, const std::set<PropertyId> &properties) const;
bool PersistUniqueConstraintDeletion(LabelId label, const std::set<PropertyId> &properties) const;
void LoadUniqueConstraintInfoIfExists() const;
uint64_t GetDiskSpaceUsage() const;
void LoadTimestampIfExists();
[[nodiscard]] std::optional<ConstraintViolation> CheckExistingVerticesBeforeCreatingExistenceConstraint(
LabelId label, PropertyId property) const;
[[nodiscard]] utils::BasicResult<ConstraintViolation, std::vector<std::pair<std::string, std::string>>>
CheckExistingVerticesBeforeCreatingUniqueConstraint(LabelId label, const std::set<PropertyId> &properties) const;
std::vector<std::pair<std::string, std::string>> SerializeVerticesForLabelIndex(LabelId label);
std::vector<std::pair<std::string, std::string>> SerializeVerticesForLabelPropertyIndex(LabelId label,
PropertyId property);
StorageInfo GetInfo() const override;
void FreeMemory(std::unique_lock<utils::RWLock> /*lock*/) override {}
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
std::unique_ptr<RocksDBStorage> kvstore_;
std::unique_ptr<kvstore::KVStore> durability_kvstore_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,349 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/disk/unique_constraints.hpp"
#include <rocksdb/utilities/transaction.h>
#include <limits>
#include <optional>
#include <tuple>
#include "spdlog/spdlog.h"
#include "storage/v2/constraints/unique_constraints.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/algorithm.hpp"
#include "utils/disk_utils.hpp"
#include "utils/file.hpp"
namespace memgraph::storage {
namespace {
bool IsVertexUnderConstraint(const Vertex &vertex, const LabelId &constraint_label,
const std::set<PropertyId> &constraint_properties) {
return utils::Contains(vertex.labels, constraint_label) && vertex.properties.HasAllProperties(constraint_properties);
}
bool IsDifferentVertexWithSameConstraintLabel(const std::string &key, const Gid gid, const LabelId constraint_label) {
const std::vector<std::string> vertex_parts = utils::Split(key, "|");
if (std::string local_gid = vertex_parts[1]; local_gid == utils::SerializeIdType(gid)) {
return false;
}
return utils::DeserializeConstraintLabelFromUniqueConstraintStorage(key) == constraint_label;
}
[[nodiscard]] bool ClearTransactionEntriesWithRemovedConstraintLabel(
rocksdb::Transaction &disk_transaction,
const std::map<Gid, std::set<std::pair<LabelId, std::set<PropertyId>>>> &transaction_entries) {
for (const auto &[vertex_gid, constraints] : transaction_entries) {
for (const auto &[constraint_label, constraint_properties] : constraints) {
auto key_to_delete = utils::SerializeVertexAsKeyForUniqueConstraint(constraint_label, constraint_properties,
utils::SerializeIdType(vertex_gid));
if (auto status = disk_transaction.Delete(key_to_delete); !status.ok()) {
return false;
}
}
}
return true;
}
} // namespace
DiskUniqueConstraints::DiskUniqueConstraints(const Config &config) {
kvstore_ = std::make_unique<RocksDBStorage>();
utils::EnsureDirOrDie(config.disk.unique_constraints_directory);
kvstore_->options_.create_if_missing = true;
kvstore_->options_.comparator = new ComparatorWithU64TsImpl();
logging::AssertRocksDBStatus(rocksdb::TransactionDB::Open(kvstore_->options_, rocksdb::TransactionDBOptions(),
config.disk.unique_constraints_directory, &kvstore_->db_));
}
bool DiskUniqueConstraints::InsertConstraint(
LabelId label, const std::set<PropertyId> &properties,
const std::vector<std::pair<std::string, std::string>> &vertices_under_constraint) {
if (!constraints_.insert(std::make_pair(label, properties)).second) {
return false;
}
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
for (const auto &[key, value] : vertices_under_constraint) {
disk_transaction->Put(key, value);
}
/// TODO: figure out a better way to handle this
disk_transaction->SetCommitTimestamp(0);
/// TODO: how about extracting to commit
auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
std::optional<ConstraintViolation> DiskUniqueConstraints::Validate(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage) const {
for (const auto &[constraint_label, constraint_properties] : constraints_) {
if (IsVertexUnderConstraint(vertex, constraint_label, constraint_properties)) {
if (auto vertex_check_result =
TestIfVertexSatisifiesUniqueConstraint(vertex, unique_storage, constraint_label, constraint_properties);
vertex_check_result.has_value()) {
return vertex_check_result.value();
}
}
}
return std::nullopt;
}
std::optional<ConstraintViolation> DiskUniqueConstraints::TestIfVertexSatisifiesUniqueConstraint(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage, const LabelId &constraint_label,
const std::set<PropertyId> &constraint_properties) const {
auto property_values = vertex.properties.ExtractPropertyValues(constraint_properties);
/// TODO: better naming. Is vertex unique
if (property_values.has_value() &&
VertexIsUnique(property_values.value(), unique_storage, constraint_label, constraint_properties, vertex.gid)) {
unique_storage.emplace_back(std::move(property_values.value()));
return std::nullopt;
}
return ConstraintViolation{ConstraintViolation::Type::UNIQUE, constraint_label, constraint_properties};
}
bool DiskUniqueConstraints::VertexIsUnique(const std::vector<PropertyValue> &property_values,
const std::vector<std::vector<PropertyValue>> &unique_storage,
const LabelId &constraint_label,
const std::set<PropertyId> &constraint_properties, const Gid gid) const {
if (utils::Contains(unique_storage, property_values)) {
return false;
}
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
disk_transaction->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
if (IsDifferentVertexWithSameConstraintLabel(it->key().ToString(), gid, constraint_label)) {
if (utils::DeserializePropertiesFromUniqueConstraintStorage(it->value().ToString())
.ExtractPropertyValues(constraint_properties) == property_values) {
return false;
}
}
}
return true;
}
bool DiskUniqueConstraints::ClearDeletedVertex(const std::string_view gid,
uint64_t transaction_commit_timestamp) const {
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
disk_transaction->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
if (std::string key = it->key().ToString(); gid == utils::ExtractGidFromUniqueConstraintStorage(key)) {
if (!disk_transaction->Delete(key).ok()) {
return false;
}
}
}
disk_transaction->SetCommitTimestamp(transaction_commit_timestamp);
auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
bool DiskUniqueConstraints::DeleteVerticesWithRemovedConstraintLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp) {
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
disk_transaction->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
bool deletion_success = true;
entries_for_deletion.WithLock([&deletion_success, transaction_start_timestamp,
disk_transaction_ptr = disk_transaction.get()](auto &tx_to_entries_for_deletion) {
if (auto tx_it = tx_to_entries_for_deletion.find(transaction_start_timestamp);
tx_it != tx_to_entries_for_deletion.end()) {
deletion_success = ClearTransactionEntriesWithRemovedConstraintLabel(*disk_transaction_ptr, tx_it->second);
tx_to_entries_for_deletion.erase(tx_it);
}
});
if (deletion_success) {
/// TODO: Extract to some useful method
disk_transaction->SetCommitTimestamp(transaction_commit_timestamp);
auto status = disk_transaction->Commit();
if (!status.ok()) {
/// TODO: better naming
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
spdlog::error("Deletetion of vertices with removed constraint label failed.");
return false;
}
bool DiskUniqueConstraints::SyncVertexToUniqueConstraintsStorage(const Vertex &vertex,
uint64_t commit_timestamp) const {
/// TODO: create method for writing transaction
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
if (auto maybe_old_disk_key = utils::GetOldDiskKeyOrNull(vertex.delta); maybe_old_disk_key.has_value()) {
spdlog::debug("Found old disk key {} for vertex {}", maybe_old_disk_key.value(),
utils::SerializeIdType(vertex.gid));
if (auto status = disk_transaction->Delete(maybe_old_disk_key.value()); !status.ok()) {
return false;
}
}
for (const auto &[constraint_label, constraint_properties] : constraints_) {
if (IsVertexUnderConstraint(vertex, constraint_label, constraint_properties)) {
auto key = utils::SerializeVertexAsKeyForUniqueConstraint(constraint_label, constraint_properties,
utils::SerializeIdType(vertex.gid));
auto value = utils::SerializeVertexAsValueForUniqueConstraint(constraint_label, vertex.labels, vertex.properties);
if (!disk_transaction->Put(key, value).ok()) {
return false;
}
}
}
/// TODO: extract and better message
disk_transaction->SetCommitTimestamp(commit_timestamp);
auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
return status.ok();
}
DiskUniqueConstraints::CreationStatus DiskUniqueConstraints::CheckIfConstraintCanBeCreated(
LabelId label, const std::set<PropertyId> &properties) const {
if (properties.empty()) {
return CreationStatus::EMPTY_PROPERTIES;
}
if (properties.size() > kUniqueConstraintsMaxProperties) {
return CreationStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED;
}
if (utils::Contains(constraints_, std::make_pair(label, properties))) {
return CreationStatus::ALREADY_EXISTS;
}
return CreationStatus::SUCCESS;
};
DiskUniqueConstraints::DeletionStatus DiskUniqueConstraints::DropConstraint(LabelId label,
const std::set<PropertyId> &properties) {
if (auto drop_properties_check_result = UniqueConstraints::CheckPropertiesBeforeDeletion(properties);
drop_properties_check_result != UniqueConstraints::DeletionStatus::SUCCESS) {
return drop_properties_check_result;
}
if (constraints_.erase({label, properties}) > 0) {
return UniqueConstraints::DeletionStatus::SUCCESS;
}
return UniqueConstraints::DeletionStatus::NOT_FOUND;
}
bool DiskUniqueConstraints::ConstraintExists(LabelId label, const std::set<PropertyId> &properties) const {
return utils::Contains(constraints_, std::make_pair(label, properties));
}
void DiskUniqueConstraints::UpdateOnRemoveLabel(LabelId removed_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) {
for (const auto &constraint : constraints_) {
if (constraint.first == removed_label) {
entries_for_deletion.WithLock(
[&constraint, transaction_start_timestamp, &vertex_before_update](auto &tx_to_entries_for_deletion) {
const auto &[constraint_label, constraint_properties] = constraint;
auto [it, _] = tx_to_entries_for_deletion.emplace(
std::piecewise_construct, std::forward_as_tuple(transaction_start_timestamp), std::forward_as_tuple());
auto &vertex_map_store = it->second;
auto [it_vertex_map_store, emplaced] = vertex_map_store.emplace(
std::piecewise_construct, std::forward_as_tuple(vertex_before_update.gid), std::forward_as_tuple());
it_vertex_map_store->second.emplace(constraint_label, constraint_properties);
});
}
}
}
void DiskUniqueConstraints::UpdateOnAddLabel(LabelId added_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) {
entries_for_deletion.WithLock(
[transaction_start_timestamp, &vertex_before_update, added_label](auto &tx_to_entries_for_deletion) {
/// TODO: change to only one if condition and maybe optimize erase if
if (auto tx_it = tx_to_entries_for_deletion.find(transaction_start_timestamp);
tx_it != tx_to_entries_for_deletion.end()) {
if (auto vertex_constraints_it = tx_it->second.find(vertex_before_update.gid);
vertex_constraints_it != tx_it->second.end()) {
std::erase_if(vertex_constraints_it->second,
[added_label](const auto &constraint) { return constraint.first == added_label; });
}
}
});
}
std::vector<std::pair<LabelId, std::set<PropertyId>>> DiskUniqueConstraints::ListConstraints() const {
return {constraints_.begin(), constraints_.end()};
}
void DiskUniqueConstraints::Clear() {
constraints_.clear();
auto disk_transaction = std::unique_ptr<rocksdb::Transaction>(
kvstore_->db_->BeginTransaction(rocksdb::WriteOptions(), rocksdb::TransactionOptions()));
disk_transaction->SetReadTimestampForValidation(std::numeric_limits<uint64_t>::max());
rocksdb::ReadOptions ro;
std::string strTs = utils::StringTimestamp(std::numeric_limits<uint64_t>::max());
rocksdb::Slice ts(strTs);
ro.timestamp = &ts;
auto it = std::unique_ptr<rocksdb::Iterator>(disk_transaction->GetIterator(ro));
for (it->SeekToFirst(); it->Valid(); it->Next()) {
disk_transaction->Delete(it->key().ToString());
}
disk_transaction->SetCommitTimestamp(0);
auto status = disk_transaction->Commit();
if (!status.ok()) {
spdlog::error("rocksdb: {}", status.getState());
}
}
RocksDBStorage *DiskUniqueConstraints::GetRocksDBStorage() const { return kvstore_.get(); }
void DiskUniqueConstraints::LoadUniqueConstraints(const std::vector<std::string> &keys) {
for (const auto &key : keys) {
std::vector<std::string> key_parts = utils::Split(key, ",");
LabelId label = LabelId::FromUint(std::stoull(key_parts[0]));
std::set<PropertyId> properties;
for (int i = 1; i < key_parts.size(); i++) {
properties.insert(PropertyId::FromUint(std::stoull(key_parts[i])));
}
constraints_.emplace(std::make_pair(label, properties));
}
}
} // namespace memgraph::storage

View File

@ -0,0 +1,78 @@
// 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 "storage/v2/config.hpp"
#include "storage/v2/constraints/unique_constraints.hpp"
#include "storage/v2/disk/rocksdb_storage.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/transaction.hpp"
#include "utils/rocksdb_serialization.hpp"
#include "utils/synchronized.hpp"
namespace memgraph::storage {
class DiskUniqueConstraints : public UniqueConstraints {
public:
explicit DiskUniqueConstraints(const Config &config);
CreationStatus CheckIfConstraintCanBeCreated(LabelId label, const std::set<PropertyId> &properties) const;
[[nodiscard]] bool InsertConstraint(
LabelId label, const std::set<PropertyId> &properties,
const std::vector<std::pair<std::string, std::string>> &vertices_under_constraint);
std::optional<ConstraintViolation> Validate(const Vertex &vertex,
std::vector<std::vector<PropertyValue>> &unique_storage) const;
[[nodiscard]] bool ClearDeletedVertex(std::string_view gid, uint64_t transaction_commit_timestamp) const;
[[nodiscard]] bool DeleteVerticesWithRemovedConstraintLabel(uint64_t transaction_start_timestamp,
uint64_t transaction_commit_timestamp);
[[nodiscard]] bool SyncVertexToUniqueConstraintsStorage(const Vertex &vertex, uint64_t commit_timestamp) const;
DeletionStatus DropConstraint(LabelId label, const std::set<PropertyId> &properties) override;
[[nodiscard]] bool ConstraintExists(LabelId label, const std::set<PropertyId> &properties) const override;
void UpdateOnRemoveLabel(LabelId removed_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) override;
void UpdateOnAddLabel(LabelId added_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) override;
std::vector<std::pair<LabelId, std::set<PropertyId>>> ListConstraints() const override;
void Clear() override;
RocksDBStorage *GetRocksDBStorage() const;
void LoadUniqueConstraints(const std::vector<std::string> &keys);
private:
utils::Synchronized<std::map<uint64_t, std::map<Gid, std::set<std::pair<LabelId, std::set<PropertyId>>>>>>
entries_for_deletion;
std::set<std::pair<LabelId, std::set<PropertyId>>> constraints_;
std::unique_ptr<RocksDBStorage> kvstore_;
[[nodiscard]] std::optional<ConstraintViolation> TestIfVertexSatisifiesUniqueConstraint(
const Vertex &vertex, std::vector<std::vector<PropertyValue>> &unique_storage, const LabelId &constraint_label,
const std::set<PropertyId> &constraint_properties) const;
bool VertexIsUnique(const std::vector<PropertyValue> &property_values,
const std::vector<std::vector<PropertyValue>> &unique_storage, const LabelId &constraint_label,
const std::set<PropertyId> &constraint_properties, Gid gid) const;
};
} // namespace memgraph::storage

View File

@ -27,6 +27,7 @@
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/indices/label_property_index.hpp"
#include "utils/event_histogram.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
@ -131,7 +132,8 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
// Recover label indices.
spdlog::info("Recreating {} label indices from metadata.", indices_constraints.indices.label.size());
for (const auto &item : indices_constraints.indices.label) {
if (!indices->label_index.CreateIndex(item, vertices->access(), paralell_exec_info))
auto *mem_label_index = static_cast<InMemoryLabelIndex *>(indices->label_index_.get());
if (!mem_label_index->CreateIndex(item, vertices->access(), paralell_exec_info))
throw RecoveryFailure("The label index must be created here!");
spdlog::info("A label index is recreated from metadata.");
@ -141,8 +143,9 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
// Recover label+property indices.
spdlog::info("Recreating {} label+property indices from metadata.",
indices_constraints.indices.label_property.size());
auto *mem_label_property_index = static_cast<InMemoryLabelPropertyIndex *>(indices->label_property_index_.get());
for (const auto &item : indices_constraints.indices.label_property) {
if (!indices->label_property_index.CreateIndex(item.first, item.second, vertices->access()))
if (!mem_label_property_index->CreateIndex(item.first, item.second, vertices->access(), std::nullopt))
throw RecoveryFailure("The label+property index must be created here!");
spdlog::info("A label+property index is recreated from metadata.");
}
@ -152,9 +155,18 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
spdlog::info("Recreating constraints from metadata.");
// Recover existence constraints.
spdlog::info("Recreating {} existence constraints from metadata.", indices_constraints.constraints.existence.size());
for (const auto &item : indices_constraints.constraints.existence) {
auto ret = CreateExistenceConstraint(constraints, item.first, item.second, vertices->access());
if (ret.HasError() || !ret.GetValue()) throw RecoveryFailure("The existence constraint must be created here!");
for (const auto &[label, property] : indices_constraints.constraints.existence) {
if (constraints->existence_constraints_->ConstraintExists(label, property)) {
throw RecoveryFailure("The existence constraint already exists!");
}
if (auto violation = ExistenceConstraints::ValidateVerticesOnConstraint(vertices->access(), label, property);
violation.has_value()) {
throw RecoveryFailure("The existence constraint failed because it couldn't be validated!");
}
constraints->existence_constraints_->InsertConstraint(label, property);
spdlog::info("A existence constraint is recreated from metadata.");
}
spdlog::info("Existence constraints are recreated from metadata.");
@ -162,7 +174,8 @@ void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_
// Recover unique constraints.
spdlog::info("Recreating {} unique constraints from metadata.", indices_constraints.constraints.unique.size());
for (const auto &item : indices_constraints.constraints.unique) {
auto ret = constraints->unique_constraints.CreateConstraint(item.first, item.second, vertices->access());
auto *mem_unique_constraints = static_cast<InMemoryUniqueConstraints *>(constraints->unique_constraints_.get());
auto ret = mem_unique_constraints->CreateConstraint(item.first, item.second, vertices->access());
if (ret.HasError() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
throw RecoveryFailure("The unique constraint must be created here!");
spdlog::info("A unique constraint is recreated from metadata.");

View File

@ -19,11 +19,11 @@
#include <variant>
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/skip_list.hpp"

View File

@ -1395,6 +1395,7 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
is_visible = true;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
@ -1517,7 +1518,7 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
// Write label indices.
{
auto label = indices->label_index.ListIndices();
auto label = indices->label_index_->ListIndices();
snapshot.WriteUint(label.size());
for (const auto &item : label) {
write_mapping(item);
@ -1526,7 +1527,7 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
// Write label+property indices.
{
auto label_property = indices->label_property_index.ListIndices();
auto label_property = indices->label_property_index_->ListIndices();
snapshot.WriteUint(label_property.size());
for (const auto &item : label_property) {
write_mapping(item.first);
@ -1542,7 +1543,7 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
// Write existence constraints.
{
auto existence = ListExistenceConstraints(*constraints);
auto existence = constraints->existence_constraints_->ListConstraints();
snapshot.WriteUint(existence.size());
for (const auto &item : existence) {
write_mapping(item.first);
@ -1552,7 +1553,7 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
// Write unique constraints.
{
auto unique = constraints->unique_constraints.ListConstraints();
auto unique = constraints->unique_constraints_->ListConstraints();
snapshot.WriteUint(unique.size());
for (const auto &item : unique) {
write_mapping(item.first);

View File

@ -16,10 +16,10 @@
#include <string>
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"

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
@ -101,6 +101,7 @@ Marker VertexActionToMarker(Delta::Action action) {
// because the Delta's represent undo actions and we want to store redo
// actions.
switch (action) {
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
return Marker::DELTA_VERTEX_CREATE;
case Delta::Action::RECREATE_OBJECT:
@ -491,6 +492,7 @@ void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, Config::Ite
encoder->WriteUint(timestamp);
std::lock_guard<utils::SpinLock> guard(vertex.lock);
switch (delta.action) {
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
case Delta::Action::RECREATE_OBJECT: {
encoder->WriteMarker(VertexActionToMarker(delta.action));
@ -558,6 +560,7 @@ void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, const Delta
encoder->WritePropertyValue(edge.properties.GetProperty(delta.property.key));
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
case Delta::Action::RECREATE_OBJECT:
// These actions are already encoded in vertex *_OUT_EDGE actions. Also,

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
@ -25,7 +25,8 @@ struct Vertex;
struct Edge {
Edge(Gid gid, Delta *delta) : gid(gid), deleted(false), delta(delta) {
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT,
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT ||
delta->action == Delta::Action::DELETE_DESERIALIZED_OBJECT,
"Edge must be created with an initial DELETE_OBJECT delta!");
}

View File

@ -14,6 +14,7 @@
#include <memory>
#include <tuple>
#include "storage/v2/delta.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/vertex_accessor.hpp"
@ -44,6 +45,7 @@ bool EdgeAccessor::IsVisible(const View view) const {
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::RECREATE_OBJECT:
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
break;
case Delta::Action::ADD_OUT_EDGE: { // relevant for the from_vertex_ -> we just deleted the edge
@ -83,6 +85,7 @@ bool EdgeAccessor::IsVisible(const View view) const {
deleted = false;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -181,6 +184,7 @@ Result<PropertyValue> EdgeAccessor::GetProperty(PropertyId property, View view)
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -232,6 +236,7 @@ Result<std::map<PropertyId, PropertyValue>> EdgeAccessor::Properties(View view)
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;

View File

@ -88,7 +88,6 @@ class EdgeAccessor final {
}
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
private:
EdgeRef edge_;
EdgeTypeId edge_type_;
Vertex *from_vertex_;

View File

@ -1,907 +0,0 @@
// 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 "indices.hpp"
#include <algorithm>
#include <iterator>
#include <limits>
#include <thread>
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/bound.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
#include "utils/synchronized.hpp"
namespace memgraph::storage {
namespace {
/// Traverses deltas visible from transaction with start timestamp greater than
/// the provided timestamp, and calls the provided callback function for each
/// delta. If the callback ever returns true, traversal is stopped and the
/// function returns true. Otherwise, the function returns false.
template <typename TCallback>
bool AnyVersionSatisfiesPredicate(uint64_t timestamp, const Delta *delta, const TCallback &predicate) {
while (delta != nullptr) {
auto ts = delta->timestamp->load(std::memory_order_acquire);
// This is a committed change that we see so we shouldn't undo it.
if (ts < timestamp) {
break;
}
if (predicate(*delta)) {
return true;
}
// Move to the next delta.
delta = delta->next.load(std::memory_order_acquire);
}
return false;
}
/// Helper function for label index garbage collection. Returns true if there's
/// a reachable version of the vertex that has the given label.
bool AnyVersionHasLabel(const Vertex &vertex, LabelId label, uint64_t timestamp) {
bool has_label;
bool deleted;
const Delta *delta;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
has_label = utils::Contains(vertex.labels, label);
deleted = vertex.deleted;
delta = vertex.delta;
}
if (!deleted && has_label) {
return true;
}
return AnyVersionSatisfiesPredicate(timestamp, delta, [&has_label, &deleted, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
return !deleted && has_label;
});
}
/// Helper function for label-property index garbage collection. Returns true if
/// there's a reachable version of the vertex that has the given label and
/// property value.
bool AnyVersionHasLabelProperty(const Vertex &vertex, LabelId label, PropertyId key, const PropertyValue &value,
uint64_t timestamp) {
bool has_label;
bool current_value_equal_to_value = value.IsNull();
bool deleted;
const Delta *delta;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
has_label = utils::Contains(vertex.labels, label);
current_value_equal_to_value = vertex.properties.IsPropertyEqual(key, value);
deleted = vertex.deleted;
delta = vertex.delta;
}
if (!deleted && has_label && current_value_equal_to_value) {
return true;
}
return AnyVersionSatisfiesPredicate(
timestamp, delta, [&has_label, &current_value_equal_to_value, &deleted, label, key, &value](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::SET_PROPERTY:
if (delta.property.key == key) {
current_value_equal_to_value = delta.property.value == value;
}
break;
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
return !deleted && has_label && current_value_equal_to_value;
});
}
// Helper function for iterating through label index. Returns true if this
// transaction can see the given vertex, and the visible version has the given
// label.
bool CurrentVersionHasLabel(const Vertex &vertex, LabelId label, Transaction *transaction, View view) {
bool deleted;
bool has_label;
const Delta *delta;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
deleted = vertex.deleted;
has_label = utils::Contains(vertex.labels, label);
delta = vertex.delta;
}
ApplyDeltasForRead(transaction, delta, view, [&deleted, &has_label, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
}
case Delta::Action::ADD_LABEL: {
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
return !deleted && has_label;
}
// Helper function for iterating through label-property index. Returns true if
// this transaction can see the given vertex, and the visible version has the
// given label and property.
bool CurrentVersionHasLabelProperty(const Vertex &vertex, LabelId label, PropertyId key, const PropertyValue &value,
Transaction *transaction, View view) {
bool deleted;
bool has_label;
bool current_value_equal_to_value = value.IsNull();
const Delta *delta;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
deleted = vertex.deleted;
has_label = utils::Contains(vertex.labels, label);
current_value_equal_to_value = vertex.properties.IsPropertyEqual(key, value);
delta = vertex.delta;
}
ApplyDeltasForRead(transaction, delta, view,
[&deleted, &has_label, &current_value_equal_to_value, key, label, &value](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == key) {
current_value_equal_to_value = delta.property.value == value;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
return !deleted && has_label && current_value_equal_to_value;
}
template <typename TIndexAccessor>
void TryInsertLabelIndex(Vertex &vertex, LabelId label, TIndexAccessor &index_accessor) {
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
return;
}
index_accessor.insert({&vertex, 0});
}
template <typename TIndexAccessor>
void TryInsertLabelPropertyIndex(Vertex &vertex, std::pair<LabelId, PropertyId> label_property_pair,
TIndexAccessor &index_accessor) {
if (vertex.deleted || !utils::Contains(vertex.labels, label_property_pair.first)) {
return;
}
auto value = vertex.properties.GetProperty(label_property_pair.second);
if (value.IsNull()) {
return;
}
index_accessor.insert({std::move(value), &vertex, 0});
}
template <typename TSkiplistIter, typename TIndex, typename TIndexKey, typename TFunc>
void CreateIndexOnSingleThread(utils::SkipList<Vertex>::Accessor &vertices, TSkiplistIter it, TIndex &index,
TIndexKey key, const TFunc &func) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
try {
auto acc = it->second.access();
for (Vertex &vertex : vertices) {
func(vertex, key, acc);
}
} catch (const utils::OutOfMemoryException &) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index.erase(it);
throw;
}
}
template <typename TIndex, typename TIndexKey, typename TSKiplistIter, typename TFunc>
void CreateIndexOnMultipleThreads(utils::SkipList<Vertex>::Accessor &vertices, TSKiplistIter skiplist_iter,
TIndex &index, TIndexKey key, const ParalellizedIndexCreationInfo &paralell_exec_info,
const TFunc &func) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
const auto &vertex_batches = paralell_exec_info.first;
const auto thread_count = std::min(paralell_exec_info.second, vertex_batches.size());
MG_ASSERT(!vertex_batches.empty(),
"The size of batches should always be greater than zero if you want to use the parallel version of index "
"creation!");
std::atomic<uint64_t> batch_counter = 0;
utils::Synchronized<std::optional<utils::OutOfMemoryException>, utils::SpinLock> maybe_error{};
{
std::vector<std::jthread> threads;
threads.reserve(thread_count);
for (auto i{0U}; i < thread_count; ++i) {
threads.emplace_back(
[&skiplist_iter, &func, &index, &vertex_batches, &maybe_error, &batch_counter, &key, &vertices]() {
while (!maybe_error.Lock()->has_value()) {
const auto batch_index = batch_counter++;
if (batch_index >= vertex_batches.size()) {
return;
}
const auto &batch = vertex_batches[batch_index];
auto index_accessor = index.at(key).access();
auto it = vertices.find(batch.first);
try {
for (auto i{0U}; i < batch.second; ++i, ++it) {
func(*it, key, index_accessor);
}
} catch (utils::OutOfMemoryException &failure) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index.erase(skiplist_iter);
*maybe_error.Lock() = std::move(failure);
}
}
});
}
}
if (maybe_error.Lock()->has_value()) {
throw utils::OutOfMemoryException((*maybe_error.Lock())->what());
}
}
} // namespace
void LabelIndex::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) {
auto it = index_.find(label);
if (it == index_.end()) return;
auto acc = it->second.access();
acc.insert(Entry{vertex, tx.start_timestamp});
}
bool LabelIndex::CreateIndex(LabelId label, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info) {
auto create_index_seq = [this](LabelId label, utils::SkipList<Vertex>::Accessor &vertices,
std::map<LabelId, utils::SkipList<Entry>>::iterator it) {
using IndexAccessor = decltype(it->second.access());
CreateIndexOnSingleThread(vertices, it, index_, label,
[](Vertex &vertex, LabelId label, IndexAccessor &index_accessor) {
TryInsertLabelIndex(vertex, label, index_accessor);
});
return true;
};
auto create_index_par = [this](LabelId label, utils::SkipList<Vertex>::Accessor &vertices,
std::map<LabelId, utils::SkipList<Entry>>::iterator label_it,
const ParalellizedIndexCreationInfo &paralell_exec_info) {
using IndexAccessor = decltype(label_it->second.access());
CreateIndexOnMultipleThreads(vertices, label_it, index_, label, paralell_exec_info,
[](Vertex &vertex, LabelId label, IndexAccessor &index_accessor) {
TryInsertLabelIndex(vertex, label, index_accessor);
});
return true;
};
auto [it, emplaced] = index_.emplace(std::piecewise_construct, std::forward_as_tuple(label), std::forward_as_tuple());
if (!emplaced) {
// Index already exists.
return false;
}
if (paralell_exec_info) {
return create_index_par(label, vertices, it, *paralell_exec_info);
}
return create_index_seq(label, vertices, it);
}
std::vector<LabelId> LabelIndex::ListIndices() const {
std::vector<LabelId> ret;
ret.reserve(index_.size());
for (const auto &item : index_) {
ret.push_back(item.first);
}
return ret;
}
void LabelIndex::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
for (auto &label_storage : index_) {
auto vertices_acc = label_storage.second.access();
for (auto it = vertices_acc.begin(); it != vertices_acc.end();) {
auto next_it = it;
++next_it;
if (it->timestamp >= oldest_active_start_timestamp) {
it = next_it;
continue;
}
if ((next_it != vertices_acc.end() && it->vertex == next_it->vertex) ||
!AnyVersionHasLabel(*it->vertex, label_storage.first, oldest_active_start_timestamp)) {
vertices_acc.remove(*it);
}
it = next_it;
}
}
}
LabelIndex::Iterable::Iterator::Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator)
: self_(self),
index_iterator_(index_iterator),
current_vertex_accessor_(nullptr, nullptr, nullptr, nullptr, self_->config_),
current_vertex_(nullptr) {
AdvanceUntilValid();
}
LabelIndex::Iterable::Iterator &LabelIndex::Iterable::Iterator::operator++() {
++index_iterator_;
AdvanceUntilValid();
return *this;
}
void LabelIndex::Iterable::Iterator::AdvanceUntilValid() {
for (; index_iterator_ != self_->index_accessor_.end(); ++index_iterator_) {
if (index_iterator_->vertex == current_vertex_) {
continue;
}
if (CurrentVersionHasLabel(*index_iterator_->vertex, self_->label_, self_->transaction_, self_->view_)) {
current_vertex_ = index_iterator_->vertex;
current_vertex_accessor_ =
VertexAccessor{current_vertex_, self_->transaction_, self_->indices_, self_->constraints_, self_->config_};
break;
}
}
}
LabelIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, View view,
Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config)
: index_accessor_(std::move(index_accessor)),
label_(label),
view_(view),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config) {}
void LabelIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
void LabelIndex::SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats) {
stats_[label] = stats;
}
std::optional<LabelIndexStats> LabelIndex::GetIndexStats(const storage::LabelId &label) const {
if (auto it = stats_.find(label); it != stats_.end()) {
return it->second;
}
return {};
}
std::vector<LabelId> LabelIndex::ClearIndexStats() {
std::vector<LabelId> deleted_indexes;
deleted_indexes.reserve(stats_.size());
std::transform(stats_.begin(), stats_.end(), std::back_inserter(deleted_indexes),
[](const auto &elem) { return elem.first; });
stats_.clear();
return deleted_indexes;
}
std::vector<LabelId> LabelIndex::DeleteIndexStats(const storage::LabelId &label) {
std::vector<LabelId> deleted_indexes;
for (auto it = stats_.cbegin(); it != stats_.cend();) {
if (it->first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
} else {
++it;
}
}
return deleted_indexes;
}
bool LabelPropertyIndex::Entry::operator<(const Entry &rhs) {
if (value < rhs.value) {
return true;
}
if (rhs.value < value) {
return false;
}
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool LabelPropertyIndex::Entry::operator==(const Entry &rhs) {
return value == rhs.value && vertex == rhs.vertex && timestamp == rhs.timestamp;
}
bool LabelPropertyIndex::Entry::operator<(const PropertyValue &rhs) { return value < rhs; }
bool LabelPropertyIndex::Entry::operator==(const PropertyValue &rhs) { return value == rhs; }
void LabelPropertyIndex::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) {
for (auto &[label_prop, storage] : index_) {
if (label_prop.first != label) {
continue;
}
auto prop_value = vertex->properties.GetProperty(label_prop.second);
if (!prop_value.IsNull()) {
auto acc = storage.access();
acc.insert(Entry{std::move(prop_value), vertex, tx.start_timestamp});
}
}
}
void LabelPropertyIndex::UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) {
if (value.IsNull()) {
return;
}
for (auto &[label_prop, storage] : index_) {
if (label_prop.second != property) {
continue;
}
if (utils::Contains(vertex->labels, label_prop.first)) {
auto acc = storage.access();
acc.insert(Entry{value, vertex, tx.start_timestamp});
}
}
}
bool LabelPropertyIndex::CreateIndex(LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info) {
auto create_index_seq = [this](LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor &vertices,
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>>::iterator it) {
using IndexAccessor = decltype(it->second.access());
CreateIndexOnSingleThread(vertices, it, index_, std::make_pair(label, property),
[](Vertex &vertex, std::pair<LabelId, PropertyId> key, IndexAccessor &index_accessor) {
TryInsertLabelPropertyIndex(vertex, key, index_accessor);
});
return true;
};
auto create_index_par =
[this](LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor &vertices,
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>>::iterator label_property_it,
const ParalellizedIndexCreationInfo &paralell_exec_info) {
using IndexAccessor = decltype(label_property_it->second.access());
CreateIndexOnMultipleThreads(
vertices, label_property_it, index_, std::make_pair(label, property), paralell_exec_info,
[](Vertex &vertex, std::pair<LabelId, PropertyId> key, IndexAccessor &index_accessor) {
TryInsertLabelPropertyIndex(vertex, key, index_accessor);
});
return true;
};
auto [it, emplaced] =
index_.emplace(std::piecewise_construct, std::forward_as_tuple(label, property), std::forward_as_tuple());
if (!emplaced) {
// Index already exists.
return false;
}
if (paralell_exec_info) {
return create_index_par(label, property, vertices, it, *paralell_exec_info);
}
return create_index_seq(label, property, vertices, it);
}
std::vector<std::pair<LabelId, PropertyId>> LabelPropertyIndex::ListIndices() const {
std::vector<std::pair<LabelId, PropertyId>> ret;
ret.reserve(index_.size());
for (const auto &item : index_) {
ret.push_back(item.first);
}
return ret;
}
void LabelPropertyIndex::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
for (auto &[label_property, index] : index_) {
auto index_acc = index.access();
for (auto it = index_acc.begin(); it != index_acc.end();) {
auto next_it = it;
++next_it;
if (it->timestamp >= oldest_active_start_timestamp) {
it = next_it;
continue;
}
if ((next_it != index_acc.end() && it->vertex == next_it->vertex && it->value == next_it->value) ||
!AnyVersionHasLabelProperty(*it->vertex, label_property.first, label_property.second, it->value,
oldest_active_start_timestamp)) {
index_acc.remove(*it);
}
it = next_it;
}
}
}
LabelPropertyIndex::Iterable::Iterator::Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator)
: self_(self),
index_iterator_(index_iterator),
current_vertex_accessor_(nullptr, nullptr, nullptr, nullptr, self_->config_),
current_vertex_(nullptr) {
AdvanceUntilValid();
}
LabelPropertyIndex::Iterable::Iterator &LabelPropertyIndex::Iterable::Iterator::operator++() {
++index_iterator_;
AdvanceUntilValid();
return *this;
}
void LabelPropertyIndex::Iterable::Iterator::AdvanceUntilValid() {
for (; index_iterator_ != self_->index_accessor_.end(); ++index_iterator_) {
if (index_iterator_->vertex == current_vertex_) {
continue;
}
if (self_->lower_bound_) {
if (index_iterator_->value < self_->lower_bound_->value()) {
continue;
}
if (!self_->lower_bound_->IsInclusive() && index_iterator_->value == self_->lower_bound_->value()) {
continue;
}
}
if (self_->upper_bound_) {
if (self_->upper_bound_->value() < index_iterator_->value) {
index_iterator_ = self_->index_accessor_.end();
break;
}
if (!self_->upper_bound_->IsInclusive() && index_iterator_->value == self_->upper_bound_->value()) {
index_iterator_ = self_->index_accessor_.end();
break;
}
}
if (CurrentVersionHasLabelProperty(*index_iterator_->vertex, self_->label_, self_->property_,
index_iterator_->value, self_->transaction_, self_->view_)) {
current_vertex_ = index_iterator_->vertex;
current_vertex_accessor_ =
VertexAccessor(current_vertex_, self_->transaction_, self_->indices_, self_->constraints_, self_->config_);
break;
}
}
}
// These constants represent the smallest possible value of each type that is
// contained in a `PropertyValue`. Note that numbers (integers and doubles) are
// treated as the same "type" in `PropertyValue`.
const PropertyValue kSmallestBool = PropertyValue(false);
static_assert(-std::numeric_limits<double>::infinity() < std::numeric_limits<int64_t>::min());
const PropertyValue kSmallestNumber = PropertyValue(-std::numeric_limits<double>::infinity());
const PropertyValue kSmallestString = PropertyValue("");
const PropertyValue kSmallestList = PropertyValue(std::vector<PropertyValue>());
const PropertyValue kSmallestMap = PropertyValue(std::map<std::string, PropertyValue>());
const PropertyValue kSmallestTemporalData =
PropertyValue(TemporalData{static_cast<TemporalType>(0), std::numeric_limits<int64_t>::min()});
LabelPropertyIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label,
PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view,
Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config)
: index_accessor_(std::move(index_accessor)),
label_(label),
property_(property),
lower_bound_(lower_bound),
upper_bound_(upper_bound),
view_(view),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config) {
// We have to fix the bounds that the user provided to us. If the user
// provided only one bound we should make sure that only values of that type
// are returned by the iterator. We ensure this by supplying either an
// inclusive lower bound of the same type, or an exclusive upper bound of the
// following type. If neither bound is set we yield all items in the index.
// First we statically verify that our assumptions about the `PropertyValue`
// type ordering holds.
static_assert(PropertyValue::Type::Bool < PropertyValue::Type::Int);
static_assert(PropertyValue::Type::Int < PropertyValue::Type::Double);
static_assert(PropertyValue::Type::Double < PropertyValue::Type::String);
static_assert(PropertyValue::Type::String < PropertyValue::Type::List);
static_assert(PropertyValue::Type::List < PropertyValue::Type::Map);
// Remove any bounds that are set to `Null` because that isn't a valid value.
if (lower_bound_ && lower_bound_->value().IsNull()) {
lower_bound_ = std::nullopt;
}
if (upper_bound_ && upper_bound_->value().IsNull()) {
upper_bound_ = std::nullopt;
}
// Check whether the bounds are of comparable types if both are supplied.
if (lower_bound_ && upper_bound_ &&
!PropertyValue::AreComparableTypes(lower_bound_->value().type(), upper_bound_->value().type())) {
bounds_valid_ = false;
return;
}
// Set missing bounds.
if (lower_bound_ && !upper_bound_) {
// Here we need to supply an upper bound. The upper bound is set to an
// exclusive lower bound of the following type.
switch (lower_bound_->value().type()) {
case PropertyValue::Type::Null:
// This shouldn't happen because of the nullopt-ing above.
LOG_FATAL("Invalid database state!");
break;
case PropertyValue::Type::Bool:
upper_bound_ = utils::MakeBoundExclusive(kSmallestNumber);
break;
case PropertyValue::Type::Int:
case PropertyValue::Type::Double:
// Both integers and doubles are treated as the same type in
// `PropertyValue` and they are interleaved when sorted.
upper_bound_ = utils::MakeBoundExclusive(kSmallestString);
break;
case PropertyValue::Type::String:
upper_bound_ = utils::MakeBoundExclusive(kSmallestList);
break;
case PropertyValue::Type::List:
upper_bound_ = utils::MakeBoundExclusive(kSmallestMap);
break;
case PropertyValue::Type::Map:
upper_bound_ = utils::MakeBoundExclusive(kSmallestTemporalData);
break;
case PropertyValue::Type::TemporalData:
// This is the last type in the order so we leave the upper bound empty.
break;
}
}
if (upper_bound_ && !lower_bound_) {
// Here we need to supply a lower bound. The lower bound is set to an
// inclusive lower bound of the current type.
switch (upper_bound_->value().type()) {
case PropertyValue::Type::Null:
// This shouldn't happen because of the nullopt-ing above.
LOG_FATAL("Invalid database state!");
break;
case PropertyValue::Type::Bool:
lower_bound_ = utils::MakeBoundInclusive(kSmallestBool);
break;
case PropertyValue::Type::Int:
case PropertyValue::Type::Double:
// Both integers and doubles are treated as the same type in
// `PropertyValue` and they are interleaved when sorted.
lower_bound_ = utils::MakeBoundInclusive(kSmallestNumber);
break;
case PropertyValue::Type::String:
lower_bound_ = utils::MakeBoundInclusive(kSmallestString);
break;
case PropertyValue::Type::List:
lower_bound_ = utils::MakeBoundInclusive(kSmallestList);
break;
case PropertyValue::Type::Map:
lower_bound_ = utils::MakeBoundInclusive(kSmallestMap);
break;
case PropertyValue::Type::TemporalData:
lower_bound_ = utils::MakeBoundInclusive(kSmallestTemporalData);
break;
}
}
}
LabelPropertyIndex::Iterable::Iterator LabelPropertyIndex::Iterable::begin() {
// If the bounds are set and don't have comparable types we don't yield any
// items from the index.
if (!bounds_valid_) return Iterator(this, index_accessor_.end());
auto index_iterator = index_accessor_.begin();
if (lower_bound_) {
index_iterator = index_accessor_.find_equal_or_greater(lower_bound_->value());
}
return Iterator(this, index_iterator);
}
LabelPropertyIndex::Iterable::Iterator LabelPropertyIndex::Iterable::end() {
return Iterator(this, index_accessor_.end());
}
int64_t LabelPropertyIndex::ApproximateVertexCount(LabelId label, PropertyId property,
const PropertyValue &value) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
auto acc = it->second.access();
if (!value.IsNull()) {
return acc.estimate_count(value, utils::SkipListLayerForCountEstimation(acc.size()));
} else {
// The value `Null` won't ever appear in the index because it indicates that
// the property shouldn't exist. Instead, this value is used as an indicator
// to estimate the average number of equal elements in the list (for any
// given value).
return acc.estimate_average_number_of_equals(
[](const auto &first, const auto &second) { return first.value == second.value; },
utils::SkipListLayerForAverageEqualsEstimation(acc.size()));
}
}
int64_t LabelPropertyIndex::ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
auto acc = it->second.access();
return acc.estimate_range_count(lower, upper, utils::SkipListLayerForCountEstimation(acc.size()));
}
/*
Iterate over all property-label pairs and deletes if label from the index is equal to label parameter.
*/
std::vector<std::pair<LabelId, PropertyId>> LabelPropertyIndex::DeleteIndexStats(const storage::LabelId &label) {
std::vector<std::pair<LabelId, PropertyId>> deleted_indexes;
for (auto it = stats_.cbegin(); it != stats_.cend();) {
if (it->first.first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
} else {
++it;
}
}
return deleted_indexes;
}
std::vector<std::pair<LabelId, PropertyId>> LabelPropertyIndex::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),
[](const auto &elem) { return elem.first; });
stats_.clear();
return deleted_indexes;
}
void LabelPropertyIndex::SetIndexStats(const std::pair<storage::LabelId, storage::PropertyId> &key,
const storage::LabelPropertyIndexStats &stats) {
stats_[key] = stats;
}
std::optional<storage::LabelPropertyIndexStats> LabelPropertyIndex::GetIndexStats(
const std::pair<storage::LabelId, storage::PropertyId> &key) const {
if (auto it = stats_.find(key); it != stats_.end()) {
return it->second;
}
return {};
}
void LabelPropertyIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
void RemoveObsoleteEntries(Indices *indices, uint64_t oldest_active_start_timestamp) {
indices->label_index.RemoveObsoleteEntries(oldest_active_start_timestamp);
indices->label_property_index.RemoveObsoleteEntries(oldest_active_start_timestamp);
}
void UpdateOnAddLabel(Indices *indices, LabelId label, Vertex *vertex, const Transaction &tx) {
indices->label_index.UpdateOnAddLabel(label, vertex, tx);
indices->label_property_index.UpdateOnAddLabel(label, vertex, tx);
}
void UpdateOnSetProperty(Indices *indices, PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) {
indices->label_property_index.UpdateOnSetProperty(property, value, vertex, tx);
}
} // namespace memgraph::storage

View File

@ -1,317 +0,0 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <optional>
#include <tuple>
#include <utility>
#include "storage/v2/config.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/bound.hpp"
#include "utils/logging.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage {
struct Indices;
struct Constraints;
using ParalellizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
struct LabelIndexStats {
uint64_t count;
double avg_degree;
};
class LabelIndex {
private:
struct Entry {
Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs) {
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool operator==(const Entry &rhs) { return vertex == rhs.vertex && timestamp == rhs.timestamp; }
};
struct LabelStorage {
LabelId label;
utils::SkipList<Entry> vertices;
bool operator<(const LabelStorage &rhs) { return label < rhs.label; }
bool operator<(LabelId rhs) { return label < rhs; }
bool operator==(const LabelStorage &rhs) { return label == rhs.label; }
bool operator==(LabelId rhs) { return label == rhs; }
};
public:
LabelIndex(Indices *indices, Constraints *constraints, Config::Items config)
: indices_(indices), constraints_(constraints), config_(config) {}
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx);
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info = std::nullopt);
/// Returns false if there was no index to drop
bool DropIndex(LabelId label) { return index_.erase(label) > 0; }
bool IndexExists(LabelId label) const { return index_.find(label) != index_.end(); }
std::vector<LabelId> ListIndices() const;
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
class Iterable {
public:
Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, View view, Transaction *transaction,
Indices *indices, Constraints *constraints, Config::Items config);
class Iterator {
public:
Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator);
VertexAccessor operator*() const { return current_vertex_accessor_; }
bool operator==(const Iterator &other) const { return index_iterator_ == other.index_iterator_; }
bool operator!=(const Iterator &other) const { return index_iterator_ != other.index_iterator_; }
Iterator &operator++();
private:
void AdvanceUntilValid();
Iterable *self_;
utils::SkipList<Entry>::Iterator index_iterator_;
VertexAccessor current_vertex_accessor_;
Vertex *current_vertex_;
};
Iterator begin() { return Iterator(this, index_accessor_.begin()); }
Iterator end() { return Iterator(this, index_accessor_.end()); }
private:
utils::SkipList<Entry>::Accessor index_accessor_;
LabelId label_;
View view_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
};
/// Returns an self with vertices visible from the given transaction.
Iterable Vertices(LabelId label, View view, Transaction *transaction) {
auto it = index_.find(label);
MG_ASSERT(it != index_.end(), "Index for label {} doesn't exist", label.AsUint());
return Iterable(it->second.access(), label, view, transaction, indices_, constraints_, config_);
}
int64_t ApproximateVertexCount(LabelId label) {
auto it = index_.find(label);
MG_ASSERT(it != index_.end(), "Index for label {} doesn't exist", label.AsUint());
return it->second.size();
}
void SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats);
std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const;
std::vector<LabelId> ClearIndexStats();
std::vector<LabelId> DeleteIndexStats(const storage::LabelId &label);
void Clear() { index_.clear(); }
void RunGC();
private:
std::map<LabelId, utils::SkipList<Entry>> index_;
std::map<LabelId, storage::LabelIndexStats> stats_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
};
struct LabelPropertyIndexStats {
uint64_t count, distinct_values_count;
double statistic, avg_group_size, avg_degree;
};
class LabelPropertyIndex {
private:
struct Entry {
PropertyValue value;
Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs);
bool operator==(const Entry &rhs);
bool operator<(const PropertyValue &rhs);
bool operator==(const PropertyValue &rhs);
};
public:
LabelPropertyIndex(Indices *indices, Constraints *constraints, Config::Items config)
: indices_(indices), constraints_(constraints), config_(config) {}
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx);
/// @throw std::bad_alloc
void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex, const Transaction &tx);
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info = std::nullopt);
bool DropIndex(LabelId label, PropertyId property) { return index_.erase({label, property}) > 0; }
bool IndexExists(LabelId label, PropertyId property) const { return index_.find({label, property}) != index_.end(); }
std::vector<std::pair<LabelId, PropertyId>> ListIndices() const;
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
class Iterable {
public:
Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction,
Indices *indices, Constraints *constraints, Config::Items config);
class Iterator {
public:
Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator);
VertexAccessor operator*() const { return current_vertex_accessor_; }
bool operator==(const Iterator &other) const { return index_iterator_ == other.index_iterator_; }
bool operator!=(const Iterator &other) const { return index_iterator_ != other.index_iterator_; }
Iterator &operator++();
private:
void AdvanceUntilValid();
Iterable *self_;
utils::SkipList<Entry>::Iterator index_iterator_;
VertexAccessor current_vertex_accessor_;
Vertex *current_vertex_;
};
Iterator begin();
Iterator end();
private:
utils::SkipList<Entry>::Accessor index_accessor_;
LabelId label_;
PropertyId property_;
std::optional<utils::Bound<PropertyValue>> lower_bound_;
std::optional<utils::Bound<PropertyValue>> upper_bound_;
bool bounds_valid_{true};
View view_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
};
Iterable Vertices(LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view,
Transaction *transaction) {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(),
property.AsUint());
return Iterable(it->second.access(), label, property, lower_bound, upper_bound, view, transaction, indices_,
constraints_, config_);
}
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(),
property.AsUint());
return it->second.size();
}
/// Supplying a specific value into the count estimation function will return
/// an estimated count of nodes which have their property's value set to
/// `value`. If the `value` specified is `Null`, then an average number of
/// equal elements is returned.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const;
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const;
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats();
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStats(const storage::LabelId &label);
void SetIndexStats(const std::pair<storage::LabelId, storage::PropertyId> &key,
const storage::LabelPropertyIndexStats &stats);
std::optional<storage::LabelPropertyIndexStats> GetIndexStats(
const std::pair<storage::LabelId, storage::PropertyId> &key) const;
void Clear() { index_.clear(); }
void RunGC();
private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
std::map<std::pair<LabelId, PropertyId>, storage::LabelPropertyIndexStats> stats_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
};
struct Indices {
Indices(Constraints *constraints, Config::Items config)
: label_index(this, constraints, config), label_property_index(this, constraints, config) {}
// Disable copy and move because members hold pointer to `this`.
Indices(const Indices &) = delete;
Indices(Indices &&) = delete;
Indices &operator=(const Indices &) = delete;
Indices &operator=(Indices &&) = delete;
~Indices() = default;
LabelIndex label_index;
LabelPropertyIndex label_property_index;
};
/// This function should be called from garbage collection to clean-up the
/// index.
void RemoveObsoleteEntries(Indices *indices, uint64_t oldest_active_start_timestamp);
// Indices are updated whenever an update occurs, instead of only on commit or
// advance command. This is necessary because we want indices to support `NEW`
// view for use in Merge.
/// This function should be called whenever a label is added to a vertex.
/// @throw std::bad_alloc
void UpdateOnAddLabel(Indices *indices, LabelId label, Vertex *vertex, const Transaction &tx);
/// This function should be called whenever a property is modified on a vertex.
/// @throw std::bad_alloc
void UpdateOnSetProperty(Indices *indices, PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx);
} // namespace memgraph::storage

View File

@ -0,0 +1,38 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/inmemory/label_index.hpp"
namespace memgraph::storage {
void Indices::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) const {
static_cast<InMemoryLabelIndex *>(label_index_.get())->RemoveObsoleteEntries(oldest_active_start_timestamp);
static_cast<InMemoryLabelPropertyIndex *>(label_property_index_.get())
->RemoveObsoleteEntries(oldest_active_start_timestamp);
}
void Indices::UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) const {
label_index_->UpdateOnAddLabel(label, vertex, tx);
label_property_index_->UpdateOnAddLabel(label, vertex, tx);
}
void Indices::UpdateOnRemoveLabel(LabelId label, Vertex *vertex, const Transaction &tx) const {
label_index_->UpdateOnRemoveLabel(label, vertex, tx);
label_property_index_->UpdateOnRemoveLabel(label, vertex, tx);
}
void Indices::UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) const {
label_property_index_->UpdateOnSetProperty(property, value, vertex, tx);
}
} // namespace memgraph::storage

View File

@ -0,0 +1,68 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <memory>
#include "storage/v2/disk/label_index.hpp"
#include "storage/v2/disk/label_property_index.hpp"
#include "storage/v2/indices/label_index.hpp"
#include "storage/v2/indices/label_property_index.hpp"
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/storage_mode.hpp"
namespace memgraph::storage {
struct Indices {
Indices(Constraints *constraints, const Config &config, StorageMode storage_mode) {
std::invoke([this, constraints, config, storage_mode]() {
if (storage_mode == StorageMode::IN_MEMORY_TRANSACTIONAL || storage_mode == StorageMode::IN_MEMORY_ANALYTICAL) {
label_index_ = std::make_unique<InMemoryLabelIndex>(this, constraints, config);
label_property_index_ = std::make_unique<InMemoryLabelPropertyIndex>(this, constraints, config);
} else {
label_index_ = std::make_unique<DiskLabelIndex>(this, constraints, config);
label_property_index_ = std::make_unique<DiskLabelPropertyIndex>(this, constraints, config);
}
});
}
Indices(const Indices &) = delete;
Indices(Indices &&) = delete;
Indices &operator=(const Indices &) = delete;
Indices &operator=(Indices &&) = delete;
~Indices() = default;
/// This function should be called from garbage collection to clean-up the
/// index.
/// TODO: unused in disk indices
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) const;
// Indices are updated whenever an update occurs, instead of only on commit or
// advance command. This is necessary because we want indices to support `NEW`
// view for use in Merge.
/// This function should be called whenever a label is added to a vertex.
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId label, Vertex *vertex, const Transaction &tx) const;
void UpdateOnRemoveLabel(LabelId label, Vertex *vertex, const Transaction &tx) const;
/// This function should be called whenever a property is modified on a vertex.
/// @throw std::bad_alloc
void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) const;
std::unique_ptr<LabelIndex> label_index_;
std::unique_ptr<LabelPropertyIndex> label_property_index_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,51 @@
// 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 "storage/v2/constraints/constraints.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
namespace memgraph::storage {
class LabelIndex {
public:
LabelIndex(Indices *indices, Constraints *constraints, const Config &config)
: indices_(indices), constraints_(constraints), config_(config) {}
LabelIndex(const LabelIndex &) = delete;
LabelIndex(LabelIndex &&) = delete;
LabelIndex &operator=(const LabelIndex &) = delete;
LabelIndex &operator=(LabelIndex &&) = delete;
virtual ~LabelIndex() = default;
virtual void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) = 0;
virtual void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_after_update, const Transaction &tx) = 0;
virtual bool DropIndex(LabelId label) = 0;
virtual bool IndexExists(LabelId label) const = 0;
virtual std::vector<LabelId> ListIndices() const = 0;
virtual uint64_t ApproximateVertexCount(LabelId label) const = 0;
protected:
/// TODO: andi maybe no need for have those in abstract class if disk storage isn't using it
Indices *indices_;
Constraints *constraints_;
Config config_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,59 @@
// 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 "storage/v2/constraints/constraints.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
namespace memgraph::storage {
class LabelPropertyIndex {
public:
LabelPropertyIndex(Indices *indices, Constraints *constraints, const Config &config)
: indices_(indices), constraints_(constraints), config_(config) {}
LabelPropertyIndex(const LabelPropertyIndex &) = delete;
LabelPropertyIndex(LabelPropertyIndex &&) = delete;
LabelPropertyIndex &operator=(const LabelPropertyIndex &) = delete;
LabelPropertyIndex &operator=(LabelPropertyIndex &&) = delete;
virtual ~LabelPropertyIndex() = default;
virtual void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) = 0;
virtual void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_after_update, const Transaction &tx) = 0;
virtual void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) = 0;
virtual bool DropIndex(LabelId label, PropertyId property) = 0;
virtual bool IndexExists(LabelId label, PropertyId property) const = 0;
virtual std::vector<std::pair<LabelId, PropertyId>> ListIndices() const = 0;
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property) const = 0;
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const = 0;
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
protected:
Indices *indices_;
Constraints *constraints_;
Config config_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,357 @@
// 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 <thread>
#include "storage/v2/delta.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/spin_lock.hpp"
#include "utils/synchronized.hpp"
namespace memgraph::storage {
using ParalellizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
/// Traverses deltas visible from transaction with start timestamp greater than
/// the provided timestamp, and calls the provided callback function for each
/// delta. If the callback ever returns true, traversal is stopped and the
/// function returns true. Otherwise, the function returns false.
template <typename TCallback>
inline bool AnyVersionSatisfiesPredicate(uint64_t timestamp, const Delta *delta, const TCallback &predicate) {
while (delta != nullptr) {
const auto ts = delta->timestamp->load(std::memory_order_acquire);
// This is a committed change that we see so we shouldn't undo it.
if (ts < timestamp) {
break;
}
if (predicate(*delta)) {
return true;
}
// Move to the next delta.
delta = delta->next.load(std::memory_order_acquire);
}
return false;
}
/// Helper function for label index garbage collection. Returns true if there's
/// a reachable version of the vertex that has the given label.
inline bool AnyVersionHasLabel(const Vertex &vertex, LabelId label, uint64_t timestamp) {
bool has_label{false};
bool deleted{false};
const Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
has_label = utils::Contains(vertex.labels, label);
deleted = vertex.deleted;
delta = vertex.delta;
}
if (!deleted && has_label) {
return true;
}
return AnyVersionSatisfiesPredicate(timestamp, delta, [&has_label, &deleted, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
return !deleted && has_label;
});
}
/// Helper function for label-property index garbage collection. Returns true if
/// there's a reachable version of the vertex that has the given label and
/// property value.
inline bool AnyVersionHasLabelProperty(const Vertex &vertex, LabelId label, PropertyId key, const PropertyValue &value,
uint64_t timestamp) {
bool has_label{false};
bool current_value_equal_to_value{value.IsNull()};
bool deleted{false};
const Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
has_label = utils::Contains(vertex.labels, label);
current_value_equal_to_value = vertex.properties.IsPropertyEqual(key, value);
deleted = vertex.deleted;
delta = vertex.delta;
}
if (!deleted && has_label && current_value_equal_to_value) {
return true;
}
return AnyVersionSatisfiesPredicate(
timestamp, delta, [&has_label, &current_value_equal_to_value, &deleted, label, key, &value](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::SET_PROPERTY:
if (delta.property.key == key) {
current_value_equal_to_value = delta.property.value == value;
}
break;
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
return !deleted && has_label && current_value_equal_to_value;
});
}
// Helper function for iterating through label index. Returns true if this
// transaction can see the given vertex, and the visible version has the given
// label.
inline bool CurrentVersionHasLabel(const Vertex &vertex, LabelId label, Transaction *transaction, View view) {
bool deleted = false;
bool has_label = false;
const Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
deleted = vertex.deleted;
has_label = utils::Contains(vertex.labels, label);
delta = vertex.delta;
}
ApplyDeltasForRead(transaction, delta, view, [&deleted, &has_label, label](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
}
case Delta::Action::ADD_LABEL: {
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
return !deleted && has_label;
}
// Helper function for iterating through label-property index. Returns true if
// this transaction can see the given vertex, and the visible version has the
// given label and property.
inline bool CurrentVersionHasLabelProperty(const Vertex &vertex, LabelId label, PropertyId key,
const PropertyValue &value, Transaction *transaction, View view) {
bool deleted = false;
bool has_label = false;
bool current_value_equal_to_value = value.IsNull();
const Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
deleted = vertex.deleted;
has_label = utils::Contains(vertex.labels, label);
current_value_equal_to_value = vertex.properties.IsPropertyEqual(key, value);
delta = vertex.delta;
}
ApplyDeltasForRead(transaction, delta, view,
[&deleted, &has_label, &current_value_equal_to_value, key, label, &value](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == key) {
current_value_equal_to_value = delta.property.value == value;
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
break;
}
case Delta::Action::RECREATE_OBJECT: {
MG_ASSERT(deleted, "Invalid database state!");
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
if (delta.label == label) {
MG_ASSERT(!has_label, "Invalid database state!");
has_label = true;
}
break;
case Delta::Action::REMOVE_LABEL:
if (delta.label == label) {
MG_ASSERT(has_label, "Invalid database state!");
has_label = false;
}
break;
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
return !deleted && has_label && current_value_equal_to_value;
}
template <typename TIndexAccessor>
inline void TryInsertLabelIndex(Vertex &vertex, LabelId label, TIndexAccessor &index_accessor) {
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
return;
}
index_accessor.insert({&vertex, 0});
}
template <typename TIndexAccessor>
inline void TryInsertLabelPropertyIndex(Vertex &vertex, std::pair<LabelId, PropertyId> label_property_pair,
TIndexAccessor &index_accessor) {
if (vertex.deleted || !utils::Contains(vertex.labels, label_property_pair.first)) {
return;
}
auto value = vertex.properties.GetProperty(label_property_pair.second);
if (value.IsNull()) {
return;
}
index_accessor.insert({std::move(value), &vertex, 0});
}
template <typename TSkiplistIter, typename TIndex, typename TIndexKey, typename TFunc>
inline void CreateIndexOnSingleThread(utils::SkipList<Vertex>::Accessor &vertices, TSkiplistIter it, TIndex &index,
TIndexKey key, const TFunc &func) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
try {
auto acc = it->second.access();
for (Vertex &vertex : vertices) {
func(vertex, key, acc);
}
} catch (const utils::OutOfMemoryException &) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index.erase(it);
throw;
}
}
template <typename TIndex, typename TIndexKey, typename TSKiplistIter, typename TFunc>
inline void CreateIndexOnMultipleThreads(utils::SkipList<Vertex>::Accessor &vertices, TSKiplistIter skiplist_iter,
TIndex &index, TIndexKey key,
const ParalellizedIndexCreationInfo &paralell_exec_info, const TFunc &func) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
const auto &vertex_batches = paralell_exec_info.first;
const auto thread_count = std::min(paralell_exec_info.second, vertex_batches.size());
MG_ASSERT(!vertex_batches.empty(),
"The size of batches should always be greater than zero if you want to use the parallel version of index "
"creation!");
std::atomic<uint64_t> batch_counter = 0;
utils::Synchronized<std::optional<utils::OutOfMemoryException>, utils::SpinLock> maybe_error{};
{
std::vector<std::jthread> threads;
threads.reserve(thread_count);
for (auto i{0U}; i < thread_count; ++i) {
threads.emplace_back(
[&skiplist_iter, &func, &index, &vertex_batches, &maybe_error, &batch_counter, &key, &vertices]() {
while (!maybe_error.Lock()->has_value()) {
const auto batch_index = batch_counter++;
if (batch_index >= vertex_batches.size()) {
return;
}
const auto &batch = vertex_batches[batch_index];
auto index_accessor = index.at(key).access();
auto it = vertices.find(batch.first);
try {
for (auto i{0U}; i < batch.second; ++i, ++it) {
func(*it, key, index_accessor);
}
} catch (utils::OutOfMemoryException &failure) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index.erase(skiplist_iter);
*maybe_error.Lock() = std::move(failure);
}
}
});
}
}
if (maybe_error.Lock()->has_value()) {
throw utils::OutOfMemoryException((*maybe_error.Lock())->what());
}
}
} // namespace memgraph::storage

View File

@ -0,0 +1,192 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/indices_utils.hpp"
namespace memgraph::storage {
InMemoryLabelIndex::InMemoryLabelIndex(Indices *indices, Constraints *constraints, Config config)
: LabelIndex(indices, constraints, config) {}
void InMemoryLabelIndex::UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) {
auto it = index_.find(added_label);
if (it == index_.end()) return;
auto acc = it->second.access();
acc.insert(Entry{vertex_after_update, tx.start_timestamp});
}
bool InMemoryLabelIndex::CreateIndex(LabelId label, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info) {
const auto create_index_seq = [this](LabelId label, utils::SkipList<Vertex>::Accessor &vertices,
std::map<LabelId, utils::SkipList<Entry>>::iterator it) {
using IndexAccessor = decltype(it->second.access());
CreateIndexOnSingleThread(vertices, it, index_, label,
[](Vertex &vertex, LabelId label, IndexAccessor &index_accessor) {
TryInsertLabelIndex(vertex, label, index_accessor);
});
return true;
};
const auto create_index_par = [this](LabelId label, utils::SkipList<Vertex>::Accessor &vertices,
std::map<LabelId, utils::SkipList<Entry>>::iterator label_it,
const ParalellizedIndexCreationInfo &paralell_exec_info) {
using IndexAccessor = decltype(label_it->second.access());
CreateIndexOnMultipleThreads(vertices, label_it, index_, label, paralell_exec_info,
[](Vertex &vertex, LabelId label, IndexAccessor &index_accessor) {
TryInsertLabelIndex(vertex, label, index_accessor);
});
return true;
};
auto [it, emplaced] = index_.emplace(std::piecewise_construct, std::forward_as_tuple(label), std::forward_as_tuple());
if (!emplaced) {
// Index already exists.
return false;
}
if (paralell_exec_info) {
return create_index_par(label, vertices, it, *paralell_exec_info);
}
return create_index_seq(label, vertices, it);
}
bool InMemoryLabelIndex::DropIndex(LabelId label) { return index_.erase(label) > 0; }
bool InMemoryLabelIndex::IndexExists(LabelId label) const { return index_.find(label) != index_.end(); }
std::vector<LabelId> InMemoryLabelIndex::ListIndices() const {
std::vector<LabelId> ret;
ret.reserve(index_.size());
for (const auto &item : index_) {
ret.push_back(item.first);
}
return ret;
}
void InMemoryLabelIndex::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
for (auto &label_storage : index_) {
auto vertices_acc = label_storage.second.access();
for (auto it = vertices_acc.begin(); it != vertices_acc.end();) {
auto next_it = it;
++next_it;
if (it->timestamp >= oldest_active_start_timestamp) {
it = next_it;
continue;
}
if ((next_it != vertices_acc.end() && it->vertex == next_it->vertex) ||
!AnyVersionHasLabel(*it->vertex, label_storage.first, oldest_active_start_timestamp)) {
vertices_acc.remove(*it);
}
it = next_it;
}
}
}
InMemoryLabelIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, View view,
Transaction *transaction, Indices *indices, Constraints *constraints,
const Config &config)
: index_accessor_(std::move(index_accessor)),
label_(label),
view_(view),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config) {}
InMemoryLabelIndex::Iterable::Iterator::Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator)
: self_(self),
index_iterator_(index_iterator),
current_vertex_accessor_(nullptr, nullptr, nullptr, nullptr, self_->config_.items),
current_vertex_(nullptr) {
AdvanceUntilValid();
}
InMemoryLabelIndex::Iterable::Iterator &InMemoryLabelIndex::Iterable::Iterator::operator++() {
++index_iterator_;
AdvanceUntilValid();
return *this;
}
void InMemoryLabelIndex::Iterable::Iterator::AdvanceUntilValid() {
for (; index_iterator_ != self_->index_accessor_.end(); ++index_iterator_) {
if (index_iterator_->vertex == current_vertex_) {
continue;
}
if (CurrentVersionHasLabel(*index_iterator_->vertex, self_->label_, self_->transaction_, self_->view_)) {
current_vertex_ = index_iterator_->vertex;
current_vertex_accessor_ = VertexAccessor{current_vertex_, self_->transaction_, self_->indices_,
self_->constraints_, self_->config_.items};
break;
}
}
}
uint64_t InMemoryLabelIndex::ApproximateVertexCount(LabelId label) const {
auto it = index_.find(label);
MG_ASSERT(it != index_.end(), "Index for label {} doesn't exist", label.AsUint());
return it->second.size();
}
void InMemoryLabelIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
InMemoryLabelIndex::Iterable InMemoryLabelIndex::Vertices(LabelId label, View view, Transaction *transaction) {
const auto it = index_.find(label);
MG_ASSERT(it != index_.end(), "Index for label {} doesn't exist", label.AsUint());
return {it->second.access(), label, view, transaction, indices_, constraints_, config_};
}
void InMemoryLabelIndex::SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats) {
stats_[label] = stats;
}
std::optional<LabelIndexStats> InMemoryLabelIndex::GetIndexStats(const storage::LabelId &label) const {
if (auto it = stats_.find(label); it != stats_.end()) {
return it->second;
}
return {};
}
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),
[](const auto &elem) { return elem.first; });
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();) {
if (it->first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
} else {
++it;
}
}
return deleted_indexes;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,117 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include "storage/v2/indices/label_index.hpp"
#include "storage/v2/vertex.hpp"
namespace memgraph::storage {
struct LabelIndexStats {
uint64_t count;
double avg_degree;
};
using ParalellizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
class InMemoryLabelIndex : public storage::LabelIndex {
private:
struct Entry {
Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs) {
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool operator==(const Entry &rhs) const { return vertex == rhs.vertex && timestamp == rhs.timestamp; }
};
public:
InMemoryLabelIndex(Indices *indices, Constraints *constraints, Config config);
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) override;
void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_before_update, const Transaction &tx) override {}
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info);
/// Returns false if there was no index to drop
bool DropIndex(LabelId label) override;
bool IndexExists(LabelId label) const override;
std::vector<LabelId> ListIndices() const override;
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
class Iterable {
public:
Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, View view, Transaction *transaction,
Indices *indices, Constraints *constraints, const Config &config);
class Iterator {
public:
Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator);
VertexAccessor operator*() const { return current_vertex_accessor_; }
bool operator==(const Iterator &other) const { return index_iterator_ == other.index_iterator_; }
bool operator!=(const Iterator &other) const { return index_iterator_ != other.index_iterator_; }
Iterator &operator++();
private:
void AdvanceUntilValid();
Iterable *self_;
utils::SkipList<Entry>::Iterator index_iterator_;
VertexAccessor current_vertex_accessor_;
Vertex *current_vertex_;
};
Iterator begin() { return {this, index_accessor_.begin()}; }
Iterator end() { return {this, index_accessor_.end()}; }
private:
utils::SkipList<Entry>::Accessor index_accessor_;
LabelId label_;
View view_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config config_;
};
uint64_t ApproximateVertexCount(LabelId label) const override;
void RunGC();
Iterable Vertices(LabelId label, View view, Transaction *transaction);
void SetIndexStats(const storage::LabelId &label, const storage::LabelIndexStats &stats);
std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const;
std::vector<LabelId> ClearIndexStats();
std::vector<LabelId> DeleteIndexStats(const storage::LabelId &label);
private:
std::map<LabelId, utils::SkipList<Entry>> index_;
std::map<LabelId, storage::LabelIndexStats> stats_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,424 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/inmemory/indices_utils.hpp"
namespace memgraph::storage {
bool InMemoryLabelPropertyIndex::Entry::operator<(const Entry &rhs) const {
if (value < rhs.value) {
return true;
}
if (rhs.value < value) {
return false;
}
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool InMemoryLabelPropertyIndex::Entry::operator==(const Entry &rhs) const {
return value == rhs.value && vertex == rhs.vertex && timestamp == rhs.timestamp;
}
bool InMemoryLabelPropertyIndex::Entry::operator<(const PropertyValue &rhs) const { return value < rhs; }
bool InMemoryLabelPropertyIndex::Entry::operator==(const PropertyValue &rhs) const { return value == rhs; }
InMemoryLabelPropertyIndex::InMemoryLabelPropertyIndex(Indices *indices, Constraints *constraints, const Config &config)
: LabelPropertyIndex(indices, constraints, config) {}
bool InMemoryLabelPropertyIndex::CreateIndex(LabelId label, PropertyId property,
utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info) {
auto create_index_seq = [this](LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor &vertices,
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>>::iterator it) {
using IndexAccessor = decltype(it->second.access());
CreateIndexOnSingleThread(vertices, it, index_, std::make_pair(label, property),
[](Vertex &vertex, std::pair<LabelId, PropertyId> key, IndexAccessor &index_accessor) {
TryInsertLabelPropertyIndex(vertex, key, index_accessor);
});
return true;
};
auto create_index_par =
[this](LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor &vertices,
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>>::iterator label_property_it,
const ParalellizedIndexCreationInfo &paralell_exec_info) {
using IndexAccessor = decltype(label_property_it->second.access());
CreateIndexOnMultipleThreads(
vertices, label_property_it, index_, std::make_pair(label, property), paralell_exec_info,
[](Vertex &vertex, std::pair<LabelId, PropertyId> key, IndexAccessor &index_accessor) {
TryInsertLabelPropertyIndex(vertex, key, index_accessor);
});
return true;
};
auto [it, emplaced] =
index_.emplace(std::piecewise_construct, std::forward_as_tuple(label, property), std::forward_as_tuple());
if (!emplaced) {
// Index already exists.
return false;
}
if (paralell_exec_info) {
return create_index_par(label, property, vertices, it, *paralell_exec_info);
}
return create_index_seq(label, property, vertices, it);
}
void InMemoryLabelPropertyIndex::UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update,
const Transaction &tx) {
for (auto &[label_prop, storage] : index_) {
if (label_prop.first != added_label) {
continue;
}
auto prop_value = vertex_after_update->properties.GetProperty(label_prop.second);
if (!prop_value.IsNull()) {
auto acc = storage.access();
acc.insert(Entry{std::move(prop_value), vertex_after_update, tx.start_timestamp});
}
}
}
void InMemoryLabelPropertyIndex::UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) {
if (value.IsNull()) {
return;
}
for (auto &[label_prop, storage] : index_) {
if (label_prop.second != property) {
continue;
}
if (utils::Contains(vertex->labels, label_prop.first)) {
auto acc = storage.access();
acc.insert(Entry{value, vertex, tx.start_timestamp});
}
}
}
bool InMemoryLabelPropertyIndex::DropIndex(LabelId label, PropertyId property) {
return index_.erase({label, property}) > 0;
}
bool InMemoryLabelPropertyIndex::IndexExists(LabelId label, PropertyId property) const {
return index_.find({label, property}) != index_.end();
}
std::vector<std::pair<LabelId, PropertyId>> InMemoryLabelPropertyIndex::ListIndices() const {
std::vector<std::pair<LabelId, PropertyId>> ret;
ret.reserve(index_.size());
for (const auto &item : index_) {
ret.push_back(item.first);
}
return ret;
}
void InMemoryLabelPropertyIndex::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
for (auto &[label_property, index] : index_) {
auto index_acc = index.access();
for (auto it = index_acc.begin(); it != index_acc.end();) {
auto next_it = it;
++next_it;
if (it->timestamp >= oldest_active_start_timestamp) {
it = next_it;
continue;
}
if ((next_it != index_acc.end() && it->vertex == next_it->vertex && it->value == next_it->value) ||
!AnyVersionHasLabelProperty(*it->vertex, label_property.first, label_property.second, it->value,
oldest_active_start_timestamp)) {
index_acc.remove(*it);
}
it = next_it;
}
}
}
InMemoryLabelPropertyIndex::Iterable::Iterator::Iterator(Iterable *self,
utils::SkipList<Entry>::Iterator index_iterator)
: self_(self),
index_iterator_(index_iterator),
current_vertex_accessor_(nullptr, nullptr, nullptr, nullptr, self_->config_.items),
current_vertex_(nullptr) {
AdvanceUntilValid();
}
InMemoryLabelPropertyIndex::Iterable::Iterator &InMemoryLabelPropertyIndex::Iterable::Iterator::operator++() {
++index_iterator_;
AdvanceUntilValid();
return *this;
}
void InMemoryLabelPropertyIndex::Iterable::Iterator::AdvanceUntilValid() {
for (; index_iterator_ != self_->index_accessor_.end(); ++index_iterator_) {
if (index_iterator_->vertex == current_vertex_) {
continue;
}
if (self_->lower_bound_) {
if (index_iterator_->value < self_->lower_bound_->value()) {
continue;
}
if (!self_->lower_bound_->IsInclusive() && index_iterator_->value == self_->lower_bound_->value()) {
continue;
}
}
if (self_->upper_bound_) {
if (self_->upper_bound_->value() < index_iterator_->value) {
index_iterator_ = self_->index_accessor_.end();
break;
}
if (!self_->upper_bound_->IsInclusive() && index_iterator_->value == self_->upper_bound_->value()) {
index_iterator_ = self_->index_accessor_.end();
break;
}
}
if (CurrentVersionHasLabelProperty(*index_iterator_->vertex, self_->label_, self_->property_,
index_iterator_->value, self_->transaction_, self_->view_)) {
current_vertex_ = index_iterator_->vertex;
current_vertex_accessor_ = VertexAccessor(current_vertex_, self_->transaction_, self_->indices_,
self_->constraints_, self_->config_.items);
break;
}
}
}
// These constants represent the smallest possible value of each type that is
// contained in a `PropertyValue`. Note that numbers (integers and doubles) are
// treated as the same "type" in `PropertyValue`.
const PropertyValue kSmallestBool = PropertyValue(false);
// NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
static_assert(-std::numeric_limits<double>::infinity() < std::numeric_limits<int64_t>::min());
const PropertyValue kSmallestNumber = PropertyValue(-std::numeric_limits<double>::infinity());
const PropertyValue kSmallestString = PropertyValue("");
const PropertyValue kSmallestList = PropertyValue(std::vector<PropertyValue>());
const PropertyValue kSmallestMap = PropertyValue(std::map<std::string, PropertyValue>());
const PropertyValue kSmallestTemporalData =
PropertyValue(TemporalData{static_cast<TemporalType>(0), std::numeric_limits<int64_t>::min()});
InMemoryLabelPropertyIndex::Iterable::Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label,
PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view,
Transaction *transaction, Indices *indices, Constraints *constraints,
const Config &config)
: index_accessor_(std::move(index_accessor)),
label_(label),
property_(property),
lower_bound_(lower_bound),
upper_bound_(upper_bound),
view_(view),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config) {
// We have to fix the bounds that the user provided to us. If the user
// provided only one bound we should make sure that only values of that type
// are returned by the iterator. We ensure this by supplying either an
// inclusive lower bound of the same type, or an exclusive upper bound of the
// following type. If neither bound is set we yield all items in the index.
// First we statically verify that our assumptions about the `PropertyValue`
// type ordering holds.
static_assert(PropertyValue::Type::Bool < PropertyValue::Type::Int);
static_assert(PropertyValue::Type::Int < PropertyValue::Type::Double);
static_assert(PropertyValue::Type::Double < PropertyValue::Type::String);
static_assert(PropertyValue::Type::String < PropertyValue::Type::List);
static_assert(PropertyValue::Type::List < PropertyValue::Type::Map);
// Remove any bounds that are set to `Null` because that isn't a valid value.
if (lower_bound_ && lower_bound_->value().IsNull()) {
lower_bound_ = std::nullopt;
}
if (upper_bound_ && upper_bound_->value().IsNull()) {
upper_bound_ = std::nullopt;
}
// Check whether the bounds are of comparable types if both are supplied.
if (lower_bound_ && upper_bound_ &&
!PropertyValue::AreComparableTypes(lower_bound_->value().type(), upper_bound_->value().type())) {
bounds_valid_ = false;
return;
}
// Set missing bounds.
if (lower_bound_ && !upper_bound_) {
// Here we need to supply an upper bound. The upper bound is set to an
// exclusive lower bound of the following type.
switch (lower_bound_->value().type()) {
case PropertyValue::Type::Null:
// This shouldn't happen because of the nullopt-ing above.
LOG_FATAL("Invalid database state!");
break;
case PropertyValue::Type::Bool:
upper_bound_ = utils::MakeBoundExclusive(kSmallestNumber);
break;
case PropertyValue::Type::Int:
case PropertyValue::Type::Double:
// Both integers and doubles are treated as the same type in
// `PropertyValue` and they are interleaved when sorted.
upper_bound_ = utils::MakeBoundExclusive(kSmallestString);
break;
case PropertyValue::Type::String:
upper_bound_ = utils::MakeBoundExclusive(kSmallestList);
break;
case PropertyValue::Type::List:
upper_bound_ = utils::MakeBoundExclusive(kSmallestMap);
break;
case PropertyValue::Type::Map:
upper_bound_ = utils::MakeBoundExclusive(kSmallestTemporalData);
break;
case PropertyValue::Type::TemporalData:
// This is the last type in the order so we leave the upper bound empty.
break;
}
}
if (upper_bound_ && !lower_bound_) {
// Here we need to supply a lower bound. The lower bound is set to an
// inclusive lower bound of the current type.
switch (upper_bound_->value().type()) {
case PropertyValue::Type::Null:
// This shouldn't happen because of the nullopt-ing above.
LOG_FATAL("Invalid database state!");
break;
case PropertyValue::Type::Bool:
lower_bound_ = utils::MakeBoundInclusive(kSmallestBool);
break;
case PropertyValue::Type::Int:
case PropertyValue::Type::Double:
// Both integers and doubles are treated as the same type in
// `PropertyValue` and they are interleaved when sorted.
lower_bound_ = utils::MakeBoundInclusive(kSmallestNumber);
break;
case PropertyValue::Type::String:
lower_bound_ = utils::MakeBoundInclusive(kSmallestString);
break;
case PropertyValue::Type::List:
lower_bound_ = utils::MakeBoundInclusive(kSmallestList);
break;
case PropertyValue::Type::Map:
lower_bound_ = utils::MakeBoundInclusive(kSmallestMap);
break;
case PropertyValue::Type::TemporalData:
lower_bound_ = utils::MakeBoundInclusive(kSmallestTemporalData);
break;
}
}
}
InMemoryLabelPropertyIndex::Iterable::Iterator InMemoryLabelPropertyIndex::Iterable::begin() {
// If the bounds are set and don't have comparable types we don't yield any
// items from the index.
if (!bounds_valid_) return {this, index_accessor_.end()};
auto index_iterator = index_accessor_.begin();
if (lower_bound_) {
index_iterator = index_accessor_.find_equal_or_greater(lower_bound_->value());
}
return {this, index_iterator};
}
InMemoryLabelPropertyIndex::Iterable::Iterator InMemoryLabelPropertyIndex::Iterable::end() {
return {this, index_accessor_.end()};
}
uint64_t InMemoryLabelPropertyIndex::ApproximateVertexCount(LabelId label, PropertyId property) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
return it->second.size();
}
uint64_t InMemoryLabelPropertyIndex::ApproximateVertexCount(LabelId label, PropertyId property,
const PropertyValue &value) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
auto acc = it->second.access();
if (!value.IsNull()) {
// NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
return acc.estimate_count(value, utils::SkipListLayerForCountEstimation(acc.size()));
}
// The value `Null` won't ever appear in the index because it indicates that
// the property shouldn't exist. Instead, this value is used as an indicator
// to estimate the average number of equal elements in the list (for any
// given value).
return acc.estimate_average_number_of_equals(
[](const auto &first, const auto &second) { return first.value == second.value; },
// NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
utils::SkipListLayerForAverageEqualsEstimation(acc.size()));
}
uint64_t InMemoryLabelPropertyIndex::ApproximateVertexCount(
LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
auto acc = it->second.access();
// NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
return acc.estimate_range_count(lower, upper, utils::SkipListLayerForCountEstimation(acc.size()));
}
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),
[](const auto &elem) { return elem.first; });
stats_.clear();
return deleted_indexes;
}
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();) {
if (it->first.first == label) {
deleted_indexes.push_back(it->first);
it = stats_.erase(it);
} else {
++it;
}
}
return deleted_indexes;
}
void InMemoryLabelPropertyIndex::SetIndexStats(const std::pair<storage::LabelId, storage::PropertyId> &key,
const LabelPropertyIndexStats &stats) {
stats_[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()) {
return it->second;
}
return {};
}
void InMemoryLabelPropertyIndex::RunGC() {
for (auto &index_entry : index_) {
index_entry.second.run_gc();
}
}
InMemoryLabelPropertyIndex::Iterable InMemoryLabelPropertyIndex::Vertices(
LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction) {
auto it = index_.find({label, property});
MG_ASSERT(it != index_.end(), "Index for label {} and property {} doesn't exist", label.AsUint(), property.AsUint());
return {it->second.access(), label, property, lower_bound, upper_bound, view,
transaction, indices_, constraints_, config_};
}
} // namespace memgraph::storage

View File

@ -0,0 +1,141 @@
// 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 "storage/v2/indices/label_property_index.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 ParalellizedIndexCreationInfo =
std::pair<std::vector<std::pair<Gid, uint64_t>> /*vertex_recovery_info*/, uint64_t /*thread_count*/>;
class InMemoryLabelPropertyIndex : public storage::LabelPropertyIndex {
private:
struct Entry {
PropertyValue value;
Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs) const;
bool operator==(const Entry &rhs) const;
bool operator<(const PropertyValue &rhs) const;
bool operator==(const PropertyValue &rhs) const;
};
public:
InMemoryLabelPropertyIndex(Indices *indices, Constraints *constraints, const Config &config);
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, PropertyId property, utils::SkipList<Vertex>::Accessor vertices,
const std::optional<ParalellizedIndexCreationInfo> &paralell_exec_info);
/// @throw std::bad_alloc
void UpdateOnAddLabel(LabelId added_label, Vertex *vertex_after_update, const Transaction &tx) override;
void UpdateOnRemoveLabel(LabelId removed_label, Vertex *vertex_before_update, const Transaction &tx) override {}
/// @throw std::bad_alloc
void UpdateOnSetProperty(PropertyId property, const PropertyValue &value, Vertex *vertex,
const Transaction &tx) override;
bool DropIndex(LabelId label, PropertyId property) override;
bool IndexExists(LabelId label, PropertyId property) const override;
std::vector<std::pair<LabelId, PropertyId>> ListIndices() const override;
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
class Iterable {
public:
Iterable(utils::SkipList<Entry>::Accessor index_accessor, LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction,
Indices *indices, Constraints *constraints, const Config &config);
class Iterator {
public:
Iterator(Iterable *self, utils::SkipList<Entry>::Iterator index_iterator);
VertexAccessor operator*() const { return current_vertex_accessor_; }
bool operator==(const Iterator &other) const { return index_iterator_ == other.index_iterator_; }
bool operator!=(const Iterator &other) const { return index_iterator_ != other.index_iterator_; }
Iterator &operator++();
private:
void AdvanceUntilValid();
Iterable *self_;
utils::SkipList<Entry>::Iterator index_iterator_;
VertexAccessor current_vertex_accessor_;
Vertex *current_vertex_;
};
Iterator begin();
Iterator end();
private:
utils::SkipList<Entry>::Accessor index_accessor_;
LabelId label_;
PropertyId property_;
std::optional<utils::Bound<PropertyValue>> lower_bound_;
std::optional<utils::Bound<PropertyValue>> upper_bound_;
bool bounds_valid_{true};
View view_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config config_;
};
uint64_t ApproximateVertexCount(LabelId label, PropertyId property) const override;
/// Supplying a specific value into the count estimation function will return
/// an estimated count of nodes which have their property's value set to
/// `value`. If the `value` specified is `Null`, then an average number of
/// equal elements is returned.
uint64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const override;
uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const override;
std::vector<std::pair<LabelId, PropertyId>> ClearIndexStats();
std::vector<std::pair<LabelId, PropertyId>> DeleteIndexStats(const storage::LabelId &label);
void SetIndexStats(const std::pair<storage::LabelId, storage::PropertyId> &key,
const storage::LabelPropertyIndexStats &stats);
std::optional<storage::LabelPropertyIndexStats> GetIndexStats(
const std::pair<storage::LabelId, storage::PropertyId> &key) const;
void RunGC();
Iterable Vertices(LabelId label, PropertyId property, const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view, Transaction *transaction);
private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
std::map<std::pair<LabelId, PropertyId>, storage::LabelPropertyIndexStats> stats_;
};
} // namespace memgraph::storage

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,522 @@
// 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 "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/storage.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/replication_persistence_helper.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
namespace memgraph::storage {
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
// The paper implements a fully serializable storage, in our implementation we
// only implement snapshot isolation for transactions.
class InMemoryStorage final : public Storage {
public:
enum class RegisterReplicaError : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
CONNECTION_FAILED,
COULD_NOT_BE_PERSISTED
};
struct TimestampInfo {
uint64_t current_timestamp_of_replica;
uint64_t current_number_of_timestamp_behind_master;
};
struct ReplicaInfo {
std::string name;
replication::ReplicationMode mode;
io::network::Endpoint endpoint;
replication::ReplicaState state;
TimestampInfo timestamp_info;
};
enum class CreateSnapshotError : uint8_t {
DisabledForReplica,
DisabledForAnalyticsPeriodicCommit,
ReachedMaxNumTries
};
/// @throw std::system_error
/// @throw std::bad_alloc
explicit InMemoryStorage(Config config = Config());
InMemoryStorage(const InMemoryStorage &) = delete;
InMemoryStorage(InMemoryStorage &&) = delete;
InMemoryStorage &operator=(const InMemoryStorage &) = delete;
InMemoryStorage &operator=(InMemoryStorage &&) = delete;
~InMemoryStorage() override;
class InMemoryAccessor final : public Storage::Accessor {
private:
friend class InMemoryStorage;
explicit InMemoryAccessor(InMemoryStorage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
public:
InMemoryAccessor(const InMemoryAccessor &) = delete;
InMemoryAccessor &operator=(const InMemoryAccessor &) = delete;
InMemoryAccessor &operator=(InMemoryAccessor &&other) = delete;
// NOTE: After the accessor is moved, all objects derived from it (accessors
// and iterators) are *invalid*. You have to get all derived objects again.
InMemoryAccessor(InMemoryAccessor &&other) noexcept;
~InMemoryAccessor() override;
/// @throw std::bad_alloc
VertexAccessor CreateVertex() override;
std::optional<VertexAccessor> FindVertex(Gid gid, View view) override;
VerticesIterable Vertices(View view) override {
auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return VerticesIterable(AllVerticesIterable(mem_storage->vertices_.access(), &transaction_, view,
&mem_storage->indices_, &mem_storage->constraints_,
mem_storage->config_.items));
}
VerticesIterable Vertices(LabelId label, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) override;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) override;
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
uint64_t ApproximateVertexCount() const override {
auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return mem_storage->vertices_.size();
}
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
uint64_t ApproximateVertexCount(LabelId label) const override {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_index_->ApproximateVertexCount(label);
}
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
uint64_t ApproximateVertexCount(LabelId label, PropertyId property) const override {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_property_index_->ApproximateVertexCount(label,
property);
}
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
uint64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const override {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_property_index_->ApproximateVertexCount(
label, property, value);
}
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const override {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_property_index_->ApproximateVertexCount(
label, property, lower, upper);
}
template <typename TResult, typename TIndex, typename TIndexKey>
std::optional<TResult> GetIndexStatsForIndex(TIndex *index, TIndexKey &&key) const {
return index->GetIndexStats(key);
}
std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const override {
return GetIndexStatsForIndex<storage::LabelIndexStats>(
static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()), label);
}
std::optional<storage::LabelPropertyIndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const override {
return GetIndexStatsForIndex<storage::LabelPropertyIndexStats>(
static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()),
std::make_pair(label, property));
}
template <typename TIndex, typename TIndexKey, typename TIndexStats>
void SetIndexStatsForIndex(TIndex *index, TIndexKey &&key, TIndexStats &stats) const {
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 storage::PropertyId &property,
const LabelPropertyIndexStats &stats) override {
SetIndexStatsForIndex(static_cast<InMemoryLabelPropertyIndex *>(storage_->indices_.label_property_index_.get()),
std::make_pair(label, property), stats);
}
template <typename TResult, typename TIndex>
std::vector<TResult> ClearIndexStatsForIndex(TIndex *index) const {
return index->ClearIndexStats();
}
std::vector<LabelId> ClearLabelIndexStats() override {
return ClearIndexStatsForIndex<LabelId>(static_cast<InMemoryLabelIndex *>(storage_->indices_.label_index_.get()));
}
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);
}
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) override;
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex) override;
void PrefetchInEdges(const VertexAccessor &vertex_acc) override{};
void PrefetchOutEdges(const VertexAccessor &vertex_acc) override{};
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) override;
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) override;
bool LabelIndexExists(LabelId label) const override {
return static_cast<InMemoryStorage *>(storage_)->indices_.label_index_->IndexExists(label);
}
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const override {
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();
}
ConstraintsInfo ListAllConstraints() const override {
const auto *mem_storage = static_cast<InMemoryStorage *>(storage_);
return mem_storage->ListAllConstraints();
}
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
// NOLINTNEXTLINE(google-default-arguments)
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) override;
/// @throw std::bad_alloc
void Abort() override;
void FinalizeTransaction() override;
private:
/// @throw std::bad_alloc
VertexAccessor CreateVertex(storage::Gid gid);
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, storage::Gid gid);
Config::Items config_;
};
std::unique_ptr<Storage::Accessor> Access(std::optional<IsolationLevel> override_isolation_level) override {
return std::unique_ptr<InMemoryAccessor>(
new InMemoryAccessor{this, override_isolation_level.value_or(isolation_level_), storage_mode_});
}
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `IndexDefinitionError`: the index already exists.
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Create an index.
/// Returns void if the index has been created.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index already exists.
/// @throw std::bad_alloc
utils::BasicResult<StorageIndexDefinitionError, void> CreateIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, std::optional<uint64_t> desired_commit_timestamp) override;
/// Drop an existing index.
/// Returns void if the index has been dropped.
/// Returns `StorageIndexDefinitionError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `IndexDefinitionError`: the index does not exist.
utils::BasicResult<StorageIndexDefinitionError, void> DropIndex(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) override;
/// 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;
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config);
bool SetMainReplicationRole();
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode,
const replication::ReplicationClientConfig &config);
/// @pre The instance should have a MAIN role
bool UnregisterReplica(const std::string &name);
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
replication::ReplicationRole GetReplicationRole() const;
std::vector<ReplicaInfo> ReplicasInfo();
void FreeMemory(std::unique_lock<utils::RWLock> main_guard) override;
utils::FileRetainer::FileLockerAccessor::ret_type IsPathLocked();
utils::FileRetainer::FileLockerAccessor::ret_type LockPath();
utils::FileRetainer::FileLockerAccessor::ret_type UnlockPath();
utils::BasicResult<CreateSnapshotError> CreateSnapshot(std::optional<bool> is_periodic);
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode) override;
private:
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage(std::unique_lock<utils::RWLock> 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);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
void RestoreReplicas();
void RestoreReplicationRole();
bool ShouldStoreAndRestoreReplicationState() const;
// Main object storage
utils::SkipList<storage::Vertex> vertices_;
utils::SkipList<storage::Edge> edges_;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
std::unique_ptr<kvstore::KVStore> storage_;
std::filesystem::path wal_directory_;
utils::Scheduler snapshot_runner_;
utils::SpinLock snapshot_lock_;
// UUID used to distinguish snapshots and to link snapshots to WALs
std::string uuid_;
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_;
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
utils::FileRetainer file_retainer_;
// Global locker that is used for clients file locking
utils::FileRetainer::FileLocker global_locker_;
// TODO: This isn't really a commit log, it doesn't even care if a
// transaction commited or aborted. We could probably combine this with
// `timestamp_` in a sensible unit, something like TransactionClock or
// whatever.
std::optional<CommitLog> commit_log_;
utils::Synchronized<std::list<Transaction>, utils::SpinLock> committed_transactions_;
utils::Scheduler gc_runner_;
std::mutex gc_lock_;
// Undo buffers that were unlinked and now are waiting to be freed.
utils::Synchronized<std::list<std::pair<uint64_t, std::list<Delta>>>, utils::SpinLock> garbage_undo_buffers_;
// Vertices that are logically deleted but still have to be removed from
// indices before removing them from the main storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_vertices_;
// Vertices that are logically deleted and removed from indices and now wait
// to be removed from the main storage.
std::list<std::pair<uint64_t, Gid>> garbage_vertices_;
// Edges that are logically deleted and wait to be removed from the main
// storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_edges_;
// Flags to inform CollectGarbage that it needs to do the more expensive full scans
std::atomic<bool> gc_full_scan_vertices_delete_ = false;
std::atomic<bool> gc_full_scan_edges_delete_ = false;
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
class ReplicationServer;
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
class ReplicationClient;
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<replication::ReplicationRole> replication_role_{replication::ReplicationRole::MAIN};
};
} // namespace memgraph::storage

View File

@ -9,24 +9,17 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/constraints.hpp"
#include <algorithm>
#include <atomic>
#include <cstring>
#include <map>
#include "storage/v2/mvcc.hpp"
#include "utils/logging.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
namespace memgraph::storage {
namespace {
/// Helper function that determines position of the given `property` in the
/// sorted `property_array` using binary search. In the case that `property`
/// cannot be found, `std::nullopt` is returned.
std::optional<size_t> FindPropertyPosition(const PropertyIdArray &property_array, PropertyId property) {
auto it = std::lower_bound(property_array.values, property_array.values + property_array.size, property);
const auto *it = std::lower_bound(property_array.values, property_array.values + property_array.size, property);
if (it == property_array.values + property_array.size || *it != property) {
return std::nullopt;
}
@ -84,6 +77,7 @@ bool LastCommittedVersionHasLabelProperty(const Vertex &vertex, LabelId label, c
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
@ -198,6 +192,7 @@ bool AnyVersionHasLabelProperty(const Vertex &vertex, LabelId label, const std::
deleted = false;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
MG_ASSERT(!deleted, "Invalid database state!");
deleted = true;
@ -225,30 +220,9 @@ bool AnyVersionHasLabelProperty(const Vertex &vertex, LabelId label, const std::
return false;
}
/// Helper function that, given the set of `properties`, extracts corresponding
/// property values from the `vertex`.
/// @throw std::bad_alloc
std::optional<std::vector<PropertyValue>> ExtractPropertyValues(const Vertex &vertex,
const std::set<PropertyId> &properties) {
std::vector<PropertyValue> value_array;
value_array.reserve(properties.size());
for (const auto &prop : properties) {
auto value = vertex.properties.GetProperty(prop);
if (value.IsNull()) {
return std::nullopt;
}
value_array.emplace_back(std::move(value));
}
return std::move(value_array);
}
} // namespace
bool operator==(const ConstraintViolation &lhs, const ConstraintViolation &rhs) {
return lhs.type == rhs.type && lhs.label == rhs.label && lhs.properties == rhs.properties;
}
bool UniqueConstraints::Entry::operator<(const Entry &rhs) {
bool InMemoryUniqueConstraints::Entry::operator<(const Entry &rhs) const {
if (values < rhs.values) {
return true;
}
@ -258,20 +232,20 @@ bool UniqueConstraints::Entry::operator<(const Entry &rhs) {
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool UniqueConstraints::Entry::operator==(const Entry &rhs) {
bool InMemoryUniqueConstraints::Entry::operator==(const Entry &rhs) const {
return values == rhs.values && vertex == rhs.vertex && timestamp == rhs.timestamp;
}
bool UniqueConstraints::Entry::operator<(const std::vector<PropertyValue> &rhs) { return values < rhs; }
bool InMemoryUniqueConstraints::Entry::operator<(const std::vector<PropertyValue> &rhs) const { return values < rhs; }
bool UniqueConstraints::Entry::operator==(const std::vector<PropertyValue> &rhs) { return values == rhs; }
bool InMemoryUniqueConstraints::Entry::operator==(const std::vector<PropertyValue> &rhs) const { return values == rhs; }
void UniqueConstraints::UpdateBeforeCommit(const Vertex *vertex, const Transaction &tx) {
void InMemoryUniqueConstraints::UpdateBeforeCommit(const Vertex *vertex, const Transaction &tx) {
for (auto &[label_props, storage] : constraints_) {
if (!utils::Contains(vertex->labels, label_props.first)) {
continue;
}
auto values = ExtractPropertyValues(*vertex, label_props.second);
auto values = vertex->properties.ExtractPropertyValues(label_props.second);
if (values) {
auto acc = storage.access();
acc.insert(Entry{std::move(*values), vertex, tx.start_timestamp});
@ -279,8 +253,9 @@ void UniqueConstraints::UpdateBeforeCommit(const Vertex *vertex, const Transacti
}
}
utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> UniqueConstraints::CreateConstraint(
LabelId label, const std::set<PropertyId> &properties, utils::SkipList<Vertex>::Accessor vertices) {
utils::BasicResult<ConstraintViolation, InMemoryUniqueConstraints::CreationStatus>
InMemoryUniqueConstraints::CreateConstraint(LabelId label, const std::set<PropertyId> &properties,
utils::SkipList<Vertex>::Accessor vertices) {
if (properties.empty()) {
return CreationStatus::EMPTY_PROPERTIES;
}
@ -305,7 +280,7 @@ utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Uniqu
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
continue;
}
auto values = ExtractPropertyValues(vertex, properties);
auto values = vertex.properties.ExtractPropertyValues(properties);
if (!values) {
continue;
}
@ -331,13 +306,11 @@ utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> Uniqu
return CreationStatus::SUCCESS;
}
UniqueConstraints::DeletionStatus UniqueConstraints::DropConstraint(LabelId label,
const std::set<PropertyId> &properties) {
if (properties.empty()) {
return UniqueConstraints::DeletionStatus::EMPTY_PROPERTIES;
}
if (properties.size() > kUniqueConstraintsMaxProperties) {
return UniqueConstraints::DeletionStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED;
InMemoryUniqueConstraints::DeletionStatus InMemoryUniqueConstraints::DropConstraint(
LabelId label, const std::set<PropertyId> &properties) {
if (auto drop_properties_check_result = UniqueConstraints::CheckPropertiesBeforeDeletion(properties);
drop_properties_check_result != UniqueConstraints::DeletionStatus::SUCCESS) {
return drop_properties_check_result;
}
if (constraints_.erase({label, properties}) > 0) {
return UniqueConstraints::DeletionStatus::SUCCESS;
@ -345,8 +318,12 @@ UniqueConstraints::DeletionStatus UniqueConstraints::DropConstraint(LabelId labe
return UniqueConstraints::DeletionStatus::NOT_FOUND;
}
std::optional<ConstraintViolation> UniqueConstraints::Validate(const Vertex &vertex, const Transaction &tx,
uint64_t commit_timestamp) const {
bool InMemoryUniqueConstraints::ConstraintExists(LabelId label, const std::set<PropertyId> &properties) const {
return constraints_.find({label, properties}) != constraints_.end();
}
std::optional<ConstraintViolation> InMemoryUniqueConstraints::Validate(const Vertex &vertex, const Transaction &tx,
uint64_t commit_timestamp) const {
if (vertex.deleted) {
return std::nullopt;
}
@ -357,7 +334,7 @@ std::optional<ConstraintViolation> UniqueConstraints::Validate(const Vertex &ver
continue;
}
auto value_array = ExtractPropertyValues(vertex, properties);
auto value_array = vertex.properties.ExtractPropertyValues(properties);
if (!value_array) {
continue;
}
@ -381,7 +358,7 @@ std::optional<ConstraintViolation> UniqueConstraints::Validate(const Vertex &ver
return std::nullopt;
}
std::vector<std::pair<LabelId, std::set<PropertyId>>> UniqueConstraints::ListConstraints() const {
std::vector<std::pair<LabelId, std::set<PropertyId>>> InMemoryUniqueConstraints::ListConstraints() const {
std::vector<std::pair<LabelId, std::set<PropertyId>>> ret;
ret.reserve(constraints_.size());
for (const auto &[label_props, _] : constraints_) {
@ -390,7 +367,7 @@ std::vector<std::pair<LabelId, std::set<PropertyId>>> UniqueConstraints::ListCon
return ret;
}
void UniqueConstraints::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
void InMemoryUniqueConstraints::RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp) {
for (auto &[label_props, storage] : constraints_) {
auto acc = storage.access();
for (auto it = acc.begin(); it != acc.end();) {
@ -412,4 +389,6 @@ void UniqueConstraints::RemoveObsoleteEntries(uint64_t oldest_active_start_times
}
}
void InMemoryUniqueConstraints::Clear() { constraints_.clear(); }
} // namespace memgraph::storage

View File

@ -0,0 +1,101 @@
// 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 "storage/v2/constraints/unique_constraints.hpp"
namespace memgraph::storage {
/// Utility class to store data in a fixed size array. The array is used
/// instead of `std::vector` to avoid `std::bad_alloc` exception where not
/// necessary.
template <class T>
struct FixedCapacityArray {
size_t size;
T values[kUniqueConstraintsMaxProperties];
explicit FixedCapacityArray(size_t array_size) : size(array_size) {
MG_ASSERT(size <= kUniqueConstraintsMaxProperties, "Invalid array size!");
}
};
using PropertyIdArray = FixedCapacityArray<PropertyId>;
class InMemoryUniqueConstraints : public UniqueConstraints {
private:
struct Entry {
std::vector<PropertyValue> values;
const Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs) const;
bool operator==(const Entry &rhs) const;
bool operator<(const std::vector<PropertyValue> &rhs) const;
bool operator==(const std::vector<PropertyValue> &rhs) const;
};
public:
/// Indexes the given vertex for relevant labels and properties.
/// This method should be called before committing and validating vertices
/// against unique constraints.
/// @throw std::bad_alloc
void UpdateBeforeCommit(const Vertex *vertex, const Transaction &tx);
/// Creates unique constraint on the given `label` and a list of `properties`.
/// Returns constraint violation if there are multiple vertices with the same
/// label and property values. Returns `CreationStatus::ALREADY_EXISTS` if
/// constraint already existed, `CreationStatus::EMPTY_PROPERTIES` if the
/// given list of properties is empty,
/// `CreationStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED` if the list of properties
/// exceeds the maximum allowed number of properties, and
/// `CreationStatus::SUCCESS` on success.
/// @throw std::bad_alloc
utils::BasicResult<ConstraintViolation, CreationStatus> CreateConstraint(LabelId label,
const std::set<PropertyId> &properties,
utils::SkipList<Vertex>::Accessor vertices);
/// Deletes the specified constraint. Returns `DeletionStatus::NOT_FOUND` if
/// there is not such constraint in the storage,
/// `DeletionStatus::EMPTY_PROPERTIES` if the given set of `properties` is
/// empty, `DeletionStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED` if the given set
/// of `properties` exceeds the maximum allowed number of properties, and
/// `DeletionStatus::SUCCESS` on success.
DeletionStatus DropConstraint(LabelId label, const std::set<PropertyId> &properties) override;
bool ConstraintExists(LabelId label, const std::set<PropertyId> &properties) const override;
void UpdateOnRemoveLabel(LabelId removed_label, const Vertex &vertex_before_update,
const uint64_t transaction_start_timestamp) override {}
void UpdateOnAddLabel(LabelId added_label, const Vertex &vertex_before_update,
uint64_t transaction_start_timestamp) override{};
/// Validates the given vertex against unique constraints before committing.
/// This method should be called while commit lock is active with
/// `commit_timestamp` being a potential commit timestamp of the transaction.
/// @throw std::bad_alloc
std::optional<ConstraintViolation> Validate(const Vertex &vertex, const Transaction &tx,
uint64_t commit_timestamp) const;
std::vector<std::pair<LabelId, std::set<PropertyId>>> ListConstraints() const override;
/// GC method that removes outdated entries from constraints' storages.
void RemoveObsoleteEntries(uint64_t oldest_active_start_timestamp);
void Clear() override;
private:
std::map<std::pair<LabelId, std::set<PropertyId>>, utils::SkipList<Entry>> constraints_;
};
} // namespace memgraph::storage

View File

@ -12,6 +12,7 @@
#pragma once
#include <atomic>
#include <optional>
#include "storage/v2/property_value.hpp"
#include "storage/v2/transaction.hpp"
@ -60,7 +61,9 @@ inline void ApplyDeltasForRead(Transaction *transaction, const Delta *delta, Vie
// We shouldn't undo our older changes because the user requested a OLD view
// of the database.
if (view == View::OLD && ts == commit_timestamp && cid < transaction->command_id) {
if (view == View::OLD && ts == commit_timestamp &&
(cid < transaction->command_id ||
(cid == transaction->command_id && delta->action == Delta::Action::DELETE_DESERIALIZED_OBJECT))) {
break;
}
@ -80,7 +83,6 @@ inline void ApplyDeltasForRead(Transaction *transaction, const Delta *delta, Vie
template <typename TObj>
inline bool PrepareForWrite(Transaction *transaction, TObj *object) {
if (object->delta == nullptr) return true;
auto ts = object->delta->timestamp->load(std::memory_order_acquire);
if (ts == transaction->transaction_id.load(std::memory_order_acquire) || ts < transaction->start_timestamp) {
return true;
@ -104,6 +106,20 @@ inline Delta *CreateDeleteObjectDelta(Transaction *transaction) {
transaction->command_id);
}
/// TODO: what if in-memory analytical
inline Delta *CreateDeleteDeserializedObjectDelta(Transaction *transaction, std::optional<std::string> old_disk_key) {
transaction->EnsureCommitTimestampExists();
return &transaction->deltas.emplace_back(Delta::DeleteDeserializedObjectTag(), transaction->commit_timestamp.get(),
old_disk_key);
}
/// TODO: what if in-memory analytical
inline Delta *CreateDeleteDeserializedIndexObjectDelta(Transaction *transaction, std::list<Delta> &deltas,
std::optional<std::string> old_disk_key) {
transaction->EnsureCommitTimestampExists();
return &deltas.emplace_back(Delta::DeleteDeserializedObjectTag(), transaction->commit_timestamp.get(), old_disk_key);
}
/// This function creates a delta in the transaction for the object and links
/// the delta into the object's delta list.
/// @throw std::bad_alloc

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
@ -20,14 +20,14 @@
namespace memgraph::storage {
class NameIdMapper final {
class NameIdMapper {
private:
struct MapNameToId {
std::string name;
uint64_t id;
bool operator<(const MapNameToId &other) { return name < other.name; }
bool operator==(const MapNameToId &other) { return name == other.name; }
bool operator<(const MapNameToId &other) const { return name < other.name; }
bool operator==(const MapNameToId &other) const { return name == other.name; }
bool operator<(const std::string_view other) const { return name < other; }
bool operator==(const std::string_view other) const { return name == other; }
@ -37,16 +37,25 @@ class NameIdMapper final {
uint64_t id;
std::string name;
bool operator<(const MapIdToName &other) { return id < other.id; }
bool operator==(const MapIdToName &other) { return id == other.id; }
bool operator<(const MapIdToName &other) const { return id < other.id; }
bool operator==(const MapIdToName &other) const { return id == other.id; }
bool operator<(uint64_t other) { return id < other; }
bool operator==(uint64_t other) { return id == other; }
bool operator<(uint64_t other) const { return id < other; }
bool operator==(uint64_t other) const { return id == other; }
};
public:
explicit NameIdMapper() = default;
NameIdMapper(const NameIdMapper &) = delete;
NameIdMapper &operator=(const NameIdMapper &) = delete;
NameIdMapper(NameIdMapper &&) = delete;
NameIdMapper &operator=(NameIdMapper &&) = delete;
virtual ~NameIdMapper() = default;
/// @throw std::bad_alloc if unable to insert a new mapping
uint64_t NameToId(const std::string_view name) {
virtual uint64_t NameToId(const std::string_view name) {
auto name_to_id_acc = name_to_id_.access();
auto found = name_to_id_acc.find(name);
uint64_t id;
@ -83,14 +92,22 @@ class NameIdMapper final {
// Currently, we never delete anything from the `utils::SkipList` so the
// references will always be valid. If you change this class to remove unused
// names, be sure to change the signature of this function.
const std::string &IdToName(uint64_t id) const {
virtual const std::string &IdToName(uint64_t id) {
auto maybe_name = MaybeIdToName(id);
MG_ASSERT(maybe_name.has_value(), "Trying to get a name for an invalid ID!");
return maybe_name.value();
}
protected:
std::optional<std::reference_wrapper<const std::string>> MaybeIdToName(uint64_t id) const {
auto id_to_name_acc = id_to_name_.access();
auto result = id_to_name_acc.find(id);
MG_ASSERT(result != id_to_name_acc.end(), "Trying to get a name for an invalid ID!");
if (result == id_to_name_acc.end()) {
return std::nullopt;
}
return result->name;
}
private:
std::atomic<uint64_t> counter_{0};
utils::SkipList<MapNameToId> name_to_id_;
utils::SkipList<MapIdToName> id_to_name_;

View File

@ -11,9 +11,12 @@
#include "storage/v2/property_store.hpp"
#include <cstdint>
#include <cstring>
#include <iterator>
#include <limits>
#include <optional>
#include <sstream>
#include <tuple>
#include <type_traits>
#include <utility>
@ -984,6 +987,39 @@ bool PropertyStore::HasProperty(PropertyId property) const {
return FindSpecificProperty(&reader, property, nullptr) == DecodeExpectedPropertyStatus::EQUAL;
}
/// TODO: andi write a unit test for it
bool PropertyStore::HasAllProperties(const std::set<PropertyId> &properties) const {
return std::all_of(properties.begin(), properties.end(), [this](const auto &prop) { return HasProperty(prop); });
}
/// TODO: andi write a unit test for it
bool PropertyStore::HasAllPropertyValues(const std::vector<PropertyValue> &property_values) const {
/// TODO: andi extract this into a private method
auto property_map = Properties();
std::vector<PropertyValue> all_property_values;
transform(property_map.begin(), property_map.end(), back_inserter(all_property_values),
[](const auto &kv_entry) { return kv_entry.second; });
return std::all_of(
property_values.begin(), property_values.end(), [&all_property_values](const PropertyValue &value) {
return std::find(all_property_values.begin(), all_property_values.end(), value) != all_property_values.end();
});
}
std::optional<std::vector<PropertyValue>> PropertyStore::ExtractPropertyValues(
const std::set<PropertyId> &properties) const {
std::vector<PropertyValue> value_array;
value_array.reserve(properties.size());
for (const auto &prop : properties) {
auto value = GetProperty(prop);
if (value.IsNull()) {
return std::nullopt;
}
value_array.emplace_back(std::move(value));
}
return value_array;
}
bool PropertyStore::IsPropertyEqual(PropertyId property, const PropertyValue &value) const {
uint64_t size;
const uint8_t *data;
@ -1234,4 +1270,41 @@ bool PropertyStore::ClearProperties() {
return true;
}
std::string PropertyStore::StringBuffer() const {
uint64_t size = 0;
const uint8_t *data = nullptr;
std::tie(size, data) = GetSizeData(buffer_);
if (size % 8 != 0) { // We are storing the data in the local buffer.
size = sizeof(buffer_) - 1;
data = &buffer_[1];
}
std::string arr(size, ' ');
for (uint i = 0; i < size; ++i) {
arr[i] = static_cast<char>(data[i]);
}
return arr;
}
void PropertyStore::SetBuffer(const std::string_view buffer) {
if (buffer.empty()) {
return;
}
uint64_t size = 0;
uint8_t *data = nullptr;
if (buffer.size() == sizeof(buffer_) - 1) { // use local buffer
buffer_[0] = kUseLocalBuffer;
size = buffer.size() - 1;
data = &buffer_[1];
} else {
size = buffer.size();
data = new uint8_t[size];
SetSizeData(buffer_, size, data);
}
for (uint i = 0; i < size; ++i) {
data[i] = static_cast<uint8_t>(buffer[i]);
}
}
} // namespace memgraph::storage

View File

@ -12,6 +12,7 @@
#pragma once
#include <map>
#include <set>
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_value.hpp"
@ -23,6 +24,12 @@ class PropertyStore {
"PropertyStore supports only architectures using little-endian.");
public:
static PropertyStore CreateFromBuffer(std::string_view buffer) {
PropertyStore store;
store.SetBuffer(buffer);
return store;
}
PropertyStore();
PropertyStore(const PropertyStore &) = delete;
@ -42,6 +49,19 @@ class PropertyStore {
/// complexity of this function is O(n).
bool HasProperty(PropertyId property) const;
/// Checks whether all properties in the set `properties` exist in the store. The time
/// complexity of this function is O(n^2).
bool HasAllProperties(const std::set<PropertyId> &properties) const;
/// Checks whether all property values in the vector `property_values` exist in the store. The time
/// complexity of this function is O(n^2).
/// TODO: andi Not so sure it is quadratic complexity
bool HasAllPropertyValues(const std::vector<PropertyValue> &property_values) const;
/// Extracts property values for all property ids in the set `properties`. The time
/// complexity of this function is O(n^2).
std::optional<std::vector<PropertyValue>> ExtractPropertyValues(const std::set<PropertyId> &properties) const;
/// Checks whether the property `property` is equal to the specified value
/// `value`. This function doesn't perform any memory allocations while
/// performing the equality check. The time complexity of this function is
@ -77,6 +97,12 @@ class PropertyStore {
/// @throw std::bad_alloc
bool ClearProperties();
/// Return property buffer as a string
std::string StringBuffer() const;
/// Sets buffer
void SetBuffer(std::string_view buffer);
private:
template <typename TContainer>
bool DoInitProperties(const TContainer &properties);

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
@ -300,6 +300,7 @@ inline bool operator==(const PropertyValue &first, const PropertyValue &second)
}
}
/// NOLINTNEXTLINE(bugprone-exception-escape)
inline bool operator<(const PropertyValue &first, const PropertyValue &second) noexcept {
if (!PropertyValue::AreComparableTypes(first.type(), second.type())) return first.type() < second.type();
switch (first.type()) {
@ -330,6 +331,9 @@ inline bool operator<(const PropertyValue &first, const PropertyValue &second) n
}
}
/// NOLINTNEXTLINE(bugprone-exception-escape)
inline bool operator>(const PropertyValue &first, const PropertyValue &second) noexcept { return second < first; }
inline PropertyValue::PropertyValue(const PropertyValue &other) : type_(other.type_) {
switch (other.type_) {
case Type::Null:

View File

@ -30,9 +30,10 @@ template <typename>
} // namespace
////// ReplicationClient //////
Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
const replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
InMemoryStorage::ReplicationClient::ReplicationClient(std::string name, InMemoryStorage *storage,
const io::network::Endpoint &endpoint,
const replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
: name_(std::move(name)), storage_(storage), mode_(mode) {
if (config.ssl) {
rpc_context_.emplace(config.ssl->key_file, config.ssl->cert_file);
@ -49,14 +50,14 @@ Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage
}
}
void Storage::ReplicationClient::TryInitializeClientAsync() {
void InMemoryStorage::ReplicationClient::TryInitializeClientAsync() {
thread_pool_.AddTask([this] {
rpc_client_->Abort();
this->TryInitializeClientSync();
});
}
void Storage::ReplicationClient::FrequentCheck() {
void InMemoryStorage::ReplicationClient::FrequentCheck() {
const auto is_success = std::invoke([this]() {
try {
auto stream{rpc_client_->Stream<replication::FrequentHeartbeatRpc>()};
@ -82,7 +83,7 @@ void Storage::ReplicationClient::FrequentCheck() {
}
/// @throws rpc::RpcFailedException
void Storage::ReplicationClient::InitializeClient() {
void InMemoryStorage::ReplicationClient::InitializeClient() {
uint64_t current_commit_timestamp{kTimestampInitialId};
std::optional<std::string> epoch_id;
@ -134,7 +135,7 @@ void Storage::ReplicationClient::InitializeClient() {
}
}
void Storage::ReplicationClient::TryInitializeClientSync() {
void InMemoryStorage::ReplicationClient::TryInitializeClientSync() {
try {
InitializeClient();
} catch (const rpc::RpcFailedException &) {
@ -145,19 +146,19 @@ void Storage::ReplicationClient::TryInitializeClientSync() {
}
}
void Storage::ReplicationClient::HandleRpcFailure() {
void InMemoryStorage::ReplicationClient::HandleRpcFailure() {
spdlog::error(utils::MessageWithLink("Couldn't replicate data to {}.", name_, "https://memgr.ph/replication"));
TryInitializeClientAsync();
}
replication::SnapshotRes Storage::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
replication::SnapshotRes InMemoryStorage::ReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
auto stream{rpc_client_->Stream<replication::SnapshotRpc>()};
replication::Encoder encoder(stream.GetBuilder());
encoder.WriteFile(path);
return stream.AwaitResponse();
}
replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles(
replication::WalFilesRes InMemoryStorage::ReplicationClient::TransferWalFiles(
const std::vector<std::filesystem::path> &wal_files) {
MG_ASSERT(!wal_files.empty(), "Wal files list is empty!");
auto stream{rpc_client_->Stream<replication::WalFilesRpc>(wal_files.size())};
@ -170,7 +171,7 @@ replication::WalFilesRes Storage::ReplicationClient::TransferWalFiles(
return stream.AwaitResponse();
}
void Storage::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
void InMemoryStorage::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
std::unique_lock guard(client_lock_);
const auto status = replica_state_.load();
switch (status) {
@ -204,7 +205,8 @@ void Storage::ReplicationClient::StartTransactionReplication(const uint64_t curr
}
}
void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) {
void InMemoryStorage::ReplicationClient::IfStreamingTransaction(
const std::function<void(ReplicaStream &handler)> &callback) {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
@ -224,7 +226,7 @@ void Storage::ReplicationClient::IfStreamingTransaction(const std::function<void
}
}
bool Storage::ReplicationClient::FinalizeTransactionReplication() {
bool InMemoryStorage::ReplicationClient::FinalizeTransactionReplication() {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
@ -241,7 +243,7 @@ bool Storage::ReplicationClient::FinalizeTransactionReplication() {
}
}
bool Storage::ReplicationClient::FinalizeTransactionReplicationInternal() {
bool InMemoryStorage::ReplicationClient::FinalizeTransactionReplicationInternal() {
MG_ASSERT(replica_stream_, "Missing stream for transaction deltas");
try {
auto response = replica_stream_->Finalize();
@ -265,7 +267,7 @@ bool Storage::ReplicationClient::FinalizeTransactionReplicationInternal() {
return false;
}
void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
void InMemoryStorage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
while (true) {
auto file_locker = storage_->file_retainer_.AddLocker();
@ -327,7 +329,7 @@ void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
}
}
uint64_t Storage::ReplicationClient::ReplicateCurrentWal() {
uint64_t InMemoryStorage::ReplicationClient::ReplicateCurrentWal() {
const auto &wal_file = storage_->wal_file_;
auto stream = TransferCurrentWalFile();
stream.AppendFilename(wal_file->Path().filename());
@ -361,7 +363,7 @@ uint64_t Storage::ReplicationClient::ReplicateCurrentWal() {
/// recovery steps, so we can safely send it to the replica.
/// We assume that the property of preserving at least 1 WAL before the snapshot
/// is satisfied as we extract the timestamp information from it.
std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient::GetRecoverySteps(
std::vector<InMemoryStorage::ReplicationClient::RecoveryStep> InMemoryStorage::ReplicationClient::GetRecoverySteps(
const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) {
// First check if we can recover using the current wal file only
// otherwise save the seq_num of the current wal file
@ -507,8 +509,8 @@ std::vector<Storage::ReplicationClient::RecoveryStep> Storage::ReplicationClient
return recovery_steps;
}
Storage::TimestampInfo Storage::ReplicationClient::GetTimestampInfo() {
Storage::TimestampInfo info;
InMemoryStorage::TimestampInfo InMemoryStorage::ReplicationClient::GetTimestampInfo() {
InMemoryStorage::TimestampInfo info;
info.current_timestamp_of_replica = 0;
info.current_number_of_timestamp_behind_master = 0;
@ -536,65 +538,71 @@ Storage::TimestampInfo Storage::ReplicationClient::GetTimestampInfo() {
}
////// ReplicaStream //////
Storage::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self,
const uint64_t previous_commit_timestamp,
const uint64_t current_seq_num)
InMemoryStorage::ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self,
const uint64_t previous_commit_timestamp,
const uint64_t current_seq_num)
: self_(self),
stream_(self_->rpc_client_->Stream<replication::AppendDeltasRpc>(previous_commit_timestamp, current_seq_num)) {
replication::Encoder encoder{stream_.GetBuilder()};
encoder.WriteString(self_->storage_->epoch_id_);
}
void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex,
uint64_t final_commit_timestamp) {
void InMemoryStorage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex,
uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, self_->storage_->config_.items, delta, vertex,
EncodeDelta(&encoder, self_->storage_->name_id_mapper_.get(), self_->storage_->config_.items, delta, vertex,
final_commit_timestamp);
}
void Storage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge,
uint64_t final_commit_timestamp) {
void InMemoryStorage::ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge,
uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, delta, edge, final_commit_timestamp);
EncodeDelta(&encoder, self_->storage_->name_id_mapper_.get(), delta, edge, final_commit_timestamp);
}
void Storage::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
void InMemoryStorage::ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeTransactionEnd(&encoder, final_commit_timestamp);
}
void Storage::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties,
uint64_t timestamp) {
void InMemoryStorage::ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation,
LabelId label,
const std::set<PropertyId> &properties,
uint64_t timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeOperation(&encoder, &self_->storage_->name_id_mapper_, operation, label, properties, timestamp);
EncodeOperation(&encoder, self_->storage_->name_id_mapper_.get(), operation, label, properties, timestamp);
}
replication::AppendDeltasRes Storage::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
replication::AppendDeltasRes InMemoryStorage::ReplicationClient::ReplicaStream::Finalize() {
return stream_.AwaitResponse();
}
////// CurrentWalHandler //////
Storage::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
InMemoryStorage::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
: self_(self), stream_(self_->rpc_client_->Stream<replication::CurrentWalRpc>()) {}
void Storage::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
void InMemoryStorage::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteString(filename);
}
void Storage::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
void InMemoryStorage::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteUint(size);
}
void Storage::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) {
void InMemoryStorage::ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteFileData(file);
}
void Storage::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) {
void InMemoryStorage::ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer,
const size_t buffer_size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteBuffer(buffer, buffer_size);
}
replication::CurrentWalRes Storage::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
replication::CurrentWalRes InMemoryStorage::ReplicationClient::CurrentWalHandler::Finalize() {
return stream_.AwaitResponse();
}
} // namespace memgraph::storage

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,6 +21,7 @@
#include "storage/v2/delta.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/property_value.hpp"
@ -28,7 +29,6 @@
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/storage.hpp"
#include "utils/file.hpp"
#include "utils/file_locker.hpp"
#include "utils/spin_lock.hpp"
@ -37,9 +37,9 @@
namespace memgraph::storage {
class Storage::ReplicationClient {
class InMemoryStorage::ReplicationClient {
public:
ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
ReplicationClient(std::string name, InMemoryStorage *storage, const io::network::Endpoint &endpoint,
replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {});
// Handler used for transfering the current transaction.
@ -123,7 +123,7 @@ class Storage::ReplicationClient {
const auto &Endpoint() const { return rpc_client_->Endpoint(); }
Storage::TimestampInfo GetTimestampInfo();
InMemoryStorage::TimestampInfo GetTimestampInfo();
private:
[[nodiscard]] bool FinalizeTransactionReplicationInternal();
@ -150,7 +150,7 @@ class Storage::ReplicationClient {
void HandleRpcFailure();
std::string name_;
Storage *storage_;
InMemoryStorage *storage_;
std::optional<communication::ClientContext> rpc_context_;
std::optional<rpc::Client> rpc_client_;

View File

@ -9,18 +9,22 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/replication/replication_server.hpp"
#include <atomic>
#include <filesystem>
#include "spdlog/spdlog.h"
#include "storage/v2/delta.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/serialization.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/transaction.hpp"
#include "utils/exceptions.hpp"
@ -40,8 +44,8 @@ std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder
};
} // namespace
Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config)
InMemoryStorage::ReplicationServer::ReplicationServer(InMemoryStorage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config)
: storage_(storage) {
// Create RPC server.
if (config.ssl) {
@ -88,22 +92,21 @@ Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::End
rpc_server_->Start();
}
void Storage::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
void InMemoryStorage::ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::HeartbeatReq req;
slk::Load(&req, req_reader);
replication::HeartbeatRes res{true, storage_->last_commit_timestamp_.load(), storage_->epoch_id_};
slk::Save(res, res_builder);
}
void Storage::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
void InMemoryStorage::ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::FrequentHeartbeatReq req;
slk::Load(&req, req_reader);
replication::FrequentHeartbeatRes res{true};
slk::Save(res, res_builder);
}
void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
spdlog::debug("Started replication recovery from appending deltas!");
void InMemoryStorage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::AppendDeltasReq req;
slk::Load(&req, req_reader);
@ -152,8 +155,7 @@ void Storage::ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, sl
spdlog::debug("Replication recovery from append deltas finished, replica is now up to date!");
}
void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
spdlog::debug("Started replication recovery from received snapshot file!");
void InMemoryStorage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::SnapshotReq req;
slk::Load(&req, req_reader);
@ -171,14 +173,16 @@ void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::B
storage_->vertices_.clear();
storage_->edges_.clear();
storage_->constraints_ = Constraints();
storage_->indices_.label_index = LabelIndex(&storage_->indices_, &storage_->constraints_, storage_->config_.items);
storage_->indices_.label_property_index =
LabelPropertyIndex(&storage_->indices_, &storage_->constraints_, storage_->config_.items);
storage_->constraints_.existence_constraints_ = std::make_unique<ExistenceConstraints>();
storage_->constraints_.unique_constraints_ = std::make_unique<InMemoryUniqueConstraints>();
storage_->indices_.label_index_ =
std::make_unique<InMemoryLabelIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
storage_->indices_.label_property_index_ =
std::make_unique<InMemoryLabelPropertyIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
try {
spdlog::debug("Loading snapshot");
auto recovered_snapshot = durability::LoadSnapshot(*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_,
&storage_->epoch_history_, &storage_->name_id_mapper_,
&storage_->epoch_history_, storage_->name_id_mapper_.get(),
&storage_->edge_count_, storage_->config_);
spdlog::debug("Snapshot loaded successfully");
// If this step is present it should always be the first step of
@ -224,8 +228,7 @@ void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::B
spdlog::debug("Replication recovery from snapshot finished!");
}
void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
spdlog::debug("Started replication recovery from received WAL files!");
void InMemoryStorage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::WalFilesReq req;
slk::Load(&req, req_reader);
@ -245,8 +248,7 @@ void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::B
spdlog::debug("Replication recovery from WAL files ended successfully, replica is now up to date!");
}
void Storage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
spdlog::debug("Started replication recovery from current WAL!");
void InMemoryStorage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::CurrentWalReq req;
slk::Load(&req, req_reader);
@ -261,7 +263,7 @@ void Storage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk:
spdlog::debug("Replication recovery from current WAL ended successfully, replica is now up to date!");
}
void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
void InMemoryStorage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
const auto temp_wal_directory = std::filesystem::temp_directory_path() / "memgraph" / durability::kWalDirectory;
utils::EnsureDir(temp_wal_directory);
auto maybe_wal_path = decoder->ReadFile(temp_wal_directory);
@ -306,7 +308,7 @@ void Storage::ReplicationServer::LoadWal(replication::Decoder *decoder) {
}
}
void Storage::ReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
void InMemoryStorage::ReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::TimestampReq req;
slk::Load(&req, req_reader);
@ -314,25 +316,29 @@ void Storage::ReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::
slk::Save(res, res_builder);
}
Storage::ReplicationServer::~ReplicationServer() {
InMemoryStorage::ReplicationServer::~ReplicationServer() {
if (rpc_server_) {
rpc_server_->Shutdown();
rpc_server_->AwaitShutdown();
}
}
uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
spdlog::debug("Reading and applying missing transaction deltas!");
uint64_t InMemoryStorage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
auto edge_acc = storage_->edges_.access();
auto vertex_acc = storage_->vertices_.access();
std::optional<std::pair<uint64_t, storage::Storage::Accessor>> commit_timestamp_and_accessor;
std::optional<std::pair<uint64_t, std::unique_ptr<storage::Storage::Accessor>>> commit_timestamp_and_accessor;
auto get_transaction = [this, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
if (!commit_timestamp_and_accessor) {
commit_timestamp_and_accessor.emplace(commit_timestamp, storage_->Access());
commit_timestamp_and_accessor.emplace(commit_timestamp, storage_->Access(std::optional<IsolationLevel>{}));
} else if (commit_timestamp_and_accessor->first != commit_timestamp) {
throw utils::BasicException("Received more than one transaction!");
}
return &commit_timestamp_and_accessor->second;
// TODO: Rethink this if we would reuse ReplicationServer for on disk storage.
if (auto *inmemoryAcc =
dynamic_cast<storage::InMemoryStorage::InMemoryAccessor *>(commit_timestamp_and_accessor->second.get())) {
return inmemoryAcc;
}
throw utils::BasicException("Received transaction for not supported storage!");
};
uint64_t applied_deltas = 0;
@ -470,6 +476,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
is_visible = true;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
@ -503,7 +510,7 @@ uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *
spdlog::trace(" Transaction end");
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
throw utils::BasicException("Invalid data!");
auto ret = commit_timestamp_and_accessor->second.Commit(commit_timestamp_and_accessor->first);
auto ret = commit_timestamp_and_accessor->second->Commit(commit_timestamp_and_accessor->first);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
commit_timestamp_and_accessor = std::nullopt;
break;

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
@ -11,13 +11,16 @@
#pragma once
#include "storage/v2/storage.hpp"
#include "rpc/server.hpp"
#include "slk/streams.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/replication/replication_client.hpp"
namespace memgraph::storage {
class Storage::ReplicationServer {
class InMemoryStorage::ReplicationServer {
public:
explicit ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
explicit ReplicationServer(InMemoryStorage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config);
ReplicationServer(const ReplicationServer &) = delete;
ReplicationServer(ReplicationServer &&) = delete;
@ -42,7 +45,7 @@ class Storage::ReplicationServer {
std::optional<communication::ServerContext> rpc_server_context_;
std::optional<rpc::Server> rpc_server_;
Storage *storage_;
InMemoryStorage *storage_;
};
} // namespace memgraph::storage

File diff suppressed because it is too large Load Diff

View File

@ -11,176 +11,50 @@
#pragma once
#include <atomic>
#include <cstdint>
#include <filesystem>
#include <optional>
#include <shared_mutex>
#include <span>
#include <variant>
#include "io/network/endpoint.hpp"
#include "kvstore/kvstore.hpp"
#include "query/exceptions.hpp"
#include "storage/v2/all_vertices_iterable.hpp"
#include "storage/v2/commit_log.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/constraints.hpp"
#include "storage/v2/durability/metadata.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/result.hpp"
#include "storage/v2/storage_error.hpp"
#include "storage/v2/storage_mode.hpp"
#include "storage/v2/transaction.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/result.hpp"
#include "utils/rw_lock.hpp"
#include "storage/v2/vertices_iterable.hpp"
#include "utils/event_counter.hpp"
#include "utils/event_histogram.hpp"
#include "utils/scheduler.hpp"
#include "utils/skip_list.hpp"
#include "utils/synchronized.hpp"
#include "utils/timer.hpp"
#include "utils/uuid.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/replication_persistence_helper.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/storage_error.hpp"
namespace memgraph::metrics {
extern const Event SnapshotCreationLatency_us;
extern const Event ActiveLabelIndices;
extern const Event ActiveLabelPropertyIndices;
} // namespace memgraph::metrics
namespace memgraph::storage {
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
// The paper implements a fully serializable storage, in our implementation we
// only implement snapshot isolation for transactions.
struct Transaction;
class EdgeAccessor;
/// Iterable for iterating through all vertices of a Storage.
///
/// An instance of this will be usually be wrapped inside VerticesIterable for
/// generic, public use.
class AllVerticesIterable final {
utils::SkipList<Vertex>::Accessor vertices_accessor_;
Transaction *transaction_;
View view_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
std::optional<VertexAccessor> vertex_;
public:
class Iterator final {
AllVerticesIterable *self_;
utils::SkipList<Vertex>::Iterator it_;
public:
Iterator(AllVerticesIterable *self, utils::SkipList<Vertex>::Iterator it);
VertexAccessor operator*() const;
Iterator &operator++();
bool operator==(const Iterator &other) const { return self_ == other.self_ && it_ == other.it_; }
bool operator!=(const Iterator &other) const { return !(*this == other); }
};
AllVerticesIterable(utils::SkipList<Vertex>::Accessor vertices_accessor, Transaction *transaction, View view,
Indices *indices, Constraints *constraints, Config::Items config)
: vertices_accessor_(std::move(vertices_accessor)),
transaction_(transaction),
view_(view),
indices_(indices),
constraints_(constraints),
config_(config) {}
Iterator begin() { return Iterator(this, vertices_accessor_.begin()); }
Iterator end() { return Iterator(this, vertices_accessor_.end()); }
};
/// Generic access to different kinds of vertex iterations.
///
/// This class should be the primary type used by the client code to iterate
/// over vertices inside a Storage instance.
class VerticesIterable final {
enum class Type { ALL, BY_LABEL, BY_LABEL_PROPERTY };
Type type_;
union {
AllVerticesIterable all_vertices_;
LabelIndex::Iterable vertices_by_label_;
LabelPropertyIndex::Iterable vertices_by_label_property_;
};
public:
explicit VerticesIterable(AllVerticesIterable);
explicit VerticesIterable(LabelIndex::Iterable);
explicit VerticesIterable(LabelPropertyIndex::Iterable);
VerticesIterable(const VerticesIterable &) = delete;
VerticesIterable &operator=(const VerticesIterable &) = delete;
VerticesIterable(VerticesIterable &&) noexcept;
VerticesIterable &operator=(VerticesIterable &&) noexcept;
~VerticesIterable();
class Iterator final {
Type type_;
union {
AllVerticesIterable::Iterator all_it_;
LabelIndex::Iterable::Iterator by_label_it_;
LabelPropertyIndex::Iterable::Iterator by_label_property_it_;
};
void Destroy() noexcept;
public:
explicit Iterator(AllVerticesIterable::Iterator);
explicit Iterator(LabelIndex::Iterable::Iterator);
explicit Iterator(LabelPropertyIndex::Iterable::Iterator);
Iterator(const Iterator &);
Iterator &operator=(const Iterator &);
Iterator(Iterator &&) noexcept;
Iterator &operator=(Iterator &&) noexcept;
~Iterator();
VertexAccessor operator*() const;
Iterator &operator++();
bool operator==(const Iterator &other) const;
bool operator!=(const Iterator &other) const { return !(*this == other); }
};
Iterator begin();
Iterator end();
};
/// Structure used to return information about existing indices in the storage.
struct IndicesInfo {
std::vector<LabelId> label;
std::vector<std::pair<LabelId, PropertyId>> label_property;
};
/// Structure used to return information about existing constraints in the
/// storage.
struct ConstraintsInfo {
std::vector<std::pair<LabelId, PropertyId>> existence;
std::vector<std::pair<LabelId, std::set<PropertyId>>> unique;
};
/// Structure used to return information about the storage.
struct StorageInfo {
uint64_t vertex_count;
uint64_t edge_count;
@ -189,541 +63,244 @@ struct StorageInfo {
uint64_t disk_usage;
};
class Storage final {
class Storage {
public:
/// @throw std::system_error
/// @throw std::bad_alloc
explicit Storage(Config config = Config());
Storage(Config config, StorageMode storage_mode);
~Storage();
Storage(const Storage &) = delete;
Storage(Storage &&) = delete;
Storage &operator=(const Storage &) = delete;
Storage &operator=(Storage &&) = delete;
class Accessor final {
private:
friend class Storage;
explicit Accessor(Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
virtual ~Storage() {}
class Accessor {
public:
Accessor(Storage *storage, IsolationLevel isolation_level, StorageMode storage_mode);
Accessor(const Accessor &) = delete;
Accessor &operator=(const Accessor &) = delete;
Accessor &operator=(Accessor &&other) = delete;
// NOTE: After the accessor is moved, all objects derived from it (accessors
// and iterators) are *invalid*. You have to get all derived objects again.
Accessor(Accessor &&other) noexcept;
~Accessor();
virtual ~Accessor() {}
/// @throw std::bad_alloc
VertexAccessor CreateVertex();
virtual VertexAccessor CreateVertex() = 0;
std::optional<VertexAccessor> FindVertex(Gid gid, View view);
virtual std::optional<VertexAccessor> FindVertex(Gid gid, View view) = 0;
VerticesIterable Vertices(View view) {
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
&storage_->indices_, &storage_->constraints_,
storage_->config_.items));
}
virtual VerticesIterable Vertices(View view) = 0;
VerticesIterable Vertices(LabelId label, View view);
virtual VerticesIterable Vertices(LabelId label, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view) = 0;
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
virtual VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view) = 0;
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount() const { return storage_->vertices_.size(); }
virtual uint64_t ApproximateVertexCount() const = 0;
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label) const {
return storage_->indices_.label_index.ApproximateVertexCount(label);
}
virtual uint64_t ApproximateVertexCount(LabelId label) const = 0;
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
}
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property) const = 0;
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
}
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const = 0;
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
}
virtual uint64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const = 0;
template <typename TResult, typename TIndex, typename TIndexKey>
std::optional<TResult> GetIndexStatsForIndex(TIndex &index, TIndexKey &&key) const {
return index.GetIndexStats(key);
}
virtual std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const = 0;
std::optional<storage::LabelIndexStats> GetIndexStats(const storage::LabelId &label) const {
return GetIndexStatsForIndex<storage::LabelIndexStats>(storage_->indices_.label_index, label);
}
virtual std::optional<storage::LabelPropertyIndexStats> GetIndexStats(
const storage::LabelId &label, const storage::PropertyId &property) const = 0;
std::optional<storage::LabelPropertyIndexStats> GetIndexStats(const storage::LabelId &label,
const storage::PropertyId &property) const {
return GetIndexStatsForIndex<storage::LabelPropertyIndexStats>(storage_->indices_.label_property_index,
std::make_pair(label, property));
}
virtual void SetIndexStats(const storage::LabelId &label, const LabelIndexStats &stats) = 0;
template <typename TIndex, typename TIndexKey, typename TIndexStats>
void SetIndexStatsForIndex(TIndex &index, TIndexKey &&key, TIndexStats &stats) const {
index.SetIndexStats(key, stats);
}
virtual void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const LabelPropertyIndexStats &stats) = 0;
void SetIndexStats(const storage::LabelId &label, const LabelIndexStats &stats) {
SetIndexStatsForIndex(storage_->indices_.label_index, label, stats);
}
virtual std::vector<std::pair<LabelId, PropertyId>> ClearLabelPropertyIndexStats() = 0;
void SetIndexStats(const storage::LabelId &label, const storage::PropertyId &property,
const LabelPropertyIndexStats &stats) {
SetIndexStatsForIndex(storage_->indices_.label_property_index, std::make_pair(label, property), stats);
}
virtual std::vector<LabelId> ClearLabelIndexStats() = 0;
template <typename TResult, typename TIndex>
std::vector<TResult> ClearIndexStatsForIndex(TIndex &index) const {
return index.ClearIndexStats();
}
virtual std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(
std::span<std::string> labels) = 0;
std::vector<std::pair<LabelId, PropertyId>> ClearLabelPropertyIndexStats() {
return ClearIndexStatsForIndex<std::pair<LabelId, PropertyId>>(storage_->indices_.label_property_index);
}
virtual std::vector<LabelId> DeleteLabelIndexStats(std::span<std::string> labels) = 0;
std::vector<LabelId> ClearLabelIndexStats() {
return ClearIndexStatsForIndex<LabelId>(storage_->indices_.label_index);
}
virtual Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex) = 0;
template <typename TResult, typename TIndex>
std::vector<TResult> DeleteIndexStatsForIndex(TIndex &index, const std::span<std::string> labels) {
std::vector<TResult> deleted_indexes;
virtual Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex) = 0;
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;
}
virtual void PrefetchInEdges(const VertexAccessor &vertex_acc) = 0;
std::vector<std::pair<LabelId, PropertyId>> DeleteLabelPropertyIndexStats(const std::span<std::string> labels) {
return DeleteIndexStatsForIndex<std::pair<LabelId, PropertyId>>(storage_->indices_.label_property_index, labels);
}
virtual void PrefetchOutEdges(const VertexAccessor &vertex_acc) = 0;
std::vector<LabelId> DeleteLabelIndexStats(const std::span<std::string> labels) {
return DeleteIndexStatsForIndex<LabelId>(storage_->indices_.label_index, labels);
}
virtual Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type) = 0;
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
virtual Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge) = 0;
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex);
virtual bool LabelIndexExists(LabelId label) const = 0;
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
virtual bool LabelPropertyIndexExists(LabelId label, PropertyId property) const = 0;
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
virtual IndicesInfo ListAllIndices() const = 0;
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
virtual ConstraintsInfo ListAllConstraints() const = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name);
// NOLINTNEXTLINE(google-default-arguments)
virtual utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {}) = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name);
virtual void Abort() = 0;
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name);
bool LabelIndexExists(LabelId label) const { return storage_->indices_.label_index.IndexExists(label); }
bool LabelPropertyIndexExists(LabelId label, PropertyId property) const {
return storage_->indices_.label_property_index.IndexExists(label, property);
}
IndicesInfo ListAllIndices() const {
return {storage_->indices_.label_index.ListIndices(), storage_->indices_.label_property_index.ListIndices()};
}
ConstraintsInfo ListAllConstraints() const {
return {ListExistenceConstraints(storage_->constraints_),
storage_->constraints_.unique_constraints.ListConstraints()};
}
void AdvanceCommand();
/// Returns void if the transaction has been committed.
/// Returns `StorageDataManipulationError` if an error occures. Error can be:
/// * `ReplicationError`: there is at least one SYNC replica that has not confirmed receiving the transaction.
/// * `ConstraintViolation`: the changes made by this transaction violate an existence or unique constraint. In this
/// case the transaction is automatically aborted.
/// @throw std::bad_alloc
utils::BasicResult<StorageDataManipulationError, void> Commit(
std::optional<uint64_t> desired_commit_timestamp = {});
/// @throw std::bad_alloc
void Abort();
void FinalizeTransaction();
virtual void FinalizeTransaction() = 0;
std::optional<uint64_t> GetTransactionId() const;
private:
/// @throw std::bad_alloc
VertexAccessor CreateVertex(storage::Gid gid);
void AdvanceCommand();
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, storage::Gid gid);
const std::string &LabelToName(LabelId label) const { return storage_->LabelToName(label); }
const std::string &PropertyToName(PropertyId property) const { return storage_->PropertyToName(property); }
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const { return storage_->EdgeTypeToName(edge_type); }
LabelId NameToLabel(std::string_view name) { return storage_->NameToLabel(name); }
PropertyId NameToProperty(std::string_view name) { return storage_->NameToProperty(name); }
EdgeTypeId NameToEdgeType(std::string_view name) { return storage_->NameToEdgeType(name); }
StorageMode GetCreationStorageMode() const;
protected:
Storage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
Config::Items config_;
private:
StorageMode creation_storage_mode_;
};
Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
return Accessor{this, override_isolation_level.value_or(isolation_level_), storage_mode_};
const std::string &LabelToName(LabelId label) const { return name_id_mapper_->IdToName(label.AsUint()); }
const std::string &PropertyToName(PropertyId property) const { return name_id_mapper_->IdToName(property.AsUint()); }
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const {
return name_id_mapper_->IdToName(edge_type.AsUint());
}
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
LabelId NameToLabel(const std::string_view name) const { return LabelId::FromUint(name_id_mapper_->NameToId(name)); }
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(std::string_view name);
PropertyId NameToProperty(const std::string_view name) const {
return PropertyId::FromUint(name_id_mapper_->NameToId(name));
}
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(std::string_view name);
EdgeTypeId NameToEdgeType(const std::string_view name) const {
return EdgeTypeId::FromUint(name_id_mapper_->NameToId(name));
}
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(std::string_view name);
void SetStorageMode(StorageMode storage_mode);
/// 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 = {});
StorageMode GetStorageMode() const;
/// 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 = {});
virtual void FreeMemory(std::unique_lock<utils::RWLock> main_guard) = 0;
void FreeMemory() { FreeMemory({}); }
/// 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 = {});
virtual std::unique_ptr<Accessor> Access(std::optional<IsolationLevel> override_isolation_level) = 0;
std::unique_ptr<Accessor> Access() { return Access(std::optional<IsolationLevel>{}); }
/// 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 = {});
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;
/// 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 = {});
virtual utils::BasicResult<StorageExistenceConstraintDefinitionError, void> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
/// 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 = {});
virtual utils::BasicResult<StorageExistenceConstraintDroppingError, void> DropExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp) = 0;
/// 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 = {});
virtual utils::BasicResult<StorageUniqueConstraintDefinitionError, UniqueConstraints::CreationStatus>
CreateUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp) = 0;
/// 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 = {});
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;
StorageInfo GetInfo() const;
utils::FileRetainer::FileLockerAccessor::ret_type IsPathLocked();
utils::FileRetainer::FileLockerAccessor::ret_type LockPath();
utils::FileRetainer::FileLockerAccessor::ret_type UnlockPath();
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
bool SetMainReplicationRole();
enum class RegisterReplicaError : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
CONNECTION_FAILED,
COULD_NOT_BE_PERSISTED
};
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config = {});
/// @pre The instance should have a MAIN role
bool UnregisterReplica(const std::string &name);
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
replication::ReplicationRole GetReplicationRole() const;
struct TimestampInfo {
uint64_t current_timestamp_of_replica;
uint64_t current_number_of_timestamp_behind_master;
};
struct ReplicaInfo {
std::string name;
replication::ReplicationMode mode;
io::network::Endpoint endpoint;
replication::ReplicaState state;
TimestampInfo timestamp_info;
};
std::vector<ReplicaInfo> ReplicasInfo();
void FreeMemory(std::unique_lock<utils::RWLock> main_guard = {});
enum class SetIsolationLevelError : uint8_t { DisabledForAnalyticalMode };
utils::BasicResult<SetIsolationLevelError> SetIsolationLevel(IsolationLevel isolation_level);
IsolationLevel GetIsolationLevel() const noexcept;
void SetStorageMode(StorageMode storage_mode);
virtual StorageInfo GetInfo() const = 0;
StorageMode GetStorageMode();
enum class CreateSnapshotError : uint8_t {
DisabledForReplica,
DisabledForAnalyticsPeriodicCommit,
ReachedMaxNumTries
};
utils::BasicResult<CreateSnapshotError> CreateSnapshot(std::optional<bool> is_periodic);
private:
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode);
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage(std::unique_lock<utils::RWLock> main_guard = {});
bool InitializeWalFile();
void FinalizeWalFile();
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataManipulation(const Transaction &transaction, uint64_t final_commit_timestamp);
/// Return true in all cases excepted if any sync replicas have not sent confirmation.
[[nodiscard]] bool AppendToWalDataDefinition(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
void RestoreReplicationRole();
void RestoreReplicas();
bool ShouldStoreAndRestoreReplicationState() const;
virtual Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode) = 0;
// Main storage lock.
//
// Accessors take a shared lock when starting, so it is possible to block
// creation of new accessors by taking a unique lock. This is used when doing
// operations on storage that affect the global state, for example index
// creation.
mutable utils::RWLock main_lock_{utils::RWLock::Priority::WRITE};
// Main object storage
utils::SkipList<storage::Vertex> vertices_;
utils::SkipList<storage::Edge> edges_;
std::atomic<uint64_t> vertex_id_{0};
std::atomic<uint64_t> edge_id_{0};
// Even though the edge count is already kept in the `edges_` SkipList, the
// list is used only when properties are enabled for edges. Because of that we
// keep a separate count of edges that is always updated.
std::atomic<uint64_t> edge_count_{0};
NameIdMapper name_id_mapper_;
Constraints constraints_;
Indices indices_;
std::unique_ptr<NameIdMapper> name_id_mapper_;
Config config_;
// Transaction engine
utils::SpinLock engine_lock_;
uint64_t timestamp_{kTimestampInitialId};
uint64_t transaction_id_{kTransactionInitialId};
// TODO: This isn't really a commit log, it doesn't even care if a
// transaction commited or aborted. We could probably combine this with
// `timestamp_` in a sensible unit, something like TransactionClock or
// whatever.
std::optional<CommitLog> commit_log_;
utils::Synchronized<std::list<Transaction>, utils::SpinLock> committed_transactions_;
IsolationLevel isolation_level_;
StorageMode storage_mode_;
Config config_;
utils::Scheduler gc_runner_;
std::mutex gc_lock_;
Indices indices_;
Constraints constraints_;
// Undo buffers that were unlinked and now are waiting to be freed.
utils::Synchronized<std::list<std::pair<uint64_t, std::list<Delta>>>, utils::SpinLock> garbage_undo_buffers_;
// Vertices that are logically deleted but still have to be removed from
// indices before removing them from the main storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_vertices_;
// Vertices that are logically deleted and removed from indices and now wait
// to be removed from the main storage.
std::list<std::pair<uint64_t, Gid>> garbage_vertices_;
// Edges that are logically deleted and wait to be removed from the main
// storage.
utils::Synchronized<std::list<Gid>, utils::SpinLock> deleted_edges_;
// Flags to inform CollectGarbage that it needs to do the more expensive full scans
std::atomic<bool> gc_full_scan_vertices_delete_ = false;
std::atomic<bool> gc_full_scan_edges_delete_ = false;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path wal_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
std::unique_ptr<kvstore::KVStore> storage_;
utils::Scheduler snapshot_runner_;
utils::SpinLock snapshot_lock_;
// UUID used to distinguish snapshots and to link snapshots to WALs
std::string uuid_;
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_;
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
utils::FileRetainer file_retainer_;
// Global locker that is used for clients file locking
utils::FileRetainer::FileLocker global_locker_;
// Last commited timestamp
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
class ReplicationServer;
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
class ReplicationClient;
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<replication::ReplicationRole> replication_role_{replication::ReplicationRole::MAIN};
std::atomic<uint64_t> vertex_id_{0};
std::atomic<uint64_t> edge_id_{0};
};
} // namespace memgraph::storage

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
@ -11,28 +11,38 @@
#pragma once
#include "storage/v2/constraints.hpp"
#include "storage/v2/constraints/constraints.hpp"
#include <iterator>
#include <variant>
namespace memgraph::storage {
struct ReplicationError {};
using StorageDataManipulationError = std::variant<ConstraintViolation, ReplicationError>;
struct IndexPersistenceError {};
struct ConstraintsPersistenceError {};
struct SerializationError {};
inline bool operator==(const SerializationError & /*err1*/, const SerializationError & /*err2*/) { return true; }
using StorageDataManipulationError = std::variant<ConstraintViolation, ReplicationError, SerializationError>;
struct IndexDefinitionError {};
using StorageIndexDefinitionError = std::variant<IndexDefinitionError, ReplicationError>;
using StorageIndexDefinitionError = std::variant<IndexDefinitionError, ReplicationError, IndexPersistenceError>;
struct ConstraintDefinitionError {};
using StorageExistenceConstraintDefinitionError =
std::variant<ConstraintViolation, ConstraintDefinitionError, ReplicationError>;
std::variant<ConstraintViolation, ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageExistenceConstraintDroppingError = std::variant<ConstraintDefinitionError, ReplicationError>;
using StorageExistenceConstraintDroppingError =
std::variant<ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageUniqueConstraintDefinitionError = std::variant<ConstraintViolation, ReplicationError>;
using StorageUniqueConstraintDefinitionError =
std::variant<ConstraintViolation, ConstraintDefinitionError, ReplicationError, ConstraintsPersistenceError>;
using StorageUniqueConstraintDroppingError = std::variant<ReplicationError>;
using StorageUniqueConstraintDroppingError = std::variant<ReplicationError, ConstraintsPersistenceError>;
} // namespace memgraph::storage

View File

@ -19,6 +19,8 @@ std::string_view StorageModeToString(memgraph::storage::StorageMode storage_mode
return "IN_MEMORY_ANALYTICAL";
case memgraph::storage::StorageMode::IN_MEMORY_TRANSACTIONAL:
return "IN_MEMORY_TRANSACTIONAL";
case memgraph::storage::StorageMode::ON_DISK_TRANSACTIONAL:
return "ON_DISK_TRANSACTIONAL";
}
}

View File

@ -16,7 +16,7 @@
namespace memgraph::storage {
enum class StorageMode : std::uint8_t { IN_MEMORY_ANALYTICAL, IN_MEMORY_TRANSACTIONAL };
enum class StorageMode : std::uint8_t { IN_MEMORY_ANALYTICAL, IN_MEMORY_TRANSACTIONAL, ON_DISK_TRANSACTIONAL };
std::string_view StorageModeToString(memgraph::storage::StorageMode storage_mode);

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
@ -25,7 +25,8 @@ namespace memgraph::storage {
struct Vertex {
Vertex(Gid gid, Delta *delta) : gid(gid), deleted(false), delta(delta) {
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT,
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT ||
delta->action == Delta::Action::DELETE_DESERIALIZED_OBJECT,
"Vertex must be created with an initial DELETE_OBJECT delta!");
}

View File

@ -15,7 +15,7 @@
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/indices.hpp"
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/property_value.hpp"
#include "utils/logging.hpp"
@ -48,6 +48,7 @@ std::pair<bool, bool> IsVisible(Vertex *vertex, Transaction *transaction, View v
deleted = false;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -79,34 +80,37 @@ Result<bool> VertexAccessor::AddLabel(LabelId label) {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
if (std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::RemoveLabelTag(), label);
vertex_->labels.push_back(label);
UpdateOnAddLabel(indices_, label, vertex_, *transaction_);
/// TODO: some by pointers, some by reference => not good, make it better
constraints_->unique_constraints_->UpdateOnAddLabel(label, *vertex_, transaction_->start_timestamp);
indices_->UpdateOnAddLabel(label, vertex_, *transaction_);
return true;
}
/// TODO: move to after update and change naming to vertex after update
Result<bool> VertexAccessor::RemoveLabel(LabelId label) {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto it = std::find(vertex_->labels.begin(), vertex_->labels.end(), label);
if (it == vertex_->labels.end()) return false;
CreateAndLinkDelta(transaction_, vertex_, Delta::AddLabelTag(), label);
std::swap(*it, *vertex_->labels.rbegin());
vertex_->labels.pop_back();
/// TODO: some by pointers, some by reference => not good, make it better
constraints_->unique_constraints_->UpdateOnRemoveLabel(label, *vertex_, transaction_->start_timestamp);
indices_->UpdateOnRemoveLabel(label, vertex_, *transaction_);
return true;
}
@ -137,6 +141,7 @@ Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -186,6 +191,7 @@ Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
labels.push_back(delta.label);
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -226,7 +232,7 @@ Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const Pro
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
vertex_->properties.SetProperty(property, value);
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
indices_->UpdateOnSetProperty(property, value, vertex_, *transaction_);
return std::move(current_value);
}
@ -242,7 +248,7 @@ Result<bool> VertexAccessor::InitProperties(const std::map<storage::PropertyId,
if (!vertex_->properties.InitProperties(properties)) return false;
for (const auto &[property, value] : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, PropertyValue());
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
indices_->UpdateOnSetProperty(property, value, vertex_, *transaction_);
}
return true;
@ -258,7 +264,7 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::ClearProperties() {
auto properties = vertex_->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property.first, property.second);
UpdateOnSetProperty(indices_, property.first, PropertyValue(), vertex_, *transaction_);
indices_->UpdateOnSetProperty(property.first, PropertyValue(), vertex_, *transaction_);
}
vertex_->properties.ClearProperties();
@ -285,6 +291,7 @@ Result<PropertyValue> VertexAccessor::GetProperty(PropertyId property, View view
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -335,6 +342,7 @@ Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view
}
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -410,6 +418,7 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::
in_edges.pop_back();
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -490,6 +499,7 @@ Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std:
out_edges.pop_back();
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
@ -536,6 +546,7 @@ Result<size_t> VertexAccessor::InDegree(View view) const {
case Delta::Action::REMOVE_IN_EDGE:
--degree;
break;
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
exists = false;
break;
@ -574,6 +585,7 @@ Result<size_t> VertexAccessor::OutDegree(View view) const {
case Delta::Action::REMOVE_OUT_EDGE:
--degree;
break;
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT:
exists = false;
break;

View File

@ -24,8 +24,8 @@ namespace memgraph::storage {
class EdgeAccessor;
class Storage;
struct Indices;
struct Constraints;
struct Indices;
class VertexAccessor final {
private:
@ -106,7 +106,6 @@ class VertexAccessor final {
}
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
private:
Vertex *vertex_;
Transaction *transaction_;
Indices *indices_;

View File

@ -0,0 +1,250 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/vertices_iterable.hpp"
namespace memgraph::storage {
VerticesIterable::VerticesIterable(AllVerticesIterable vertices) : type_(Type::ALL) {
new (&all_vertices_) AllVerticesIterable(std::move(vertices));
}
VerticesIterable::VerticesIterable(InMemoryLabelIndex::Iterable vertices) : type_(Type::BY_LABEL_IN_MEMORY) {
new (&in_memory_vertices_by_label_) InMemoryLabelIndex::Iterable(std::move(vertices));
}
VerticesIterable::VerticesIterable(InMemoryLabelPropertyIndex::Iterable vertices)
: type_(Type::BY_LABEL_PROPERTY_IN_MEMORY) {
new (&in_memory_vertices_by_label_property_) InMemoryLabelPropertyIndex::Iterable(std::move(vertices));
}
VerticesIterable::VerticesIterable(VerticesIterable &&other) noexcept : type_(other.type_) {
switch (other.type_) {
case Type::ALL:
new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
break;
case Type::BY_LABEL_IN_MEMORY:
new (&in_memory_vertices_by_label_) InMemoryLabelIndex::Iterable(std::move(other.in_memory_vertices_by_label_));
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_vertices_by_label_property_)
InMemoryLabelPropertyIndex::Iterable(std::move(other.in_memory_vertices_by_label_property_));
break;
}
}
VerticesIterable &VerticesIterable::operator=(VerticesIterable &&other) noexcept {
switch (type_) {
case Type::ALL:
all_vertices_.AllVerticesIterable::~AllVerticesIterable();
break;
case Type::BY_LABEL_IN_MEMORY:
in_memory_vertices_by_label_.InMemoryLabelIndex::Iterable::~Iterable();
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
in_memory_vertices_by_label_property_.InMemoryLabelPropertyIndex::Iterable::~Iterable();
break;
}
type_ = other.type_;
switch (other.type_) {
case Type::ALL:
new (&all_vertices_) AllVerticesIterable(std::move(other.all_vertices_));
break;
case Type::BY_LABEL_IN_MEMORY:
new (&in_memory_vertices_by_label_) InMemoryLabelIndex::Iterable(std::move(other.in_memory_vertices_by_label_));
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_vertices_by_label_property_)
InMemoryLabelPropertyIndex::Iterable(std::move(other.in_memory_vertices_by_label_property_));
break;
}
return *this;
}
VerticesIterable::~VerticesIterable() {
switch (type_) {
case Type::ALL:
all_vertices_.AllVerticesIterable::~AllVerticesIterable();
break;
case Type::BY_LABEL_IN_MEMORY:
in_memory_vertices_by_label_.InMemoryLabelIndex::Iterable::~Iterable();
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
in_memory_vertices_by_label_property_.InMemoryLabelPropertyIndex::Iterable::~Iterable();
break;
}
}
VerticesIterable::Iterator VerticesIterable::begin() {
switch (type_) {
case Type::ALL:
return Iterator(all_vertices_.begin());
case Type::BY_LABEL_IN_MEMORY:
return Iterator(in_memory_vertices_by_label_.begin());
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
return Iterator(in_memory_vertices_by_label_property_.begin());
}
}
VerticesIterable::Iterator VerticesIterable::end() {
switch (type_) {
case Type::ALL:
return Iterator(all_vertices_.end());
case Type::BY_LABEL_IN_MEMORY:
return Iterator(in_memory_vertices_by_label_.end());
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
return Iterator(in_memory_vertices_by_label_property_.end());
}
}
VerticesIterable::Iterator::Iterator(AllVerticesIterable::Iterator it) : type_(Type::ALL) {
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&all_it_) AllVerticesIterable::Iterator(std::move(it));
}
VerticesIterable::Iterator::Iterator(InMemoryLabelIndex::Iterable::Iterator it) : type_(Type::BY_LABEL_IN_MEMORY) {
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&in_memory_by_label_it_) InMemoryLabelIndex::Iterable::Iterator(std::move(it));
}
VerticesIterable::Iterator::Iterator(InMemoryLabelPropertyIndex::Iterable::Iterator it)
: type_(Type::BY_LABEL_PROPERTY_IN_MEMORY) {
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&in_memory_by_label_property_it_) InMemoryLabelPropertyIndex::Iterable::Iterator(std::move(it));
}
VerticesIterable::Iterator::Iterator(const VerticesIterable::Iterator &other) : type_(other.type_) {
switch (other.type_) {
case Type::ALL:
new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
break;
case Type::BY_LABEL_IN_MEMORY:
new (&in_memory_by_label_it_) InMemoryLabelIndex::Iterable::Iterator(other.in_memory_by_label_it_);
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_by_label_property_it_)
InMemoryLabelPropertyIndex::Iterable::Iterator(other.in_memory_by_label_property_it_);
break;
}
}
// NOLINTNEXTLINE(cert-oop54-cpp)
VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(const VerticesIterable::Iterator &other) {
Destroy();
type_ = other.type_;
switch (other.type_) {
case Type::ALL:
new (&all_it_) AllVerticesIterable::Iterator(other.all_it_);
break;
case Type::BY_LABEL_IN_MEMORY:
new (&in_memory_by_label_it_) InMemoryLabelIndex::Iterable::Iterator(other.in_memory_by_label_it_);
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_by_label_property_it_)
InMemoryLabelPropertyIndex::Iterable::Iterator(other.in_memory_by_label_property_it_);
break;
}
return *this;
}
VerticesIterable::Iterator::Iterator(VerticesIterable::Iterator &&other) noexcept : type_(other.type_) {
switch (other.type_) {
case Type::ALL:
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&all_it_) AllVerticesIterable::Iterator(std::move(other.all_it_));
break;
case Type::BY_LABEL_IN_MEMORY:
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&in_memory_by_label_it_) InMemoryLabelIndex::Iterable::Iterator(std::move(other.in_memory_by_label_it_));
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_by_label_property_it_)
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
InMemoryLabelPropertyIndex::Iterable::Iterator(std::move(other.in_memory_by_label_property_it_));
break;
}
}
VerticesIterable::Iterator &VerticesIterable::Iterator::operator=(VerticesIterable::Iterator &&other) noexcept {
Destroy();
type_ = other.type_;
switch (other.type_) {
case Type::ALL:
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&all_it_) AllVerticesIterable::Iterator(std::move(other.all_it_));
break;
case Type::BY_LABEL_IN_MEMORY:
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
new (&in_memory_by_label_it_) InMemoryLabelIndex::Iterable::Iterator(std::move(other.in_memory_by_label_it_));
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
new (&in_memory_by_label_property_it_)
// NOLINTNEXTLINE(hicpp-move-const-arg,performance-move-const-arg)
InMemoryLabelPropertyIndex::Iterable::Iterator(std::move(other.in_memory_by_label_property_it_));
break;
}
return *this;
}
VerticesIterable::Iterator::~Iterator() { Destroy(); }
void VerticesIterable::Iterator::Destroy() noexcept {
switch (type_) {
case Type::ALL:
all_it_.AllVerticesIterable::Iterator::~Iterator();
break;
case Type::BY_LABEL_IN_MEMORY:
in_memory_by_label_it_.InMemoryLabelIndex::Iterable::Iterator::~Iterator();
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
in_memory_by_label_property_it_.InMemoryLabelPropertyIndex::Iterable::Iterator::~Iterator();
break;
}
}
VertexAccessor VerticesIterable::Iterator::operator*() const {
switch (type_) {
case Type::ALL:
return *all_it_;
case Type::BY_LABEL_IN_MEMORY:
return *in_memory_by_label_it_;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
return *in_memory_by_label_property_it_;
}
}
VerticesIterable::Iterator &VerticesIterable::Iterator::operator++() {
switch (type_) {
case Type::ALL:
++all_it_;
break;
case Type::BY_LABEL_IN_MEMORY:
++in_memory_by_label_it_;
break;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
++in_memory_by_label_property_it_;
break;
}
return *this;
}
bool VerticesIterable::Iterator::operator==(const Iterator &other) const {
switch (type_) {
case Type::ALL:
return all_it_ == other.all_it_;
case Type::BY_LABEL_IN_MEMORY:
return in_memory_by_label_it_ == other.in_memory_by_label_it_;
case Type::BY_LABEL_PROPERTY_IN_MEMORY:
return in_memory_by_label_property_it_ == other.in_memory_by_label_property_it_;
}
}
} // namespace memgraph::storage

View File

@ -0,0 +1,78 @@
// 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 "storage/v2/all_vertices_iterable.hpp"
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
namespace memgraph::storage {
class VerticesIterable final {
enum class Type { ALL, BY_LABEL_IN_MEMORY, BY_LABEL_PROPERTY_IN_MEMORY };
Type type_;
union {
AllVerticesIterable all_vertices_;
InMemoryLabelIndex::Iterable in_memory_vertices_by_label_;
InMemoryLabelPropertyIndex::Iterable in_memory_vertices_by_label_property_;
};
public:
explicit VerticesIterable(AllVerticesIterable);
explicit VerticesIterable(InMemoryLabelIndex::Iterable);
explicit VerticesIterable(InMemoryLabelPropertyIndex::Iterable);
VerticesIterable(const VerticesIterable &) = delete;
VerticesIterable &operator=(const VerticesIterable &) = delete;
VerticesIterable(VerticesIterable &&) noexcept;
VerticesIterable &operator=(VerticesIterable &&) noexcept;
~VerticesIterable();
class Iterator final {
Type type_;
union {
AllVerticesIterable::Iterator all_it_;
InMemoryLabelIndex::Iterable::Iterator in_memory_by_label_it_;
InMemoryLabelPropertyIndex::Iterable::Iterator in_memory_by_label_property_it_;
};
void Destroy() noexcept;
public:
explicit Iterator(AllVerticesIterable::Iterator);
explicit Iterator(InMemoryLabelIndex::Iterable::Iterator);
explicit Iterator(InMemoryLabelPropertyIndex::Iterable::Iterator);
Iterator(const Iterator &);
Iterator &operator=(const Iterator &);
Iterator(Iterator &&) noexcept;
Iterator &operator=(Iterator &&) noexcept;
~Iterator();
VertexAccessor operator*() const;
Iterator &operator++();
bool operator==(const Iterator &other) const;
bool operator!=(const Iterator &other) const { return !(*this == other); }
};
Iterator begin();
Iterator end();
};
} // namespace memgraph::storage

28
src/utils/disk_utils.hpp Normal file
View File

@ -0,0 +1,28 @@
// 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 "storage/v2/delta.hpp"
namespace memgraph::utils {
inline std::optional<std::string> GetOldDiskKeyOrNull(storage::Delta *head) {
while (head->next != nullptr) {
head = head->next;
}
if (head->action == storage::Delta::Action::DELETE_DESERIALIZED_OBJECT) {
return head->old_disk_key;
}
return std::nullopt;
}
} // 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
@ -75,4 +75,15 @@ void Fatal(const char *msg, const Args &...msg_args) {
#endif
inline void RedirectToStderr() { spdlog::set_default_logger(spdlog::stderr_color_mt("stderr")); }
// /// Use it for operations that must successfully finish.
inline void AssertRocksDBStatus(const auto &status) { MG_ASSERT(status.ok(), "rocksdb: {}", status.ToString()); }
inline bool CheckRocksDBStatus(const auto &status) {
if (!status.ok()) [[unlikely]] {
spdlog::error("rocksdb: {}", status.ToString());
}
return status.ok();
}
} // namespace memgraph::logging

View File

@ -81,11 +81,9 @@ bool LessThanDecimal(T a, T b) {
return (b - a) > std::numeric_limits<T>::epsilon();
}
/*
* return 0 if a == b
* return 1 if a > b
* return -1 if a < b
*/
/// @return 0 if a == b
/// @return 1 if a > b
/// @return -1 if a < b
template <FloatingPoint T>
int CompareDecimal(T a, T b) {
if (ApproxEqualDecimal(a, b)) return 0;

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

View File

@ -0,0 +1,291 @@
// 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 <cstdint>
#include <iomanip>
#include <iterator>
#include <numeric>
#include <string>
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_store.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/exceptions.hpp"
#include "utils/string.hpp"
namespace memgraph::utils {
static constexpr const char *outEdgeDirection = "0";
static constexpr const char *inEdgeDirection = "1";
/// TODO: try to move this to hpp files so that we can follow jump on readings
inline std::string SerializeIdType(const auto &id) { return std::to_string(id.AsUint()); }
inline bool SerializedVertexHasLabels(const std::string &labels) { return !labels.empty(); }
template <typename Collection>
inline std::vector<std::string> TransformIDsToString(const Collection &labels) {
std::vector<std::string> transformed_labels{};
std::transform(labels.begin(), labels.end(), std::back_inserter(transformed_labels),
[](const auto &label) { return SerializeIdType(label); });
return transformed_labels;
}
inline std::vector<storage::LabelId> TransformFromStringLabels(const std::vector<std::string> &labels) {
std::vector<storage::LabelId> transformed_labels;
std::transform(labels.begin(), labels.end(), std::back_inserter(transformed_labels),
[](const auto &label) { return storage::LabelId::FromUint(std::stoull(label)); });
return transformed_labels;
}
inline std::string SerializeLabels(const std::vector<std::string> &labels) {
if (labels.empty()) {
return "";
}
std::string result = labels[0];
std::string ser_labels =
std::accumulate(std::next(labels.begin()), labels.end(), result,
[](const std::string &join, const auto &label_id) { return join + "," + label_id; });
return ser_labels;
}
inline std::string SerializeProperties(const storage::PropertyStore &properties) { return properties.StringBuffer(); }
/// TODO: andi Probably it is better to add delimiter between label,property and the rest of labels
/// TODO: reuse PutIndexingLabelAndPropertiesFirst
inline std::string PutIndexingLabelAndPropertyFirst(const std::string &indexing_label,
const std::string &indexing_property,
const std::vector<std::string> &vertex_labels) {
std::string result = indexing_label + "," + indexing_property;
for (const auto &label : vertex_labels) {
if (label != indexing_label) {
result += "," + label;
}
}
return result;
}
inline std::string PutIndexingLabelAndPropertiesFirst(const std::string &target_label,
const std::vector<std::string> &target_properties) {
std::string result = target_label;
for (const auto &target_property : target_properties) {
result += "," + target_property;
}
return result;
}
inline std::string SerializeVertexAsValueForAuxiliaryStorages(storage::LabelId label_to_remove,
const std::vector<storage::LabelId> &vertex_labels,
const storage::PropertyStore &property_store) {
std::vector<storage::LabelId> labels_without_target;
std::copy_if(vertex_labels.begin(), vertex_labels.end(), std::back_inserter(labels_without_target),
[&label_to_remove](const auto &label) { return label_to_remove != label; });
std::string result = SerializeLabels(TransformIDsToString(vertex_labels)) + "|";
return result + SerializeProperties(property_store);
}
inline std::string ExtractGidFromKey(const std::string &key) {
std::vector<std::string> key_vector = utils::Split(key, "|");
return key_vector[1];
}
inline storage::PropertyStore DeserializePropertiesFromAuxiliaryStorages(const std::string &value) {
std::vector<std::string> value_vector = utils::Split(value, "|");
std::string properties_str = value_vector[1];
return storage::PropertyStore::CreateFromBuffer(properties_str);
}
inline std::string SerializeVertex(const storage::Vertex &vertex) {
std::string result = utils::SerializeLabels(TransformIDsToString(vertex.labels)) + "|";
result += utils::SerializeIdType(vertex.gid);
return result;
}
inline std::vector<storage::LabelId> DeserializeLabelsFromMainDiskStorage(const std::string &key) {
std::vector<std::string> key_vector = utils::Split(key, "|");
std::string labels_str = key_vector[0];
if (SerializedVertexHasLabels(labels_str)) {
return TransformFromStringLabels(utils::Split(labels_str, ","));
}
return {};
}
inline std::vector<std::string> ExtractLabelsFromMainDiskStorage(const std::string &key) {
std::vector<std::string> key_vector = utils::Split(key, "|");
std::string labels_str = key_vector[0];
return utils::Split(labels_str, ",");
}
inline storage::PropertyStore DeserializePropertiesFromMainDiskStorage(const std::string_view value) {
return storage::PropertyStore::CreateFromBuffer(value);
}
inline std::string ExtractGidFromMainDiskStorage(const std::string &key) { return ExtractGidFromKey(key); }
inline std::string ExtractGidFromUniqueConstraintStorage(const std::string &key) { return ExtractGidFromKey(key); }
/// Serialize vertex to string as a key in unique constraint index KV store.
/// target_label, target_property_1, target_property_2, ... GID |
/// commit_timestamp
inline std::string SerializeVertexAsKeyForUniqueConstraint(const storage::LabelId &constraint_label,
const std::set<storage::PropertyId> &constraint_properties,
const std::string &gid) {
auto key_for_indexing = PutIndexingLabelAndPropertiesFirst(SerializeIdType(constraint_label),
TransformIDsToString(constraint_properties));
return key_for_indexing + "|" + gid;
}
inline std::string SerializeVertexAsValueForUniqueConstraint(const storage::LabelId &constraint_label,
const std::vector<storage::LabelId> &vertex_labels,
const storage::PropertyStore &property_store) {
return SerializeVertexAsValueForAuxiliaryStorages(constraint_label, vertex_labels, property_store);
}
inline storage::LabelId DeserializeConstraintLabelFromUniqueConstraintStorage(const std::string &key) {
std::vector<std::string> key_vector = utils::Split(key, "|");
std::vector<std::string> constraint_key = utils::Split(key_vector[0], ",");
/// TODO: andi Change this to deserialization method directly into the LabelId class
return storage::LabelId::FromUint(std::stoull(constraint_key[0]));
}
inline storage::PropertyStore DeserializePropertiesFromUniqueConstraintStorage(const std::string &value) {
return DeserializePropertiesFromAuxiliaryStorages(value);
}
inline std::string SerializeVertexAsKeyForLabelIndex(const std::string &indexing_label, const std::string &gid) {
return indexing_label + "|" + gid;
}
inline std::string SerializeVertexAsKeyForLabelIndex(storage::LabelId label, storage::Gid gid) {
return SerializeVertexAsKeyForLabelIndex(SerializeIdType(label), utils::SerializeIdType(gid));
}
inline std::string ExtractGidFromLabelIndexStorage(const std::string &key) { return ExtractGidFromKey(key); }
inline std::string SerializeVertexAsValueForLabelIndex(storage::LabelId indexing_label,
const std::vector<storage::LabelId> &vertex_labels,
const storage::PropertyStore &property_store) {
return SerializeVertexAsValueForAuxiliaryStorages(indexing_label, vertex_labels, property_store);
}
inline std::vector<storage::LabelId> DeserializeLabelsFromLabelIndexStorage(const std::string &value) {
const auto value_splitted = utils::Split(value, "|");
return TransformFromStringLabels(utils::Split(value_splitted[0], ","));
}
inline storage::PropertyStore DeserializePropertiesFromLabelIndexStorage(const std::string &value) {
return DeserializePropertiesFromAuxiliaryStorages(value);
}
inline std::string SerializeVertexAsKeyForLabelPropertyIndex(const std::string &indexing_label,
const std::string &indexing_property,
const std::string &gid) {
return indexing_label + "|" + indexing_property + "|" + gid;
}
inline std::string SerializeVertexAsKeyForLabelPropertyIndex(storage::LabelId label, storage::PropertyId property,
storage::Gid gid) {
return SerializeVertexAsKeyForLabelPropertyIndex(SerializeIdType(label), SerializeIdType(property),
utils::SerializeIdType(gid));
}
inline std::string SerializeVertexAsValueForLabelPropertyIndex(storage::LabelId indexing_label,
const std::vector<storage::LabelId> &vertex_labels,
const storage::PropertyStore &property_store) {
return SerializeVertexAsValueForAuxiliaryStorages(indexing_label, vertex_labels, property_store);
}
inline std::string ExtractGidFromLabelPropertyIndexStorage(const std::string &key) {
std::vector<std::string> key_vector = utils::Split(key, "|");
return key_vector[2];
}
/// TODO: refactor into one method with label index storage
inline std::vector<storage::LabelId> DeserializeLabelsFromLabelPropertyIndexStorage(const std::string &value) {
const auto value_splitted = utils::Split(value, "|");
return TransformFromStringLabels(utils::Split(value_splitted[0], ","));
}
inline storage::PropertyStore DeserializePropertiesFromLabelPropertyIndexStorage(const std::string &value) {
return DeserializePropertiesFromAuxiliaryStorages(value);
}
/// Serialize edge as two KV entries
/// vertex_gid_1 | vertex_gid_2 | direction | edge_type | GID | commit_timestamp
inline std::string SerializeEdge(storage::EdgeAccessor *edge_acc) {
// Serialized objects
auto from_gid = utils::SerializeIdType(edge_acc->FromVertex().Gid());
auto to_gid = utils::SerializeIdType(edge_acc->ToVertex().Gid());
auto edge_type = utils::SerializeIdType(edge_acc->EdgeType());
auto edge_gid = utils::SerializeIdType(edge_acc->Gid());
// source->destination key
std::string src_dest_key = from_gid + "|";
src_dest_key += to_gid + "|";
src_dest_key += outEdgeDirection;
src_dest_key += "|" + edge_type + "|";
src_dest_key += edge_gid;
return src_dest_key;
}
/// Serialize edge as two KV entries
/// vertex_gid_1 | vertex_gid_2 | direction | edge_type | GID | commit_timestamp
/// @tparam src_vertex_gid, dest_vertex_gid: Gid of the source and destination vertices
/// @tparam edge: Edge to be serialized
/// @tparam edge_type_id: EdgeTypeId of the edge
inline std::string SerializeEdge(storage::Gid src_vertex_gid, storage::Gid dest_vertex_gid,
storage::EdgeTypeId edge_type_id, const storage::EdgeRef edge_ref,
bool properties_on_edges) {
// Serialized objects
auto from_gid = utils::SerializeIdType(src_vertex_gid);
auto to_gid = utils::SerializeIdType(dest_vertex_gid);
auto edge_type = utils::SerializeIdType(edge_type_id);
std::string edge_gid;
if (properties_on_edges) {
edge_gid = utils::SerializeIdType(edge_ref.ptr->gid);
} else {
edge_gid = utils::SerializeIdType(edge_ref.gid);
}
// source->destination key
std::string src_dest_key = from_gid + "|";
src_dest_key += to_gid + "|";
src_dest_key += outEdgeDirection;
src_dest_key += "|" + edge_type + "|";
src_dest_key += edge_gid;
return src_dest_key;
}
/// TODO: (andi): This can potentially be a problem on big-endian machines.
inline void PutFixed64(std::string *dst, uint64_t value) {
// NOLINTNEXTLINE(cppcoreguidelines-pro-type-const-cast)
dst->append(const_cast<const char *>(reinterpret_cast<char *>(&value)), sizeof(value));
}
inline uint64_t DecodeFixed64(const char *ptr) {
// Load the raw bytes
uint64_t result = 0;
memcpy(&result, ptr, sizeof(result)); // gcc optimizes this to a plain load
return result;
}
inline std::string StringTimestamp(uint64_t ts) {
std::string ret;
PutFixed64(&ret, ts);
return ret;
}
} // namespace memgraph::utils

View File

@ -21,13 +21,17 @@
#include <random>
#include <utility>
#include "spdlog/spdlog.h"
#include "utils/bound.hpp"
#include "utils/linux.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
#include "utils/memory_tracker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/readable_size.hpp"
#include "utils/spin_lock.hpp"
#include "utils/stack.hpp"
#include "utils/stat.hpp"
// This code heavily depends on atomic operations. For a more detailed
// description of how exactly atomic operations work, see:
@ -345,9 +349,6 @@ class SkipListGc final {
MemoryResource *GetMemoryResource() const { return memory_; }
void Clear() {
#ifndef NDEBUG
MG_ASSERT(alive_accessors_ == 0, "The SkipList can't be cleared while there are existing accessors!");
#endif
// Delete all allocated blocks.
Block *head = head_.load(std::memory_order_acquire);
while (head != nullptr) {
@ -890,7 +891,7 @@ class SkipList final {
MemoryResource *GetMemoryResource() const { return gc_.GetMemoryResource(); }
/// This function removes all elements from the list.
/// NOTE: The function *isn't* thread-safe. It must be called while there are
/// NOTE: The function *isn't* thread-safe. It must be called only if there are
/// no more active accessors using the list.
void clear() {
TNode *curr = head_->nexts[0].load(std::memory_order_acquire);

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
@ -15,46 +15,44 @@
#include "query/config.hpp"
#include "query/interpreter.hpp"
#include "query/typed_value.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/isolation_level.hpp"
#include "storage/v2/storage.hpp"
class ExpansionBenchFixture : public benchmark::Fixture {
protected:
std::optional<memgraph::storage::Storage> db;
std::optional<memgraph::query::InterpreterContext> interpreter_context;
std::optional<memgraph::query::Interpreter> interpreter;
std::filesystem::path data_directory{std::filesystem::temp_directory_path() / "expansion-benchmark"};
void SetUp(const benchmark::State &state) override {
db.emplace();
interpreter_context.emplace(memgraph::storage::Config{}, memgraph::query::InterpreterConfig{}, data_directory);
auto *db = interpreter_context->db.get();
auto label = db->NameToLabel("Starting");
{
auto dba = db->Access();
for (int i = 0; i < state.range(0); i++) dba.CreateVertex();
for (int i = 0; i < state.range(0); i++) dba->CreateVertex();
// the fixed part is one vertex expanding to 1000 others
auto start = dba.CreateVertex();
auto start = dba->CreateVertex();
MG_ASSERT(start.AddLabel(label).HasValue());
auto edge_type = dba.NameToEdgeType("edge_type");
auto edge_type = dba->NameToEdgeType("edge_type");
for (int i = 0; i < 1000; i++) {
auto dest = dba.CreateVertex();
MG_ASSERT(dba.CreateEdge(&start, &dest, edge_type).HasValue());
auto dest = dba->CreateVertex();
MG_ASSERT(dba->CreateEdge(&start, &dest, edge_type).HasValue());
}
MG_ASSERT(!dba.Commit().HasError());
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db->CreateIndex(label).HasError());
interpreter_context.emplace(&*db, memgraph::query::InterpreterConfig{}, data_directory);
interpreter.emplace(&*interpreter_context);
}
void TearDown(const benchmark::State &) override {
interpreter = std::nullopt;
interpreter_context = std::nullopt;
db = std::nullopt;
std::filesystem::remove_all(data_directory);
}
};
@ -63,7 +61,7 @@ BENCHMARK_DEFINE_F(ExpansionBenchFixture, Match)(benchmark::State &state) {
auto query = "MATCH (s:Starting) return s";
while (state.KeepRunning()) {
ResultStreamFaker results(&*db);
ResultStreamFaker results(interpreter_context->db.get());
interpreter->Prepare(query, {}, nullptr);
interpreter->PullAll(&results);
}
@ -78,7 +76,7 @@ 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(&*db);
ResultStreamFaker results(interpreter_context->db.get());
interpreter->Prepare(query, {}, nullptr);
interpreter->PullAll(&results);
}

View File

@ -14,6 +14,7 @@
#include "query/db_accessor.hpp"
#include "query/interpret/eval.hpp"
#include "query/interpreter.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage.hpp"
// The following classes are wrappers for memgraph::utils::MemoryResource, so that we can
@ -38,9 +39,9 @@ static void MapLiteral(benchmark::State &state) {
memgraph::query::SymbolTable symbol_table;
TMemory memory;
memgraph::query::Frame frame(symbol_table.max_position(), memory.get());
memgraph::storage::Storage db;
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
std::unordered_map<memgraph::query::PropertyIx, memgraph::query::Expression *> elements;
for (int64_t i = 0; i < state.range(0); ++i) {
elements.emplace(ast.GetPropertyIx("prop" + std::to_string(i)), ast.Create<memgraph::query::PrimitiveLiteral>(i));
@ -69,9 +70,9 @@ static void AdditionOperator(benchmark::State &state) {
memgraph::query::SymbolTable symbol_table;
TMemory memory;
memgraph::query::Frame frame(symbol_table.max_position(), memory.get());
memgraph::storage::Storage db;
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::query::Expression *expr = ast.Create<memgraph::query::PrimitiveLiteral>(0);
for (int64_t i = 0; i < state.range(0); ++i) {
expr = ast.Create<memgraph::query::AdditionOperator>(expr, ast.Create<memgraph::query::PrimitiveLiteral>(i));

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
@ -30,7 +30,7 @@
#include "query/frontend/semantic/required_privileges.hpp"
#include "query/frontend/semantic/symbol_generator.hpp"
#include "query/interpreter.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
// The following classes are wrappers for memgraph::utils::MemoryResource, so that we can
// use BENCHMARK_TEMPLATE
@ -62,8 +62,8 @@ class PoolResource final {
static void AddVertices(memgraph::storage::Storage *db, int vertex_count) {
auto dba = db->Access();
for (int i = 0; i < vertex_count; i++) dba.CreateVertex();
MG_ASSERT(!dba.Commit().HasError());
for (int i = 0; i < vertex_count; i++) dba->CreateVertex();
MG_ASSERT(!dba->Commit().HasError());
}
static const char *kStartLabel = "start";
@ -71,17 +71,17 @@ static const char *kStartLabel = "start";
static void AddStarGraph(memgraph::storage::Storage *db, int spoke_count, int depth) {
{
auto dba = db->Access();
auto center_vertex = dba.CreateVertex();
MG_ASSERT(center_vertex.AddLabel(dba.NameToLabel(kStartLabel)).HasValue());
auto center_vertex = dba->CreateVertex();
MG_ASSERT(center_vertex.AddLabel(dba->NameToLabel(kStartLabel)).HasValue());
for (int i = 0; i < spoke_count; ++i) {
auto prev_vertex = center_vertex;
for (int j = 0; j < depth; ++j) {
auto dest = dba.CreateVertex();
MG_ASSERT(dba.CreateEdge(&prev_vertex, &dest, dba.NameToEdgeType("Type")).HasValue());
auto dest = dba->CreateVertex();
MG_ASSERT(dba->CreateEdge(&prev_vertex, &dest, dba->NameToEdgeType("Type")).HasValue());
prev_vertex = dest;
}
}
MG_ASSERT(!dba.Commit().HasError());
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
@ -91,19 +91,19 @@ static void AddTree(memgraph::storage::Storage *db, int vertex_count) {
auto dba = db->Access();
std::vector<memgraph::storage::VertexAccessor> vertices;
vertices.reserve(vertex_count);
auto root = dba.CreateVertex();
MG_ASSERT(root.AddLabel(dba.NameToLabel(kStartLabel)).HasValue());
auto root = dba->CreateVertex();
MG_ASSERT(root.AddLabel(dba->NameToLabel(kStartLabel)).HasValue());
vertices.push_back(root);
// NOLINTNEXTLINE(cert-msc32-c,cert-msc51-cpp)
std::mt19937_64 rg(42);
for (int i = 1; i < vertex_count; ++i) {
auto v = dba.CreateVertex();
auto v = dba->CreateVertex();
std::uniform_int_distribution<> dis(0U, vertices.size() - 1U);
auto &parent = vertices.at(dis(rg));
MG_ASSERT(dba.CreateEdge(&parent, &v, dba.NameToEdgeType("Type")).HasValue());
MG_ASSERT(dba->CreateEdge(&parent, &v, dba->NameToEdgeType("Type")).HasValue());
vertices.push_back(v);
}
MG_ASSERT(!dba.Commit().HasError());
MG_ASSERT(!dba->Commit().HasError());
}
MG_ASSERT(!db->CreateIndex(db->NameToLabel(kStartLabel)).HasError());
}
@ -124,16 +124,16 @@ template <class TMemory>
static void Distinct(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddVertices(&db, state.range(0));
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(0));
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
auto query_string = "MATCH (s) RETURN DISTINCT s";
auto *cypher_query = ParseCypherQuery(query_string, &ast);
auto symbol_table = memgraph::query::MakeSymbolTable(cypher_query);
auto context = memgraph::query::plan::MakePlanningContext(&ast, &symbol_table, cypher_query, &dba);
auto plan_and_cost = memgraph::query::plan::MakeLogicalPlan(&context, parameters, false);
ResultStreamFaker results(&db);
ResultStreamFaker results(db.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -174,12 +174,12 @@ template <class TMemory>
static void ExpandVariable(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddStarGraph(&db, state.range(0), state.range(1));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddStarGraph(db.get(), state.range(0), state.range(1));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::DEPTH_FIRST, &symbol_table);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -213,12 +213,12 @@ template <class TMemory>
static void ExpandBfs(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddTree(&db, state.range(0));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddTree(db.get(), state.range(0));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::BREADTH_FIRST, &symbol_table);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -246,14 +246,14 @@ template <class TMemory>
static void ExpandShortest(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddTree(&db, state.range(0));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddTree(db.get(), state.range(0));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::BREADTH_FIRST, &symbol_table);
expand_variable.common_.existing_node = true;
auto dest_symbol = expand_variable.common_.node_symbol;
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -284,8 +284,8 @@ template <class TMemory>
static void ExpandWeightedShortest(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddTree(&db, state.range(0));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddTree(db.get(), state.range(0));
memgraph::query::SymbolTable symbol_table;
auto expand_variable = MakeExpandVariable(memgraph::query::EdgeAtom::Type::WEIGHTED_SHORTEST_PATH, &symbol_table);
expand_variable.common_.existing_node = true;
@ -293,8 +293,8 @@ static void ExpandWeightedShortest(benchmark::State &state) {
symbol_table.CreateSymbol("edge", false), symbol_table.CreateSymbol("vertex", false),
ast.Create<memgraph::query::PrimitiveLiteral>(1)};
auto dest_symbol = expand_variable.common_.node_symbol;
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -327,8 +327,8 @@ template <class TMemory>
static void Accumulate(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddVertices(&db, state.range(1));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(1));
memgraph::query::SymbolTable symbol_table;
auto scan_all = std::make_shared<memgraph::query::plan::ScanAll>(nullptr, symbol_table.CreateSymbol("v", false));
std::vector<memgraph::query::Symbol> symbols;
@ -338,8 +338,8 @@ static void Accumulate(benchmark::State &state) {
}
memgraph::query::plan::Accumulate accumulate(scan_all, symbols,
/* advance_command= */ false);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -368,8 +368,8 @@ template <class TMemory>
static void Aggregate(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddVertices(&db, state.range(1));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(1));
memgraph::query::SymbolTable symbol_table;
auto scan_all = std::make_shared<memgraph::query::plan::ScanAll>(nullptr, symbol_table.CreateSymbol("v", false));
std::vector<memgraph::query::Symbol> symbols;
@ -387,8 +387,8 @@ static void Aggregate(benchmark::State &state) {
symbol_table.CreateSymbol("out" + std::to_string(i), false)});
}
memgraph::query::plan::Aggregate aggregate(scan_all, aggregations, group_by, symbols);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -421,8 +421,8 @@ template <class TMemory>
static void OrderBy(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddVertices(&db, state.range(1));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(1));
memgraph::query::SymbolTable symbol_table;
auto scan_all = std::make_shared<memgraph::query::plan::ScanAll>(nullptr, symbol_table.CreateSymbol("v", false));
std::vector<memgraph::query::Symbol> symbols;
@ -437,8 +437,8 @@ static void OrderBy(benchmark::State &state) {
sort_items.push_back({memgraph::query::Ordering::ASC, ast.Create<memgraph::query::PrimitiveLiteral>(rand_value)});
}
memgraph::query::plan::OrderBy order_by(scan_all, sort_items, symbols);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -467,16 +467,16 @@ template <class TMemory>
static void Unwind(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::query::Parameters parameters;
memgraph::storage::Storage db;
AddVertices(&db, state.range(0));
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
AddVertices(db.get(), state.range(0));
memgraph::query::SymbolTable symbol_table;
auto scan_all = std::make_shared<memgraph::query::plan::ScanAll>(nullptr, symbol_table.CreateSymbol("v", false));
auto list_sym = symbol_table.CreateSymbol("list", false);
auto *list_expr = ast.Create<memgraph::query::Identifier>("list")->MapTo(list_sym);
auto out_sym = symbol_table.CreateSymbol("out", false);
memgraph::query::plan::Unwind unwind(scan_all, list_expr, out_sym);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
// We need to only set the memory for temporary (per pull) evaluations
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
@ -503,7 +503,7 @@ template <class TMemory>
// NOLINTNEXTLINE(google-runtime-references)
static void Foreach(benchmark::State &state) {
memgraph::query::AstStorage ast;
memgraph::storage::Storage db;
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
memgraph::query::SymbolTable symbol_table;
auto list_sym = symbol_table.CreateSymbol("list", false);
auto *list_expr = ast.Create<memgraph::query::Identifier>("list")->MapTo(list_sym);
@ -512,8 +512,8 @@ static void Foreach(benchmark::State &state) {
std::make_shared<memgraph::query::plan::CreateNode>(nullptr, memgraph::query::plan::NodeCreationInfo{});
auto foreach = std::make_shared<memgraph::query::plan::Foreach>(nullptr, std::move(create_node), list_expr, out_sym);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
TMemory per_pull_memory;
memgraph::query::EvaluationContext evaluation_context{per_pull_memory.get()};
while (state.KeepRunning()) {

View File

@ -17,7 +17,7 @@
#include "query/plan/cost_estimator.hpp"
#include "query/plan/planner.hpp"
#include "query/plan/vertex_count_cache.hpp"
#include "storage/v2/storage.hpp"
#include "storage/v2/inmemory/storage.hpp"
// Add chained MATCH (node1) -- (node2), MATCH (node2) -- (node3) ... clauses.
static memgraph::query::CypherQuery *AddChainedMatches(int num_matches, memgraph::query::AstStorage &storage) {
@ -43,9 +43,9 @@ static memgraph::query::CypherQuery *AddChainedMatches(int num_matches, memgraph
}
static void BM_PlanChainedMatches(benchmark::State &state) {
memgraph::storage::Storage db;
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
while (state.KeepRunning()) {
state.PauseTiming();
memgraph::query::AstStorage storage;
@ -98,24 +98,24 @@ static auto CreateIndexedVertices(int index_count, int vertex_count, memgraph::s
auto dba = db->Access();
for (int vi = 0; vi < vertex_count; ++vi) {
for (int index = 0; index < index_count; ++index) {
auto vertex = dba.CreateVertex();
auto vertex = dba->CreateVertex();
MG_ASSERT(vertex.AddLabel(label).HasValue());
MG_ASSERT(vertex.SetProperty(prop, memgraph::storage::PropertyValue(index)).HasValue());
}
}
MG_ASSERT(!dba.Commit().HasError());
MG_ASSERT(!dba->Commit().HasError());
return std::make_pair("label", "prop");
}
static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
memgraph::storage::Storage db;
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
std::string label;
std::string prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, &db);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db.get());
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
memgraph::query::Parameters parameters;
while (state.KeepRunning()) {
state.PauseTiming();
@ -137,14 +137,14 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
}
static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(benchmark::State &state) {
memgraph::storage::Storage db;
std::unique_ptr<memgraph::storage::Storage> db(new memgraph::storage::InMemoryStorage());
std::string label;
std::string prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, &db);
auto storage_dba = db.Access();
memgraph::query::DbAccessor dba(&storage_dba);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db.get());
auto storage_dba = db->Access();
memgraph::query::DbAccessor dba(storage_dba.get());
auto vertex_counts = memgraph::query::plan::MakeVertexCountCache(&dba);
memgraph::query::Parameters parameters;
while (state.KeepRunning()) {

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
@ -13,6 +13,7 @@
#include <gflags/gflags.h>
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage.hpp"
#include "utils/timer.hpp"
@ -43,12 +44,12 @@ void UpdateLabelFunc(int thread_id, memgraph::storage::Storage *storage,
for (int iter = 0; iter < num_iterations; ++iter) {
auto acc = storage->Access();
memgraph::storage::Gid gid = vertices.at(vertex_dist(gen));
std::optional<memgraph::storage::VertexAccessor> vertex = acc.FindVertex(gid, memgraph::storage::View::OLD);
auto vertex = acc->FindVertex(gid, memgraph::storage::View::OLD);
MG_ASSERT(vertex.has_value(), "Vertex with GID {} doesn't exist", gid.AsUint());
if (vertex->AddLabel(memgraph::storage::LabelId::FromUint(label_dist(gen))).HasValue()) {
MG_ASSERT(!acc.Commit().HasError());
MG_ASSERT(!acc->Commit().HasError());
} else {
acc.Abort();
acc->Abort();
}
}
}
@ -57,21 +58,21 @@ int main(int argc, char *argv[]) {
gflags::ParseCommandLineFlags(&argc, &argv, true);
for (const auto &config : TestConfigurations) {
memgraph::storage::Storage storage(config.second);
std::unique_ptr<memgraph::storage::Storage> storage(new memgraph::storage::InMemoryStorage(config.second));
std::vector<memgraph::storage::Gid> vertices;
{
auto acc = storage.Access();
auto acc = storage->Access();
for (int i = 0; i < FLAGS_num_vertices; ++i) {
vertices.push_back(acc.CreateVertex().Gid());
vertices.push_back(acc->CreateVertex().Gid());
}
MG_ASSERT(!acc.Commit().HasError());
MG_ASSERT(!acc->Commit().HasError());
}
memgraph::utils::Timer timer;
std::vector<std::thread> threads;
threads.reserve(FLAGS_num_threads);
for (int i = 0; i < FLAGS_num_threads; ++i) {
threads.emplace_back(UpdateLabelFunc, i, &storage, vertices, FLAGS_num_iterations);
threads.emplace_back(UpdateLabelFunc, i, storage.get(), vertices, FLAGS_num_iterations);
}
for (int i = 0; i < FLAGS_num_threads; ++i) {

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