Parallelize recovery (#868)

* Parallelize edge recovery

* Load vertex labels and properties parallel

* Add parallel connectivity loading

* Add batches information to snapshot

* Introduce `items_per_batch` and `recovery_thread_count` flags

* Make possible to load snapshots with old version

* Add vertex batches to `RecoveryInfo`

* Extend durability integration tests with v15 test cases

* Add `std::vector` based `InitProperties`

* Use `InitProperties` in snapshot loading
This commit is contained in:
János Benjamin Antal 2023-04-25 16:25:25 +02:00
parent 64fd281b2e
commit 6c947947eb
48 changed files with 1185 additions and 52 deletions

View File

@ -192,6 +192,16 @@ DEFINE_VALIDATED_uint64(storage_wal_file_flush_every_n_tx,
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_bool(storage_snapshot_on_exit, false, "Controls whether the storage creates another snapshot on exit.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_uint64(storage_items_per_batch, memgraph::storage::Config::Durability().items_per_batch,
"The number of edges and vertices stored in a batch in a snapshot file.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_uint64(storage_recovery_thread_count,
std::max(static_cast<uint64_t>(std::thread::hardware_concurrency()),
memgraph::storage::Config::Durability().recovery_thread_count),
"The number of threads used to recover persisted data from disk.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_bool(telemetry_enabled, false,
"Set to true to enable telemetry. We collect information about the "
@ -852,7 +862,9 @@ int main(int argc, char **argv) {
.wal_file_size_kibibytes = FLAGS_storage_wal_file_size_kib,
.wal_file_flush_every_n_tx = FLAGS_storage_wal_file_flush_every_n_tx,
.snapshot_on_exit = FLAGS_storage_snapshot_on_exit,
.restore_replicas_on_startup = true},
.restore_replicas_on_startup = true,
.items_per_batch = FLAGS_storage_items_per_batch,
.recovery_thread_count = FLAGS_storage_recovery_thread_count},
.transaction = {.isolation_level = ParseIsolationLevel()}};
if (FLAGS_storage_snapshot_interval_sec == 0) {
if (FLAGS_storage_wal_enabled) {

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

@ -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
@ -50,6 +50,9 @@ struct Config {
bool snapshot_on_exit{false};
bool restore_replicas_on_startup{false};
uint64_t items_per_batch{1'000'000};
uint64_t recovery_thread_count{8};
} durability;
struct Transaction {

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
@ -163,7 +163,7 @@ std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_di
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
utils::SkipList<Vertex> *vertices, utils::SkipList<Edge> *edges,
std::atomic<uint64_t> *edge_count, NameIdMapper *name_id_mapper,
Indices *indices, Constraints *constraints, Config::Items items,
Indices *indices, Constraints *constraints, const Config &config,
uint64_t *wal_seq_num) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
spdlog::info("Recovering persisted data using snapshot ({}) and WAL directory ({}).", snapshot_directory,
@ -195,7 +195,7 @@ std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_di
}
spdlog::info("Starting snapshot recovery from {}.", path);
try {
recovered_snapshot = LoadSnapshot(path, vertices, edges, epoch_history, name_id_mapper, edge_count, items);
recovered_snapshot = LoadSnapshot(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config);
spdlog::info("Snapshot recovery successful!");
break;
} catch (const RecoveryFailure &e) {
@ -319,7 +319,7 @@ std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_di
}
try {
auto info = LoadWal(wal_file.path, &indices_constraints, last_loaded_timestamp, vertices, edges, name_id_mapper,
edge_count, items);
edge_count, config.items);
recovery_info.next_vertex_id = std::max(recovery_info.next_vertex_id, info.next_vertex_id);
recovery_info.next_edge_id = std::max(recovery_info.next_edge_id, info.next_edge_id);
recovery_info.next_timestamp = std::max(recovery_info.next_timestamp, info.next_timestamp);

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
@ -108,7 +108,7 @@ std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_di
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
utils::SkipList<Vertex> *vertices, utils::SkipList<Edge> *edges,
std::atomic<uint64_t> *edge_count, NameIdMapper *name_id_mapper,
Indices *indices, Constraints *constraints, Config::Items items,
Indices *indices, Constraints *constraints, const Config &config,
uint64_t *wal_seq_num);
} // namespace memgraph::storage::durability

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -12,6 +12,7 @@
#pragma once
#include <algorithm>
#include <optional>
#include <set>
#include <utility>
#include <vector>
@ -29,6 +30,8 @@ struct RecoveryInfo {
// last timestamp read from a WAL file
std::optional<uint64_t> last_commit_timestamp;
std::vector<std::pair<Gid /*first vertex gid*/, uint64_t /*batch size*/>> vertex_batches;
};
/// Structure used to track indices and constraints during recovery.

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,18 +11,26 @@
#include "storage/v2/durability/snapshot.hpp"
#include <thread>
#include "storage/v2/durability/exceptions.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/serialization.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_ref.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/vertex.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/concepts.hpp"
#include "utils/file_locker.hpp"
#include "utils/logging.hpp"
#include "utils/message.hpp"
#include "utils/spin_lock.hpp"
#include "utils/synchronized.hpp"
namespace memgraph::storage::durability {
@ -40,6 +48,8 @@ namespace memgraph::storage::durability {
// * offset to the constraints section
// * offset to the mapper section
// * offset to the metadata section
// * offset to the offset-count pair of the first edge batch (`0` if properties on edges are disabled)
// * offset to the offset-count pair of the first vertex batch
//
// 4) Encoded edges (if properties on edges are enabled); each edge is written
// in the following format:
@ -87,9 +97,23 @@ namespace memgraph::storage::durability {
// * number of edges
// * number of vertices
//
// 10) Batch infos
// * number of edge batch infos
// * edge batch infos
// * starting offset of the batch
// * number of edges in the batch
// * vertex batch infos
// * starting offset of the batch
// * number of vertices in the batch
//
// IMPORTANT: When changing snapshot encoding/decoding bump the snapshot/WAL
// version in `version.hpp`.
struct BatchInfo {
uint64_t offset;
uint64_t count;
};
// Function used to read information about the snapshot file.
SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
// Check magic and version.
@ -124,6 +148,13 @@ SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
info.offset_mapper = read_offset();
info.offset_epoch_history = read_offset();
info.offset_metadata = read_offset();
if (*version >= 15U) {
info.offset_edge_batches = read_offset();
info.offset_vertex_batches = read_offset();
} else {
info.offset_edge_batches = 0U;
info.offset_vertex_batches = 0U;
}
}
// Read metadata.
@ -157,17 +188,385 @@ SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
return info;
}
RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
std::vector<BatchInfo> ReadBatchInfos(Decoder &snapshot) {
std::vector<BatchInfo> infos;
const auto infos_size = snapshot.ReadUint();
if (!infos_size.has_value()) {
throw RecoveryFailure("Invalid snapshot data!");
}
infos.reserve(*infos_size);
for (auto i{0U}; i < *infos_size; ++i) {
const auto offset = snapshot.ReadUint();
if (!offset.has_value()) {
throw RecoveryFailure("Invalid snapshot data!");
}
const auto count = snapshot.ReadUint();
if (!count.has_value()) {
throw RecoveryFailure("Invalid snapshot data!");
}
infos.push_back(BatchInfo{*offset, *count});
}
return infos;
}
template <typename TFunc>
void LoadPartialEdges(const std::filesystem::path &path, utils::SkipList<Edge> &edges, const uint64_t from_offset,
const uint64_t edges_count, const Config::Items items, TFunc get_property_from_id) {
Decoder snapshot;
snapshot.Initialize(path, kSnapshotMagic);
// Recover edges.
auto edge_acc = edges.access();
uint64_t last_edge_gid = 0;
spdlog::info("Recovering {} edges.", edges_count);
if (!snapshot.SetPosition(from_offset)) throw RecoveryFailure("Couldn't read data from snapshot!");
std::vector<std::pair<PropertyId, PropertyValue>> read_properties;
for (uint64_t i = 0; i < edges_count; ++i) {
{
const auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_EDGE) throw RecoveryFailure("Invalid snapshot data!");
}
// Read edge GID.
auto gid = snapshot.ReadUint();
if (!gid) throw RecoveryFailure("Invalid snapshot data!");
if (i > 0 && *gid <= last_edge_gid) throw RecoveryFailure("Invalid snapshot data!");
last_edge_gid = *gid;
if (items.properties_on_edges) {
spdlog::debug("Recovering edge {} with properties.", *gid);
auto [it, inserted] = edge_acc.insert(Edge{Gid::FromUint(*gid), nullptr});
if (!inserted) throw RecoveryFailure("The edge must be inserted here!");
// Recover properties.
{
auto props_size = snapshot.ReadUint();
if (!props_size) throw RecoveryFailure("Invalid snapshot data!");
auto &props = it->properties;
read_properties.clear();
read_properties.reserve(*props_size);
for (uint64_t j = 0; j < *props_size; ++j) {
auto key = snapshot.ReadUint();
if (!key) throw RecoveryFailure("Invalid snapshot data!");
auto value = snapshot.ReadPropertyValue();
if (!value) throw RecoveryFailure("Invalid snapshot data!");
read_properties.emplace_back(get_property_from_id(*key), std::move(*value));
}
props.InitProperties(std::move(read_properties));
}
} else {
spdlog::debug("Ensuring edge {} doesn't have any properties.", *gid);
// Read properties.
{
auto props_size = snapshot.ReadUint();
if (!props_size) throw RecoveryFailure("Invalid snapshot data!");
if (*props_size != 0)
throw RecoveryFailure(
"The snapshot has properties on edges, but the storage is "
"configured without properties on edges!");
}
}
}
spdlog::info("Partial edges are recovered.");
}
// Returns the gid of the last recovered vertex
template <typename TLabelFromIdFunc, typename TPropertyFromIdFunc>
uint64_t LoadPartialVertices(const std::filesystem::path &path, utils::SkipList<Vertex> &vertices,
const uint64_t from_offset, const uint64_t vertices_count,
TLabelFromIdFunc get_label_from_id, TPropertyFromIdFunc get_property_from_id) {
Decoder snapshot;
snapshot.Initialize(path, kSnapshotMagic);
if (!snapshot.SetPosition(from_offset)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto vertex_acc = vertices.access();
uint64_t last_vertex_gid = 0;
spdlog::info("Recovering {} vertices.", vertices_count);
std::vector<std::pair<PropertyId, PropertyValue>> read_properties;
for (uint64_t i = 0; i < vertices_count; ++i) {
{
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_VERTEX) throw RecoveryFailure("Invalid snapshot data!");
}
// Insert vertex.
auto gid = snapshot.ReadUint();
if (!gid) throw RecoveryFailure("Invalid snapshot data!");
if (i > 0 && *gid <= last_vertex_gid) {
throw RecoveryFailure("Invalid snapshot data!");
}
last_vertex_gid = *gid;
spdlog::debug("Recovering vertex {}.", *gid);
auto [it, inserted] = vertex_acc.insert(Vertex{Gid::FromUint(*gid), nullptr});
if (!inserted) throw RecoveryFailure("The vertex must be inserted here!");
// Recover labels.
spdlog::trace("Recovering labels for vertex {}.", *gid);
{
auto labels_size = snapshot.ReadUint();
if (!labels_size) throw RecoveryFailure("Invalid snapshot data!");
auto &labels = it->labels;
labels.reserve(*labels_size);
for (uint64_t j = 0; j < *labels_size; ++j) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
labels.emplace_back(get_label_from_id(*label));
}
}
// Recover properties.
spdlog::trace("Recovering properties for vertex {}.", *gid);
{
auto props_size = snapshot.ReadUint();
if (!props_size) throw RecoveryFailure("Invalid snapshot data!");
auto &props = it->properties;
read_properties.clear();
read_properties.reserve(*props_size);
for (uint64_t j = 0; j < *props_size; ++j) {
auto key = snapshot.ReadUint();
if (!key) throw RecoveryFailure("Invalid snapshot data!");
auto value = snapshot.ReadPropertyValue();
if (!value) throw RecoveryFailure("Invalid snapshot data!");
read_properties.emplace_back(get_property_from_id(*key), std::move(*value));
}
props.InitProperties(std::move(read_properties));
}
// Skip in edges.
{
auto in_size = snapshot.ReadUint();
if (!in_size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t j = 0; j < *in_size; ++j) {
auto edge_gid = snapshot.ReadUint();
if (!edge_gid) throw RecoveryFailure("Invalid snapshot data!");
auto from_gid = snapshot.ReadUint();
if (!from_gid) throw RecoveryFailure("Invalid snapshot data!");
auto edge_type = snapshot.ReadUint();
if (!edge_type) throw RecoveryFailure("Invalid snapshot data!");
}
}
// Skip out edges.
auto out_size = snapshot.ReadUint();
if (!out_size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t j = 0; j < *out_size; ++j) {
auto edge_gid = snapshot.ReadUint();
if (!edge_gid) throw RecoveryFailure("Invalid snapshot data!");
auto to_gid = snapshot.ReadUint();
if (!to_gid) throw RecoveryFailure("Invalid snapshot data!");
auto edge_type = snapshot.ReadUint();
if (!edge_type) throw RecoveryFailure("Invalid snapshot data!");
}
}
spdlog::info("Partial vertices are recovered.");
return last_vertex_gid;
}
// Returns the number of edges recovered
struct LoadPartialConnectivityResult {
uint64_t edge_count;
uint64_t highest_edge_id;
Gid first_vertex_gid;
};
template <typename TEdgeTypeFromIdFunc>
LoadPartialConnectivityResult LoadPartialConnectivity(const std::filesystem::path &path,
utils::SkipList<Vertex> &vertices, utils::SkipList<Edge> &edges,
const uint64_t from_offset, const uint64_t vertices_count,
const Config::Items items, const bool snapshot_has_edges,
TEdgeTypeFromIdFunc get_edge_type_from_id) {
Decoder snapshot;
snapshot.Initialize(path, kSnapshotMagic);
if (!snapshot.SetPosition(from_offset)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto vertex_acc = vertices.access();
auto edge_acc = edges.access();
// Read the first gid to find the necessary iterator in vertices
const auto first_vertex_gid = std::invoke([&]() mutable {
{
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_VERTEX) throw RecoveryFailure("Invalid snapshot data!");
}
auto gid = snapshot.ReadUint();
if (!gid) throw RecoveryFailure("Invalid snapshot data!");
return Gid::FromUint(*gid);
});
uint64_t edge_count{0};
uint64_t highest_edge_gid{0};
auto vertex_it = vertex_acc.find(first_vertex_gid);
if (vertex_it == vertex_acc.end()) {
throw RecoveryFailure("Invalid snapshot data!");
}
spdlog::info("Recovering connectivity for {} vertices.", vertices_count);
if (!snapshot.SetPosition(from_offset)) throw RecoveryFailure("Couldn't read data from snapshot!");
for (uint64_t i = 0; i < vertices_count; ++i) {
auto &vertex = *vertex_it;
{
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_VERTEX) throw RecoveryFailure("Invalid snapshot data!");
}
auto gid = snapshot.ReadUint();
if (!gid) throw RecoveryFailure("Invalid snapshot data!");
if (gid != vertex.gid.AsUint()) throw RecoveryFailure("Invalid snapshot data!");
// Skip labels.
{
auto labels_size = snapshot.ReadUint();
if (!labels_size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t j = 0; j < *labels_size; ++j) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
}
}
// Skip properties.
{
auto props_size = snapshot.ReadUint();
if (!props_size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t j = 0; j < *props_size; ++j) {
auto key = snapshot.ReadUint();
if (!key) throw RecoveryFailure("Invalid snapshot data!");
auto value = snapshot.SkipPropertyValue();
if (!value) throw RecoveryFailure("Invalid snapshot data!");
}
}
// Recover in edges.
{
spdlog::trace("Recovering inbound edges for vertex {}.", vertex.gid.AsUint());
auto in_size = snapshot.ReadUint();
if (!in_size) throw RecoveryFailure("Invalid snapshot data!");
vertex.in_edges.reserve(*in_size);
for (uint64_t j = 0; j < *in_size; ++j) {
auto edge_gid = snapshot.ReadUint();
if (!edge_gid) throw RecoveryFailure("Invalid snapshot data!");
highest_edge_gid = std::max(highest_edge_gid, *edge_gid);
auto from_gid = snapshot.ReadUint();
if (!from_gid) throw RecoveryFailure("Invalid snapshot data!");
auto edge_type = snapshot.ReadUint();
if (!edge_type) throw RecoveryFailure("Invalid snapshot data!");
auto from_vertex = vertex_acc.find(Gid::FromUint(*from_gid));
if (from_vertex == vertex_acc.end()) throw RecoveryFailure("Invalid from vertex!");
EdgeRef edge_ref(Gid::FromUint(*edge_gid));
if (items.properties_on_edges) {
// The snapshot contains the individiual edges only if it was created with a config where properties are
// allowed on edges. That means the snapshots that were created without edge properties will only contain the
// edges in the in/out edges list of vertices, therefore the edges has to be created here.
if (snapshot_has_edges) {
auto edge = edge_acc.find(Gid::FromUint(*edge_gid));
if (edge == edge_acc.end()) throw RecoveryFailure("Invalid edge!");
edge_ref = EdgeRef(&*edge);
} else {
auto [edge, inserted] = edge_acc.insert(Edge{Gid::FromUint(*edge_gid), nullptr});
edge_ref = EdgeRef(&*edge);
}
}
vertex.in_edges.emplace_back(get_edge_type_from_id(*edge_type), &*from_vertex, edge_ref);
}
}
// Recover out edges.
{
spdlog::trace("Recovering outbound edges for vertex {}.", vertex.gid.AsUint());
auto out_size = snapshot.ReadUint();
if (!out_size) throw RecoveryFailure("Invalid snapshot data!");
vertex.out_edges.reserve(*out_size);
for (uint64_t j = 0; j < *out_size; ++j) {
auto edge_gid = snapshot.ReadUint();
if (!edge_gid) throw RecoveryFailure("Invalid snapshot data!");
auto to_gid = snapshot.ReadUint();
if (!to_gid) throw RecoveryFailure("Invalid snapshot data!");
auto edge_type = snapshot.ReadUint();
if (!edge_type) throw RecoveryFailure("Invalid snapshot data!");
auto to_vertex = vertex_acc.find(Gid::FromUint(*to_gid));
if (to_vertex == vertex_acc.end()) throw RecoveryFailure("Invalid to vertex!");
EdgeRef edge_ref(Gid::FromUint(*edge_gid));
if (items.properties_on_edges) {
// The snapshot contains the individiual edges only if it was created with a config where properties are
// allowed on edges. That means the snapshots that were created without edge properties will only contain the
// edges in the in/out edges list of vertices, therefore the edges has to be created here.
if (snapshot_has_edges) {
auto edge = edge_acc.find(Gid::FromUint(*edge_gid));
if (edge == edge_acc.end()) throw RecoveryFailure("Invalid edge!");
edge_ref = EdgeRef(&*edge);
} else {
auto [edge, inserted] = edge_acc.insert(Edge{Gid::FromUint(*edge_gid), nullptr});
edge_ref = EdgeRef(&*edge);
}
}
vertex.out_edges.emplace_back(get_edge_type_from_id(*edge_type), &*to_vertex, edge_ref);
// Increment edge count. We only increment the count here because the
// information is duplicated in in_edges.
edge_count++;
}
}
++vertex_it;
}
spdlog::info("Partial connectivities are recovered.");
return {edge_count, highest_edge_gid, first_vertex_gid};
}
template <typename TFunc>
void RecoverOnMultipleThreads(size_t thread_count, const TFunc &func, const std::vector<BatchInfo> &batches) {
utils::Synchronized<std::optional<RecoveryFailure>, utils::SpinLock> maybe_error{};
{
std::atomic<uint64_t> batch_counter = 0;
thread_count = std::min(thread_count, batches.size());
std::vector<std::jthread> threads;
threads.reserve(thread_count);
for (auto i{0U}; i < thread_count; ++i) {
threads.emplace_back([&func, &batches, &maybe_error, &batch_counter]() {
while (!maybe_error.Lock()->has_value()) {
const auto batch_index = batch_counter++;
if (batch_index >= batches.size()) {
return;
}
const auto &batch = batches[batch_index];
try {
func(batch_index, batch);
} catch (RecoveryFailure &failure) {
*maybe_error.Lock() = std::move(failure);
}
}
});
}
}
if (maybe_error.Lock()->has_value()) {
throw RecoveryFailure((*maybe_error.Lock())->what());
}
}
RecoveredSnapshot LoadSnapshotVersion14(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, Config::Items items) {
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count,
Config::Items items) {
RecoveryInfo ret;
RecoveredIndicesAndConstraints indices_constraints;
Decoder snapshot;
auto version = snapshot.Initialize(path, kSnapshotMagic);
if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version));
if (*version != 14U) throw RecoveryFailure(fmt::format("Expected snapshot version is 14, but got {}", *version));
// Cleanup of loaded data in case of failure.
bool success = false;
@ -625,10 +1024,297 @@ RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipLis
return {info, ret, std::move(indices_constraints)};
}
RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config) {
RecoveryInfo recovery_info;
RecoveredIndicesAndConstraints indices_constraints;
Decoder snapshot;
const auto version = snapshot.Initialize(path, kSnapshotMagic);
if (!version) throw RecoveryFailure("Couldn't read snapshot magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure(fmt::format("Invalid snapshot version {}", *version));
if (*version == 14U) {
return LoadSnapshotVersion14(path, vertices, edges, epoch_history, name_id_mapper, edge_count, config.items);
}
// Cleanup of loaded data in case of failure.
bool success = false;
utils::OnScopeExit cleanup([&] {
if (!success) {
edges->clear();
vertices->clear();
epoch_history->clear();
}
});
// Read snapshot info.
const auto info = ReadSnapshotInfo(path);
spdlog::info("Recovering {} vertices and {} edges.", info.vertices_count, info.edges_count);
// Check for edges.
bool snapshot_has_edges = info.offset_edges != 0;
// Recover mapper.
std::unordered_map<uint64_t, uint64_t> snapshot_id_map;
{
spdlog::info("Recovering mapper metadata.");
if (!snapshot.SetPosition(info.offset_mapper)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_MAPPER) throw RecoveryFailure("Invalid snapshot data!");
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
for (uint64_t i = 0; i < *size; ++i) {
auto id = snapshot.ReadUint();
if (!id) throw RecoveryFailure("Invalid snapshot data!");
auto name = snapshot.ReadString();
if (!name) throw RecoveryFailure("Invalid snapshot data!");
auto my_id = name_id_mapper->NameToId(*name);
snapshot_id_map.emplace(*id, my_id);
SPDLOG_TRACE("Mapping \"{}\"from snapshot id {} to actual id {}.", *name, *id, my_id);
}
}
auto get_label_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return LabelId::FromUint(it->second);
};
auto get_property_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return PropertyId::FromUint(it->second);
};
auto get_edge_type_from_id = [&snapshot_id_map](uint64_t snapshot_id) {
auto it = snapshot_id_map.find(snapshot_id);
if (it == snapshot_id_map.end()) throw RecoveryFailure("Invalid snapshot data!");
return EdgeTypeId::FromUint(it->second);
};
// Reset current edge count.
edge_count->store(0, std::memory_order_release);
{
spdlog::info("Recovering edges.");
// Recover edges.
if (snapshot_has_edges) {
// We don't need to check whether we store properties on edge or not, because `LoadPartialEdges` will always
// iterate over the edges in the snapshot (if they exist) and the current configuration of properties on edge only
// affect what it does:
// 1. If properties are allowed on edges, then it loads the edges.
// 2. If properties are not allowed on edges, then it checks that none of the edges have any properties.
if (!snapshot.SetPosition(info.offset_edge_batches)) {
throw RecoveryFailure("Couldn't read data from snapshot!");
}
const auto edge_batches = ReadBatchInfos(snapshot);
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, edges, items = config.items, &get_property_from_id](const size_t /*batch_index*/,
const BatchInfo &batch) {
LoadPartialEdges(path, *edges, batch.offset, batch.count, items, get_property_from_id);
},
edge_batches);
}
spdlog::info("Edges are recovered.");
// Recover vertices (labels and properties).
spdlog::info("Recovering vertices.", info.vertices_count);
uint64_t last_vertex_gid{0};
if (!snapshot.SetPosition(info.offset_vertex_batches)) {
throw RecoveryFailure("Couldn't read data from snapshot!");
}
const auto vertex_batches = ReadBatchInfos(snapshot);
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, vertices, &vertex_batches, &get_label_from_id, &get_property_from_id, &last_vertex_gid](
const size_t batch_index, const BatchInfo &batch) {
const auto last_vertex_gid_in_batch =
LoadPartialVertices(path, *vertices, batch.offset, batch.count, get_label_from_id, get_property_from_id);
if (batch_index == vertex_batches.size() - 1) {
last_vertex_gid = last_vertex_gid_in_batch;
}
},
vertex_batches);
spdlog::info("Vertices are recovered.");
// Recover vertices (in/out edges).
spdlog::info("Recover connectivity.");
recovery_info.vertex_batches.reserve(vertex_batches.size());
for (const auto batch : vertex_batches) {
recovery_info.vertex_batches.emplace_back(std::make_pair(Gid::FromUint(0), batch.count));
}
std::atomic<uint64_t> highest_edge_gid{0};
RecoverOnMultipleThreads(
config.durability.recovery_thread_count,
[path, vertices, edges, edge_count, items = config.items, snapshot_has_edges, &get_edge_type_from_id,
&highest_edge_gid, &recovery_info](const size_t batch_index, const BatchInfo &batch) {
const auto result = LoadPartialConnectivity(path, *vertices, *edges, batch.offset, batch.count, items,
snapshot_has_edges, get_edge_type_from_id);
edge_count->fetch_add(result.edge_count);
auto known_highest_edge_gid = highest_edge_gid.load();
while (known_highest_edge_gid < result.highest_edge_id) {
highest_edge_gid.compare_exchange_weak(known_highest_edge_gid, result.highest_edge_id);
}
recovery_info.vertex_batches[batch_index].first = result.first_vertex_gid;
},
vertex_batches);
spdlog::info("Connectivity is recovered.");
// Set initial values for edge/vertex ID generators.
recovery_info.next_edge_id = highest_edge_gid + 1;
recovery_info.next_vertex_id = last_vertex_gid + 1;
}
// Recover indices.
{
spdlog::info("Recovering metadata of indices.");
if (!snapshot.SetPosition(info.offset_indices)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_INDICES) throw RecoveryFailure("Invalid snapshot data!");
// Recover label indices.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label indices.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.indices.label, get_label_from_id(*label),
"The label index already exists!");
SPDLOG_TRACE("Recovered metadata of label index for :{}", name_id_mapper->IdToName(snapshot_id_map.at(*label)));
}
spdlog::info("Metadata of label indices are recovered.");
}
// Recover label+property indices.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} label+property indices.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.indices.label_property,
{get_label_from_id(*label), get_property_from_id(*property)},
"The label+property index already exists!");
SPDLOG_TRACE("Recovered metadata of label+property index for :{}({})",
name_id_mapper->IdToName(snapshot_id_map.at(*label)),
name_id_mapper->IdToName(snapshot_id_map.at(*property)));
}
spdlog::info("Metadata of label+property indices are recovered.");
}
spdlog::info("Metadata of indices are recovered.");
}
// Recover constraints.
{
spdlog::info("Recovering metadata of constraints.");
if (!snapshot.SetPosition(info.offset_constraints)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_CONSTRAINTS) throw RecoveryFailure("Invalid snapshot data!");
// Recover existence constraints.
{
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} existence constraints.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
AddRecoveredIndexConstraint(&indices_constraints.constraints.existence,
{get_label_from_id(*label), get_property_from_id(*property)},
"The existence constraint already exists!");
SPDLOG_TRACE("Recovered metadata of existence constraint for :{}({})",
name_id_mapper->IdToName(snapshot_id_map.at(*label)),
name_id_mapper->IdToName(snapshot_id_map.at(*property)));
}
spdlog::info("Metadata of existence constraints are recovered.");
}
// Recover unique constraints.
// Snapshot version should be checked since unique constraints were
// implemented in later versions of snapshot.
if (*version >= kUniqueConstraintVersion) {
auto size = snapshot.ReadUint();
if (!size) throw RecoveryFailure("Invalid snapshot data!");
spdlog::info("Recovering metadata of {} unique constraints.", *size);
for (uint64_t i = 0; i < *size; ++i) {
auto label = snapshot.ReadUint();
if (!label) throw RecoveryFailure("Invalid snapshot data!");
auto properties_count = snapshot.ReadUint();
if (!properties_count) throw RecoveryFailure("Invalid snapshot data!");
std::set<PropertyId> properties;
for (uint64_t j = 0; j < *properties_count; ++j) {
auto property = snapshot.ReadUint();
if (!property) throw RecoveryFailure("Invalid snapshot data!");
properties.insert(get_property_from_id(*property));
}
AddRecoveredIndexConstraint(&indices_constraints.constraints.unique, {get_label_from_id(*label), properties},
"The unique constraint already exists!");
SPDLOG_TRACE("Recovered metadata of unique constraints for :{}",
name_id_mapper->IdToName(snapshot_id_map.at(*label)));
}
spdlog::info("Metadata of unique constraints are recovered.");
}
spdlog::info("Metadata of constraints are recovered.");
}
spdlog::info("Recovering metadata.");
// Recover epoch history
{
if (!snapshot.SetPosition(info.offset_epoch_history)) throw RecoveryFailure("Couldn't read data from snapshot!");
const auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_EPOCH_HISTORY) throw RecoveryFailure("Invalid snapshot data!");
const auto history_size = snapshot.ReadUint();
if (!history_size) {
throw RecoveryFailure("Invalid snapshot data!");
}
for (int i = 0; i < *history_size; ++i) {
auto maybe_epoch_id = snapshot.ReadString();
if (!maybe_epoch_id) {
throw RecoveryFailure("Invalid snapshot data!");
}
const auto maybe_last_commit_timestamp = snapshot.ReadUint();
if (!maybe_last_commit_timestamp) {
throw RecoveryFailure("Invalid snapshot data!");
}
epoch_history->emplace_back(std::move(*maybe_epoch_id), *maybe_last_commit_timestamp);
}
}
spdlog::info("Metadata recovered.");
// Recover timestamp.
recovery_info.next_timestamp = info.start_timestamp + 1;
// Set success flag (to disable cleanup).
success = true;
return {info, recovery_info, std::move(indices_constraints)};
}
void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snapshot_directory,
const std::filesystem::path &wal_directory, uint64_t snapshot_retention_count,
utils::SkipList<Vertex> *vertices, utils::SkipList<Edge> *edges, NameIdMapper *name_id_mapper,
Indices *indices, Constraints *constraints, Config::Items items, const std::string &uuid,
Indices *indices, Constraints *constraints, const Config &config, const std::string &uuid,
const std::string_view epoch_id, const std::deque<std::pair<std::string, uint64_t>> &epoch_history,
utils::FileRetainer *file_retainer) {
// Ensure that the storage directory exists.
@ -649,6 +1335,8 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
uint64_t offset_mapper = 0;
uint64_t offset_metadata = 0;
uint64_t offset_epoch_history = 0;
uint64_t offset_edge_batches = 0;
uint64_t offset_vertex_batches = 0;
{
snapshot.WriteMarker(Marker::SECTION_OFFSETS);
offset_offsets = snapshot.GetPosition();
@ -659,6 +1347,8 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
snapshot.WriteUint(offset_mapper);
snapshot.WriteUint(offset_epoch_history);
snapshot.WriteUint(offset_metadata);
snapshot.WriteUint(offset_edge_batches);
snapshot.WriteUint(offset_vertex_batches);
}
// Object counters.
@ -672,9 +1362,13 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
snapshot.WriteUint(mapping.AsUint());
};
std::vector<BatchInfo> edge_batch_infos;
auto items_in_current_batch{0UL};
auto batch_start_offset{0UL};
// Store all edges.
if (items.properties_on_edges) {
if (config.items.properties_on_edges) {
offset_edges = snapshot.GetPosition();
batch_start_offset = offset_edges;
auto acc = edges->access();
for (auto &edge : acc) {
// The edge visibility check must be done here manually because we don't
@ -713,8 +1407,8 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
// type and invalid from/to pointers because we don't know them here,
// but that isn't an issue because we won't use that part of the API
// here.
auto ea =
EdgeAccessor{edge_ref, EdgeTypeId::FromUint(0UL), nullptr, nullptr, transaction, indices, constraints, items};
auto ea = EdgeAccessor{
edge_ref, EdgeTypeId::FromUint(0UL), nullptr, nullptr, transaction, indices, constraints, config.items};
// Get edge data.
auto maybe_props = ea.Properties(View::OLD);
@ -733,16 +1427,29 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
}
++edges_count;
++items_in_current_batch;
if (items_in_current_batch == config.durability.items_per_batch) {
edge_batch_infos.push_back(BatchInfo{batch_start_offset, items_in_current_batch});
batch_start_offset = snapshot.GetPosition();
items_in_current_batch = 0;
}
}
}
if (items_in_current_batch > 0) {
edge_batch_infos.push_back(BatchInfo{batch_start_offset, items_in_current_batch});
}
std::vector<BatchInfo> vertex_batch_infos;
// Store all vertices.
{
items_in_current_batch = 0;
offset_vertices = snapshot.GetPosition();
batch_start_offset = offset_vertices;
auto acc = vertices->access();
for (auto &vertex : acc) {
// The visibility check is implemented for vertices so we use it here.
auto va = VertexAccessor::Create(&vertex, transaction, indices, constraints, items, View::OLD);
auto va = VertexAccessor::Create(&vertex, transaction, indices, constraints, config.items, View::OLD);
if (!va) continue;
// Get vertex data.
@ -789,6 +1496,16 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
}
++vertices_count;
++items_in_current_batch;
if (items_in_current_batch == config.durability.items_per_batch) {
vertex_batch_infos.push_back(BatchInfo{batch_start_offset, items_in_current_batch});
batch_start_offset = snapshot.GetPosition();
items_in_current_batch = 0;
}
}
if (items_in_current_batch > 0) {
vertex_batch_infos.push_back(BatchInfo{batch_start_offset, items_in_current_batch});
}
}
@ -879,6 +1596,26 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
snapshot.WriteUint(vertices_count);
}
auto write_batch_infos = [&snapshot](const std::vector<BatchInfo> &batch_infos) {
snapshot.WriteUint(batch_infos.size());
for (const auto &batch_info : batch_infos) {
snapshot.WriteUint(batch_info.offset);
snapshot.WriteUint(batch_info.count);
}
};
// Write edge batches
{
offset_edge_batches = snapshot.GetPosition();
write_batch_infos(edge_batch_infos);
}
// Write vertex batches
{
offset_vertex_batches = snapshot.GetPosition();
write_batch_infos(vertex_batch_infos);
}
// Write true offsets.
{
snapshot.SetPosition(offset_offsets);
@ -889,6 +1626,8 @@ void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snaps
snapshot.WriteUint(offset_mapper);
snapshot.WriteUint(offset_epoch_history);
snapshot.WriteUint(offset_metadata);
snapshot.WriteUint(offset_edge_batches);
snapshot.WriteUint(offset_vertex_batches);
}
// Finalize snapshot file.

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
@ -37,6 +37,8 @@ struct SnapshotInfo {
uint64_t offset_mapper;
uint64_t offset_epoch_history;
uint64_t offset_metadata;
uint64_t offset_edge_batches;
uint64_t offset_vertex_batches;
std::string uuid;
std::string epoch_id;
@ -62,13 +64,13 @@ SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path);
RecoveredSnapshot LoadSnapshot(const std::filesystem::path &path, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges,
std::deque<std::pair<std::string, uint64_t>> *epoch_history,
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, Config::Items items);
NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count, const Config &config);
/// Function used to create a snapshot using the given transaction.
void CreateSnapshot(Transaction *transaction, const std::filesystem::path &snapshot_directory,
const std::filesystem::path &wal_directory, uint64_t snapshot_retention_count,
utils::SkipList<Vertex> *vertices, utils::SkipList<Edge> *edges, NameIdMapper *name_id_mapper,
Indices *indices, Constraints *constraints, Config::Items items, const std::string &uuid,
Indices *indices, Constraints *constraints, const Config &config, const std::string &uuid,
std::string_view epoch_id, const std::deque<std::pair<std::string, uint64_t>> &epoch_history,
utils::FileRetainer *file_retainer);

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

View File

@ -1144,7 +1144,8 @@ bool PropertyStore::SetProperty(PropertyId property, const PropertyValue &value)
return !existed;
}
bool PropertyStore::InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
template <typename TContainer>
bool PropertyStore::DoInitProperties(const TContainer &properties) {
uint64_t size = 0;
uint8_t *data = nullptr;
std::tie(size, data) = GetSizeData(buffer_);
@ -1201,6 +1202,20 @@ bool PropertyStore::InitProperties(const std::map<storage::PropertyId, storage::
return true;
}
template bool PropertyStore::DoInitProperties<std::map<PropertyId, PropertyValue>>(
const std::map<PropertyId, PropertyValue> &);
template bool PropertyStore::DoInitProperties<std::vector<std::pair<PropertyId, PropertyValue>>>(
const std::vector<std::pair<PropertyId, PropertyValue>> &);
bool PropertyStore::InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties) {
return DoInitProperties(properties);
}
bool PropertyStore::InitProperties(std::vector<std::pair<storage::PropertyId, storage::PropertyValue>> properties) {
std::sort(properties.begin(), properties.end());
return DoInitProperties(properties);
}
bool PropertyStore::ClearProperties() {
bool in_local_buffer = false;

View File

@ -60,11 +60,17 @@ class PropertyStore {
bool SetProperty(PropertyId property, const PropertyValue &value);
/// Init property values and return `true` if insertion took place. `false` is
/// returned if there exists property in property store and insertion couldn't take place. The time complexity of this
/// function is O(n).
/// returned if there is any existing property in property store and insertion couldn't take place. The time
/// complexity of this function is O(n).
/// @throw std::bad_alloc
bool InitProperties(const std::map<storage::PropertyId, storage::PropertyValue> &properties);
/// Init property values and return `true` if insertion took place. `false` is
/// returned if there is any existing property in property store and insertion couldn't take place. The time
/// complexity of this function is O(n*log(n)):
/// @throw std::bad_alloc
bool InitProperties(std::vector<std::pair<storage::PropertyId, storage::PropertyValue>> properties);
/// Remove all properties and return `true` if any removal took place.
/// `false` is returned if there were no properties to remove. The time
/// complexity of this function is O(1).
@ -72,6 +78,9 @@ class PropertyStore {
bool ClearProperties();
private:
template <typename TContainer>
bool DoInitProperties(const TContainer &properties);
uint8_t buffer_[sizeof(uint64_t) + sizeof(uint8_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
@ -173,7 +173,7 @@ void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::B
spdlog::debug("Loading snapshot");
auto recovered_snapshot = durability::LoadSnapshot(*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_,
&storage_->epoch_history_, &storage_->name_id_mapper_,
&storage_->edge_count_, storage_->config_.items);
&storage_->edge_count_, storage_->config_);
spdlog::debug("Snapshot loaded successfully");
// If this step is present it should always be the first step of
// the recovery so we use the UUID we read from snasphost

View File

@ -360,7 +360,7 @@ Storage::Storage(Config config)
if (config_.durability.recover_on_startup) {
auto info = durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_,
&vertices_, &edges_, &edge_count_, &name_id_mapper_, &indices_, &constraints_,
config_.items, &wal_seq_num_);
config_, &wal_seq_num_);
if (info) {
vertex_id_ = info->next_vertex_id;
edge_id_ = info->next_edge_id;
@ -1947,8 +1947,7 @@ utils::BasicResult<Storage::CreateSnapshotError> Storage::CreateSnapshot(std::op
// Create snapshot.
durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_,
config_.durability.snapshot_retention_count, &vertices_, &edges_, &name_id_mapper_,
&indices_, &constraints_, config_.items, uuid_, epoch_id_, epoch_history_,
&file_retainer_);
&indices_, &constraints_, config_, uuid_, epoch_id_, epoch_history_, &file_retainer_);
// Finalize snapshot transaction.
commit_log_->MarkFinished(transaction.start_timestamp);
};

View File

@ -1,3 +1,14 @@
// 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>

View File

@ -38,7 +38,12 @@ def test_does_default_config_match():
flag_name = flag[0]
# The default value of these is dependent on the given machine.
machine_dependent_configurations = ["bolt_num_workers", "data_directory", "log_file"]
machine_dependent_configurations = [
"bolt_num_workers",
"data_directory",
"log_file",
"storage_recovery_thread_count",
]
if flag_name in machine_dependent_configurations:
continue

View File

@ -116,12 +116,18 @@ startup_config_dict = {
"The time duration between two replica checks/pings. If < 1, replicas will NOT be checked at all. NOTE: The MAIN instance allocates a new thread for each REPLICA.",
),
"storage_gc_cycle_sec": ("30", "30", "Storage garbage collector interval (in seconds)."),
"storage_items_per_batch": (
"1000000",
"1000000",
"The number of edges and vertices stored in a batch in a snapshot file.",
),
"storage_properties_on_edges": ("false", "true", "Controls whether edges have properties."),
"storage_recover_on_startup": (
"false",
"false",
"Controls whether the storage recovers persisted data on startup.",
),
"storage_recovery_thread_count": ("12", "12", "The number of threads used to recover persisted data from disk."),
"storage_snapshot_interval_sec": (
"0",
"300",

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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

@ -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

@ -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

@ -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
@ -618,7 +618,7 @@ class DurabilityTest : public ::testing::TestWithParam<bool> {
std::vector<memgraph::storage::Gid> extended_edge_gids_;
};
void DestroySnapshot(const std::filesystem::path &path) {
void CorruptSnapshot(const std::filesystem::path &path) {
auto info = memgraph::storage::durability::ReadSnapshotInfo(path);
spdlog::info("Destroying snapshot {}", path);
memgraph::utils::OutputFile file;
@ -752,7 +752,7 @@ TEST_P(DurabilityTest, SnapshotFallback) {
{
auto snapshots = GetSnapshotsList();
ASSERT_EQ(snapshots.size(), 2);
DestroySnapshot(*snapshots.begin());
CorruptSnapshot(*snapshots.begin());
}
// Recover snapshot.
@ -835,7 +835,7 @@ TEST_P(DurabilityTest, SnapshotEverythingCorrupt) {
spdlog::info("Skipping snapshot {}", snapshot);
continue;
}
DestroySnapshot(snapshot);
CorruptSnapshot(snapshot);
}
}
@ -2323,7 +2323,7 @@ TEST_P(DurabilityTest, WalAndSnapshotWalRetention) {
}
// Destroy current snapshot.
DestroySnapshot(snapshots[i]);
CorruptSnapshot(snapshots[i]);
}
// Recover data after all of the snapshots have been destroyed. The recovery

View File

@ -14,6 +14,7 @@
#include <limits>
#include "storage/v2/id_types.hpp"
#include "storage/v2/property_store.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/temporal.hpp"
@ -651,24 +652,40 @@ TEST(PropertyStore, IsPropertyEqualTemporalData) {
}
TEST(PropertyStore, SetMultipleProperties) {
memgraph::storage::PropertyStore store;
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
memgraph::storage::PropertyValue(123),
memgraph::storage::PropertyValue()};
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
const memgraph::storage::TemporalData temporal{memgraph::storage::TemporalType::LocalDateTime, 23};
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> data{
// The order of property ids are purposfully not monotonic to test that PropertyStore orders them properly
const std::vector<std::pair<memgraph::storage::PropertyId, memgraph::storage::PropertyValue>> data{
{memgraph::storage::PropertyId::FromInt(1), memgraph::storage::PropertyValue(true)},
{memgraph::storage::PropertyId::FromInt(2), memgraph::storage::PropertyValue(123)},
{memgraph::storage::PropertyId::FromInt(10), memgraph::storage::PropertyValue(123)},
{memgraph::storage::PropertyId::FromInt(3), memgraph::storage::PropertyValue(123.5)},
{memgraph::storage::PropertyId::FromInt(4), memgraph::storage::PropertyValue("nandare")},
{memgraph::storage::PropertyId::FromInt(5), memgraph::storage::PropertyValue(vec)},
{memgraph::storage::PropertyId::FromInt(12), memgraph::storage::PropertyValue(vec)},
{memgraph::storage::PropertyId::FromInt(6), memgraph::storage::PropertyValue(map)},
{memgraph::storage::PropertyId::FromInt(7), memgraph::storage::PropertyValue(temporal)}};
store.InitProperties(data);
const std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> data_in_map{data.begin(), data.end()};
auto check_store = [data](const memgraph::storage::PropertyStore &store) {
for (auto &[key, value] : data) {
ASSERT_TRUE(store.IsPropertyEqual(key, value));
}
};
{
memgraph::storage::PropertyStore store;
EXPECT_TRUE(store.InitProperties(data));
check_store(store);
EXPECT_FALSE(store.InitProperties(data));
EXPECT_FALSE(store.InitProperties(data_in_map));
}
{
memgraph::storage::PropertyStore store;
EXPECT_TRUE(store.InitProperties(data_in_map));
check_store(store);
EXPECT_FALSE(store.InitProperties(data_in_map));
EXPECT_FALSE(store.InitProperties(data));
}
}

View File

@ -1,4 +1,13 @@
// 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 <gtest/gtest.h>
#include <chrono>

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