Copy storage v2 to create storage v3 (#416)

* Copy storage v2 to v3

* Integrate v3 to cmake

* Fix clang-tidy warnings

* Add dummy unit test for storage-v3 to trigger build for code analysis builds
This commit is contained in:
János Benjamin Antal 2022-07-05 08:20:59 +02:00 committed by GitHub
parent 21870a0e7e
commit 1bdc32ba5d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
59 changed files with 13896 additions and 44 deletions

View File

@ -6,6 +6,7 @@ Checks: '*,
-altera-unroll-loops,
-android-*,
-cert-err58-cpp,
-cert-str34-c,
-cppcoreguidelines-avoid-c-arrays,
-cppcoreguidelines-avoid-goto,
-cppcoreguidelines-avoid-magic-numbers,
@ -49,6 +50,7 @@ Checks: '*,
-misc-non-private-member-variables-in-classes,
-modernize-avoid-c-arrays,
-modernize-concat-nested-namespaces,
-modernize-loop-convert,
-modernize-pass-by-value,
-modernize-use-equals-default,
-modernize-use-nodiscard,

View File

@ -10,6 +10,7 @@ add_subdirectory(telemetry)
add_subdirectory(communication)
add_subdirectory(memory)
add_subdirectory(storage/v2)
add_subdirectory(storage/v3)
add_subdirectory(integrations)
add_subdirectory(query)
add_subdirectory(slk)

View File

@ -0,0 +1,38 @@
set(storage_v3_src_files
commit_log.cpp
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)
# #### Replication #####
define_add_lcp(add_lcp_storage lcp_storage_cpp_files generated_lcp_storage_files)
add_lcp_storage(replication/rpc.lcp SLK_SERIALIZE)
add_custom_target(generate_lcp_storage_v3 DEPENDS ${generated_lcp_storage_files})
set(storage_v3_src_files
${storage_v3_src_files}
replication/replication_client.cpp
replication/replication_server.cpp
replication/serialization.cpp
replication/slk.cpp
${lcp_storage_cpp_files})
# ######################
find_package(gflags REQUIRED)
find_package(Threads REQUIRED)
add_library(mg-storage-v3 STATIC ${storage_v3_src_files})
target_link_libraries(mg-storage-v3 Threads::Threads mg-utils gflags)
add_dependencies(mg-storage-v3 generate_lcp_storage)
target_link_libraries(mg-storage-v3 mg-rpc mg-slk)

View File

@ -0,0 +1,113 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/commit_log.hpp"
#include "utils/memory.hpp"
namespace memgraph::storage::v3 {
CommitLog::CommitLog() : allocator_(utils::NewDeleteResource()) {}
CommitLog::CommitLog(uint64_t oldest_active)
: head_start_{oldest_active / kIdsInBlock * kIdsInBlock},
next_start_{head_start_ + kIdsInBlock},
allocator_{utils::NewDeleteResource()} {
head_ = allocator_.allocate(1);
allocator_.construct(head_);
// set all the previous ids
const auto field_idx = (oldest_active % kIdsInBlock) / kIdsInField;
for (size_t i = 0; i < field_idx; ++i) {
head_->field[i] = std::numeric_limits<uint64_t>::max();
}
const auto idx_in_field = oldest_active % kIdsInField;
if (idx_in_field != 0) {
head_->field[field_idx] = std::numeric_limits<uint64_t>::max();
head_->field[field_idx] >>= kIdsInField - idx_in_field;
}
oldest_active_ = oldest_active;
}
CommitLog::~CommitLog() {
while (head_) {
Block *tmp = head_->next;
head_->~Block();
allocator_.deallocate(head_, 1);
head_ = tmp;
}
}
void CommitLog::MarkFinished(uint64_t id) {
std::lock_guard<utils::SpinLock> guard(lock_);
Block *block = FindOrCreateBlock(id);
block->field[(id % kIdsInBlock) / kIdsInField] |= 1ULL << (id % kIdsInField);
if (id == oldest_active_) {
UpdateOldestActive();
}
}
uint64_t CommitLog::OldestActive() {
std::lock_guard<utils::SpinLock> guard(lock_);
return oldest_active_;
}
void CommitLog::UpdateOldestActive() {
while (head_) {
// This is necessary for amortized constant complexity. If we always start
// from the 0th field, the amount of steps we make through each block is
// quadratic in kBlockSize.
uint64_t start_field = oldest_active_ >= head_start_ ? (oldest_active_ - head_start_) / kIdsInField : 0;
for (uint64_t i = start_field; i < kBlockSize; ++i) {
if (head_->field[i] != std::numeric_limits<uint64_t>::max()) {
// NOLINTNEXTLINE(cppcoreguidelines-narrowing-conversions)
oldest_active_ = head_start_ + i * kIdsInField + __builtin_ffsl(static_cast<int64_t>(~head_->field[i])) - 1;
return;
}
}
// All IDs in this block are marked, we can delete it now.
Block *tmp = head_->next;
head_->~Block();
allocator_.deallocate(head_, 1);
head_ = tmp;
head_start_ += kIdsInBlock;
}
oldest_active_ = next_start_;
}
CommitLog::Block *CommitLog::FindOrCreateBlock(const uint64_t id) {
if (!head_) {
head_ = allocator_.allocate(1);
allocator_.construct(head_);
head_start_ = next_start_;
next_start_ += kIdsInBlock;
}
Block *current = head_;
uint64_t current_start = head_start_;
while (id >= current_start + kIdsInBlock) {
if (!current->next) {
current->next = allocator_.allocate(1);
allocator_.construct(current->next);
next_start_ += kIdsInBlock;
}
current = current->next;
current_start += kIdsInBlock;
}
return current;
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,79 @@
// 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.
/// @file commit_log.hpp
#pragma once
#include <cstdint>
#include <mutex>
#include "utils/memory.hpp"
#include "utils/spin_lock.hpp"
namespace memgraph::storage::v3 {
/// This class keeps track of finalized transactions to provide info on the
/// oldest active transaction (minimal transaction ID which could still be
/// active).
///
/// Basically, it is a set which, at the beginning, contains all transaction
/// IDs and supports two operations: remove an ID from the set (\ref
/// SetFinished) and retrieve the minimal ID still in the set (\ref
/// OldestActive).
///
/// This class is thread-safe.
class CommitLog final {
public:
// TODO(mtomic): use pool allocator for blocks
CommitLog();
/// Create a commit log which has the oldest active id set to
/// oldest_active
/// @param oldest_active the oldest active id
explicit CommitLog(uint64_t oldest_active);
CommitLog(const CommitLog &) = delete;
CommitLog &operator=(const CommitLog &) = delete;
CommitLog(CommitLog &&) = delete;
CommitLog &operator=(CommitLog &&) = delete;
~CommitLog();
/// Mark a transaction as finished.
/// @throw std::bad_alloc
void MarkFinished(uint64_t id);
/// Retrieve the oldest transaction still not marked as finished.
uint64_t OldestActive();
private:
static constexpr uint64_t kBlockSize = 8192;
static constexpr uint64_t kIdsInField = sizeof(uint64_t) * 8;
static constexpr uint64_t kIdsInBlock = kBlockSize * kIdsInField;
struct Block {
Block *next{nullptr};
uint64_t field[kBlockSize]{};
};
void UpdateOldestActive();
/// @throw std::bad_alloc
Block *FindOrCreateBlock(uint64_t id);
Block *head_{nullptr};
uint64_t head_start_{0};
uint64_t next_start_{0};
uint64_t oldest_active_{0};
utils::SpinLock lock_;
utils::Allocator<Block> allocator_;
};
} // namespace memgraph::storage::v3

60
src/storage/v3/config.hpp Normal file
View File

@ -0,0 +1,60 @@
// 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 <chrono>
#include <cstdint>
#include <filesystem>
#include "storage/v3/isolation_level.hpp"
#include "storage/v3/transaction.hpp"
namespace memgraph::storage::v3 {
/// Pass this class to the \ref Storage constructor to change the behavior of
/// the storage. This class also defines the default behavior.
struct Config {
struct Gc {
enum class Type { NONE, PERIODIC };
Type type{Type::PERIODIC};
std::chrono::milliseconds interval{std::chrono::milliseconds(1000)};
} gc;
struct Items {
bool properties_on_edges{true};
} items;
struct Durability {
enum class SnapshotWalMode { DISABLED, PERIODIC_SNAPSHOT, PERIODIC_SNAPSHOT_WITH_WAL };
std::filesystem::path storage_directory{"storage"};
bool recover_on_startup{false};
SnapshotWalMode snapshot_wal_mode{SnapshotWalMode::DISABLED};
std::chrono::milliseconds snapshot_interval{std::chrono::minutes(2)};
uint64_t snapshot_retention_count{3};
uint64_t wal_file_size_kibibytes{static_cast<uint64_t>(20 * 1024)};
uint64_t wal_file_flush_every_n_tx{100000};
bool snapshot_on_exit{false};
} durability;
struct Transaction {
IsolationLevel isolation_level{IsolationLevel::SNAPSHOT_ISOLATION};
} transaction;
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,414 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/constraints.hpp"
#include <algorithm>
#include <cstring>
#include <map>
#include "storage/v3/mvcc.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage::v3 {
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) {
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;
}
return it - property_array.values;
}
/// Helper function for validating unique constraints on commit. Returns true if
/// the last committed version of the given vertex contains the given label and
/// set of property values. This function should be called when commit lock is
/// active.
bool LastCommittedVersionHasLabelProperty(const Vertex &vertex, LabelId label, const std::set<PropertyId> &properties,
const std::vector<PropertyValue> &value_array, const Transaction &transaction,
uint64_t commit_timestamp) {
MG_ASSERT(properties.size() == value_array.size(), "Invalid database state!");
PropertyIdArray property_array(properties.size());
bool current_value_equal_to_value[kUniqueConstraintsMaxProperties];
memset(current_value_equal_to_value, 0, sizeof(current_value_equal_to_value));
// Since the commit lock is active, any transaction that tries to write to
// a vertex which is part of the given `transaction` will result in a
// serialization error. But, note that the given `vertex`'s data does not have
// to be modified in the current `transaction`, meaning that a guard lock to
// access vertex's data is still necessary because another active transaction
// could modify it in the meantime.
Delta *delta{nullptr};
bool deleted{false};
bool has_label{false};
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
delta = vertex.delta;
deleted = vertex.deleted;
has_label = utils::Contains(vertex.labels, label);
size_t i = 0;
for (const auto &property : properties) {
current_value_equal_to_value[i] = vertex.properties.IsPropertyEqual(property, value_array[i]);
property_array.values[i] = property;
i++;
}
}
while (delta != nullptr) {
auto ts = delta->timestamp->load(std::memory_order_acquire);
if (ts < commit_timestamp || ts == transaction.transaction_id) {
break;
}
switch (delta->action) {
case Delta::Action::SET_PROPERTY: {
auto pos = FindPropertyPosition(property_array, delta->property.key);
if (pos) {
current_value_equal_to_value[*pos] = delta->property.value == value_array[*pos];
}
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;
}
delta = delta->next.load(std::memory_order_acquire);
}
for (size_t i = 0; i < properties.size(); ++i) {
if (!current_value_equal_to_value[i]) {
return false;
}
}
return !deleted && has_label;
}
/// Helper function for unique constraint garbage collection. Returns true if
/// there's a reachable version of the vertex that has the given label and
/// property values.
bool AnyVersionHasLabelProperty(const Vertex &vertex, LabelId label, const std::set<PropertyId> &properties,
const std::vector<PropertyValue> &values, uint64_t timestamp) {
MG_ASSERT(properties.size() == values.size(), "Invalid database state!");
PropertyIdArray property_array(properties.size());
bool current_value_equal_to_value[kUniqueConstraintsMaxProperties];
memset(current_value_equal_to_value, 0, sizeof(current_value_equal_to_value));
bool has_label{false};
bool deleted{false};
Delta *delta{nullptr};
{
std::lock_guard<utils::SpinLock> guard(vertex.lock);
has_label = utils::Contains(vertex.labels, label);
deleted = vertex.deleted;
delta = vertex.delta;
size_t i = 0;
for (const auto &property : properties) {
current_value_equal_to_value[i] = vertex.properties.IsPropertyEqual(property, values[i]);
property_array.values[i] = property;
i++;
}
}
{
bool all_values_match = true;
for (size_t i = 0; i < values.size(); ++i) {
if (!current_value_equal_to_value[i]) {
all_values_match = false;
break;
}
}
if (!deleted && has_label && all_values_match) {
return true;
}
}
while (delta != nullptr) {
auto ts = delta->timestamp->load(std::memory_order_acquire);
if (ts < timestamp) {
break;
}
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: {
auto pos = FindPropertyPosition(property_array, delta->property.key);
if (pos) {
current_value_equal_to_value[*pos] = delta->property.value == values[*pos];
}
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;
}
bool all_values_match = true;
for (size_t i = 0; i < values.size(); ++i) {
if (!current_value_equal_to_value[i]) {
all_values_match = false;
break;
}
}
if (!deleted && has_label && all_values_match) {
return true;
}
delta = delta->next.load(std::memory_order_acquire);
}
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) const {
if (values < rhs.values) {
return true;
}
if (rhs.values < values) {
return false;
}
return std::make_tuple(vertex, timestamp) < std::make_tuple(rhs.vertex, rhs.timestamp);
}
bool UniqueConstraints::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) const { return values < rhs; }
bool UniqueConstraints::Entry::operator==(const std::vector<PropertyValue> &rhs) const { return values == rhs; }
void UniqueConstraints::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);
if (values) {
auto acc = storage.access();
acc.insert(Entry{std::move(*values), vertex, tx.start_timestamp});
}
}
}
utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> UniqueConstraints::CreateConstraint(
LabelId label, const std::set<PropertyId> &properties, utils::SkipList<Vertex>::Accessor vertices) {
if (properties.empty()) {
return CreationStatus::EMPTY_PROPERTIES;
}
if (properties.size() > kUniqueConstraintsMaxProperties) {
return CreationStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED;
}
auto [constraint, emplaced] =
constraints_.emplace(std::piecewise_construct, std::forward_as_tuple(label, properties), std::forward_as_tuple());
if (!emplaced) {
// Constraint already exists.
return CreationStatus::ALREADY_EXISTS;
}
bool violation_found = false;
{
auto acc = constraint->second.access();
for (const Vertex &vertex : vertices) {
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
continue;
}
auto values = ExtractPropertyValues(vertex, properties);
if (!values) {
continue;
}
// Check whether there already is a vertex with the same values for the
// given label and property.
auto it = acc.find_equal_or_greater(*values);
if (it != acc.end() && it->values == *values) {
violation_found = true;
break;
}
acc.insert(Entry{std::move(*values), &vertex, 0});
}
}
if (violation_found) {
// In the case of the violation, storage for the current constraint has to
// be removed.
constraints_.erase(constraint);
return ConstraintViolation{ConstraintViolation::Type::UNIQUE, label, properties};
}
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;
}
if (constraints_.erase({label, properties}) > 0) {
return UniqueConstraints::DeletionStatus::SUCCESS;
}
return UniqueConstraints::DeletionStatus::NOT_FOUND;
}
std::optional<ConstraintViolation> UniqueConstraints::Validate(const Vertex &vertex, const Transaction &tx,
uint64_t commit_timestamp) const {
if (vertex.deleted) {
return std::nullopt;
}
for (const auto &[label_props, storage] : constraints_) {
const auto &label = label_props.first;
const auto &properties = label_props.second;
if (!utils::Contains(vertex.labels, label)) {
continue;
}
auto value_array = ExtractPropertyValues(vertex, properties);
if (!value_array) {
continue;
}
auto acc = storage.access();
auto it = acc.find_equal_or_greater(*value_array);
for (; it != acc.end(); ++it) {
if (*value_array < it->values) {
break;
}
// The `vertex` that is going to be committed violates a unique constraint
// if it's different than a vertex indexed in the list of constraints and
// has the same label and property value as the last committed version of
// the vertex from the list.
if (&vertex != it->vertex &&
LastCommittedVersionHasLabelProperty(*it->vertex, label, properties, *value_array, tx, commit_timestamp)) {
return ConstraintViolation{ConstraintViolation::Type::UNIQUE, label, properties};
}
}
}
return std::nullopt;
}
std::vector<std::pair<LabelId, std::set<PropertyId>>> UniqueConstraints::ListConstraints() const {
std::vector<std::pair<LabelId, std::set<PropertyId>>> ret;
ret.reserve(constraints_.size());
for (const auto &[label_props, _] : constraints_) {
ret.push_back(label_props);
}
return ret;
}
void UniqueConstraints::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();) {
auto next_it = it;
++next_it;
if (it->timestamp >= oldest_active_start_timestamp) {
it = next_it;
continue;
}
if ((next_it != acc.end() && it->vertex == next_it->vertex && it->values == next_it->values) ||
!AnyVersionHasLabelProperty(*it->vertex, label_props.first, label_props.second, it->values,
oldest_active_start_timestamp)) {
acc.remove(*it);
}
it = next_it;
}
}
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,199 @@
// 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/v3/id_types.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/vertex.hpp"
#include "utils/logging.hpp"
#include "utils/result.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3 {
// 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) const;
bool operator==(const Entry &rhs) const;
bool operator<(const std::vector<PropertyValue> &rhs) const;
bool operator==(const std::vector<PropertyValue> &rhs) const;
};
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::v3

250
src/storage/v3/delta.hpp Normal file
View File

@ -0,0 +1,250 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include "storage/v3/edge_ref.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/property_value.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage::v3 {
// Forward declarations because we only store pointers here.
struct Vertex;
struct Edge;
struct Delta;
// This class stores one of three pointers (`Delta`, `Vertex` and `Edge`)
// without using additional memory for storing the type. The type is stored in
// the pointer itself in the lower bits. All of those structures contain large
// items in themselves (e.g. `uint64_t`) that require the pointer to be aligned
// to their size (for `uint64_t` it is 8). That means that the pointer will
// always be a multiple of 8 which implies that the lower 3 bits of the pointer
// will always be 0. We can use those 3 bits to store information about the type
// of the pointer stored (2 bits).
class PreviousPtr {
private:
static constexpr uintptr_t kDelta = 0b01UL;
static constexpr uintptr_t kVertex = 0b10UL;
static constexpr uintptr_t kEdge = 0b11UL;
static constexpr uintptr_t kMask = 0b11UL;
public:
enum class Type {
NULLPTR,
DELTA,
VERTEX,
EDGE,
};
struct Pointer {
Pointer() = default;
explicit Pointer(Delta *delta) : type(Type::DELTA), delta(delta) {}
explicit Pointer(Vertex *vertex) : type(Type::VERTEX), vertex(vertex) {}
explicit Pointer(Edge *edge) : type(Type::EDGE), edge(edge) {}
Type type{Type::NULLPTR};
Delta *delta{nullptr};
Vertex *vertex{nullptr};
Edge *edge{nullptr};
};
PreviousPtr() : storage_(0) {}
PreviousPtr(const PreviousPtr &other) noexcept : storage_(other.storage_.load(std::memory_order_acquire)) {}
PreviousPtr(PreviousPtr &&) = delete;
PreviousPtr &operator=(const PreviousPtr &) = delete;
PreviousPtr &operator=(PreviousPtr &&) = delete;
~PreviousPtr() = default;
Pointer Get() const {
uintptr_t value = storage_.load(std::memory_order_acquire);
if (value == 0) {
return {};
}
uintptr_t type = value & kMask;
if (type == kDelta) {
// NOLINTNEXTLINE(performance-no-int-to-ptr)
return Pointer{reinterpret_cast<Delta *>(value & ~kMask)};
}
if (type == kVertex) {
// NOLINTNEXTLINE(performance-no-int-to-ptr)
return Pointer{reinterpret_cast<Vertex *>(value & ~kMask)};
}
if (type == kEdge) {
// NOLINTNEXTLINE(performance-no-int-to-ptr)
return Pointer{reinterpret_cast<Edge *>(value & ~kMask)};
}
LOG_FATAL("Invalid pointer type!");
}
void Set(Delta *delta) {
auto value = reinterpret_cast<uintptr_t>(delta);
MG_ASSERT((value & kMask) == 0, "Invalid pointer!");
storage_.store(value | kDelta, std::memory_order_release);
}
void Set(Vertex *vertex) {
auto value = reinterpret_cast<uintptr_t>(vertex);
MG_ASSERT((value & kMask) == 0, "Invalid pointer!");
storage_.store(value | kVertex, std::memory_order_release);
}
void Set(Edge *edge) {
auto value = reinterpret_cast<uintptr_t>(edge);
MG_ASSERT((value & kMask) == 0, "Invalid pointer!");
storage_.store(value | kEdge, std::memory_order_release);
}
private:
std::atomic<uintptr_t> storage_;
};
inline bool operator==(const PreviousPtr::Pointer &a, const PreviousPtr::Pointer &b) {
if (a.type != b.type) return false;
switch (a.type) {
case PreviousPtr::Type::VERTEX:
return a.vertex == b.vertex;
case PreviousPtr::Type::EDGE:
return a.edge == b.edge;
case PreviousPtr::Type::DELTA:
return a.delta == b.delta;
case PreviousPtr::Type::NULLPTR:
return b.type == PreviousPtr::Type::NULLPTR;
}
}
inline bool operator!=(const PreviousPtr::Pointer &a, const PreviousPtr::Pointer &b) { return !(a == b); }
struct Delta {
enum class Action {
// Used for both Vertex and Edge
DELETE_OBJECT,
RECREATE_OBJECT,
SET_PROPERTY,
// Used only for Vertex
ADD_LABEL,
REMOVE_LABEL,
ADD_IN_EDGE,
ADD_OUT_EDGE,
REMOVE_IN_EDGE,
REMOVE_OUT_EDGE,
};
// Used for both Vertex and Edge
struct DeleteObjectTag {};
struct RecreateObjectTag {};
struct SetPropertyTag {};
// Used only for Vertex
struct AddLabelTag {};
struct RemoveLabelTag {};
struct AddInEdgeTag {};
struct AddOutEdgeTag {};
struct RemoveInEdgeTag {};
struct RemoveOutEdgeTag {};
Delta(DeleteObjectTag /*unused*/, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::DELETE_OBJECT), timestamp(timestamp), command_id(command_id) {}
Delta(RecreateObjectTag /*unused*/, std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::RECREATE_OBJECT), timestamp(timestamp), command_id(command_id) {}
Delta(AddLabelTag /*unused*/, 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 /*unused*/, 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 /*unused*/, 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 /*unused*/, 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 /*unused*/, 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 /*unused*/, 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 /*unused*/, EdgeTypeId edge_type, Vertex *vertex, EdgeRef edge,
std::atomic<uint64_t> *timestamp, uint64_t command_id)
: action(Action::REMOVE_OUT_EDGE),
timestamp(timestamp),
command_id(command_id),
vertex_edge({edge_type, vertex, edge}) {}
Delta(const Delta &) = delete;
Delta(Delta &&) = delete;
Delta &operator=(const Delta &) = delete;
Delta &operator=(Delta &&) = delete;
~Delta() {
switch (action) {
case Action::DELETE_OBJECT:
case Action::RECREATE_OBJECT:
case Action::ADD_LABEL:
case Action::REMOVE_LABEL:
case Action::ADD_IN_EDGE:
case Action::ADD_OUT_EDGE:
case Action::REMOVE_IN_EDGE:
case Action::REMOVE_OUT_EDGE:
break;
case Action::SET_PROPERTY:
property.value.~PropertyValue();
break;
}
}
Action action;
// TODO: optimize with in-place copy
std::atomic<uint64_t> *timestamp;
uint64_t command_id;
PreviousPtr prev;
std::atomic<Delta *> next{nullptr};
union {
LabelId label;
struct {
PropertyId key;
PropertyValue value;
} property;
struct {
EdgeTypeId edge_type;
Vertex *vertex;
EdgeRef edge;
} vertex_edge;
};
};
static_assert(alignof(Delta) >= 8, "The Delta should be aligned to at least 8!");
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,347 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/durability/durability.hpp"
#include <pwd.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <unistd.h>
#include <cerrno>
#include <cstring>
#include <algorithm>
#include <tuple>
#include <utility>
#include <vector>
#include "storage/v3/durability/paths.hpp"
#include "storage/v3/durability/snapshot.hpp"
#include "storage/v3/durability/wal.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
#include "utils/message.hpp"
namespace memgraph::storage::v3::durability {
void VerifyStorageDirectoryOwnerAndProcessUserOrDie(const std::filesystem::path &storage_directory) {
// Get the process user ID.
auto process_euid = geteuid();
// Get the data directory owner ID.
struct stat statbuf;
auto ret = stat(storage_directory.c_str(), &statbuf);
if (ret != 0 && errno == ENOENT) {
// The directory doesn't currently exist.
return;
}
MG_ASSERT(ret == 0, "Couldn't get stat for '{}' because of: {} ({})", storage_directory, strerror(errno), errno);
auto directory_owner = statbuf.st_uid;
auto get_username = [](auto uid) {
auto info = getpwuid(uid);
if (!info) return std::to_string(uid);
return std::string(info->pw_name);
};
auto user_process = get_username(process_euid);
auto user_directory = get_username(directory_owner);
MG_ASSERT(process_euid == directory_owner,
"The process is running as user {}, but the data directory is "
"owned by user {}. Please start the process as user {}!",
user_process, user_directory, user_directory);
}
std::vector<SnapshotDurabilityInfo> GetSnapshotFiles(const std::filesystem::path &snapshot_directory,
const std::string_view uuid) {
std::vector<SnapshotDurabilityInfo> snapshot_files;
std::error_code error_code;
if (utils::DirExists(snapshot_directory)) {
for (const auto &item : std::filesystem::directory_iterator(snapshot_directory, error_code)) {
if (!item.is_regular_file()) continue;
try {
auto info = ReadSnapshotInfo(item.path());
if (uuid.empty() || info.uuid == uuid) {
snapshot_files.emplace_back(item.path(), std::move(info.uuid), info.start_timestamp);
}
} catch (const RecoveryFailure &) {
continue;
}
}
MG_ASSERT(!error_code, "Couldn't recover data because an error occurred: {}!", error_code.message());
}
return snapshot_files;
}
std::optional<std::vector<WalDurabilityInfo>> GetWalFiles(const std::filesystem::path &wal_directory,
const std::string_view uuid,
const std::optional<size_t> current_seq_num) {
if (!utils::DirExists(wal_directory)) return std::nullopt;
std::vector<WalDurabilityInfo> wal_files;
std::error_code error_code;
for (const auto &item : std::filesystem::directory_iterator(wal_directory, error_code)) {
if (!item.is_regular_file()) continue;
try {
auto info = ReadWalInfo(item.path());
if ((uuid.empty() || info.uuid == uuid) && (!current_seq_num || info.seq_num < *current_seq_num))
wal_files.emplace_back(info.seq_num, info.from_timestamp, info.to_timestamp, std::move(info.uuid),
std::move(info.epoch_id), item.path());
} catch (const RecoveryFailure &e) {
spdlog::warn("Failed to read {}", item.path());
continue;
}
}
MG_ASSERT(!error_code, "Couldn't recover data because an error occurred: {}!", error_code.message());
std::sort(wal_files.begin(), wal_files.end());
return std::move(wal_files);
}
// Function used to recover all discovered indices and constraints. The
// indices and constraints must be recovered after the data recovery is done
// to ensure that the indices and constraints are consistent at the end of the
// recovery process.
void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_constraints, Indices *indices,
Constraints *constraints, utils::SkipList<Vertex> *vertices) {
spdlog::info("Recreating indices from metadata.");
// 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()))
throw RecoveryFailure("The label index must be created here!");
spdlog::info("A label index is recreated from metadata.");
}
spdlog::info("Label indices are recreated.");
// Recover label+property indices.
spdlog::info("Recreating {} label+property indices from metadata.",
indices_constraints.indices.label_property.size());
for (const auto &item : indices_constraints.indices.label_property) {
if (!indices->label_property_index.CreateIndex(item.first, item.second, vertices->access()))
throw RecoveryFailure("The label+property index must be created here!");
spdlog::info("A label+property index is recreated from metadata.");
}
spdlog::info("Label+property indices are recreated.");
spdlog::info("Indices are recreated.");
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!");
spdlog::info("A existence constraint is recreated from metadata.");
}
spdlog::info("Existence constraints are recreated from metadata.");
// 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());
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.");
}
spdlog::info("Unique constraints are recreated from metadata.");
spdlog::info("Constraints are recreated from metadata.");
}
std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_directory,
const std::filesystem::path &wal_directory, std::string *uuid,
std::string *epoch_id,
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,
uint64_t *wal_seq_num) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
spdlog::info("Recovering persisted data using snapshot ({}) and WAL directory ({}).", snapshot_directory,
wal_directory);
if (!utils::DirExists(snapshot_directory) && !utils::DirExists(wal_directory)) {
spdlog::warn(utils::MessageWithLink("Snapshot or WAL directory don't exist, there is nothing to recover.",
"https://memgr.ph/durability"));
return std::nullopt;
}
auto snapshot_files = GetSnapshotFiles(snapshot_directory);
RecoveryInfo recovery_info;
RecoveredIndicesAndConstraints indices_constraints;
std::optional<uint64_t> snapshot_timestamp;
if (!snapshot_files.empty()) {
spdlog::info("Try recovering from snapshot directory {}.", snapshot_directory);
// Order the files by name
std::sort(snapshot_files.begin(), snapshot_files.end());
// UUID used for durability is the UUID of the last snapshot file.
*uuid = snapshot_files.back().uuid;
std::optional<RecoveredSnapshot> recovered_snapshot;
for (auto it = snapshot_files.rbegin(); it != snapshot_files.rend(); ++it) {
const auto &[path, file_uuid, _] = *it;
if (file_uuid != *uuid) {
spdlog::warn("The snapshot file {} isn't related to the latest snapshot file!", path);
continue;
}
spdlog::info("Starting snapshot recovery from {}.", path);
try {
recovered_snapshot = LoadSnapshot(path, vertices, edges, epoch_history, name_id_mapper, edge_count, items);
spdlog::info("Snapshot recovery successful!");
break;
} catch (const RecoveryFailure &e) {
spdlog::warn("Couldn't recover snapshot from {} because of: {}.", path, e.what());
continue;
}
}
MG_ASSERT(recovered_snapshot,
"The database is configured to recover on startup, but couldn't "
"recover using any of the specified snapshots! Please inspect them "
"and restart the database.");
recovery_info = recovered_snapshot->recovery_info;
indices_constraints = std::move(recovered_snapshot->indices_constraints);
snapshot_timestamp = recovered_snapshot->snapshot_info.start_timestamp;
*epoch_id = std::move(recovered_snapshot->snapshot_info.epoch_id);
if (!utils::DirExists(wal_directory)) {
RecoverIndicesAndConstraints(indices_constraints, indices, constraints, vertices);
return recovered_snapshot->recovery_info;
}
} else {
spdlog::info("No snapshot file was found, collecting information from WAL directory {}.", wal_directory);
std::error_code error_code;
if (!utils::DirExists(wal_directory)) return std::nullopt;
// We use this smaller struct that contains only a subset of information
// necessary for the rest of the recovery function.
// Also, the struct is sorted primarily on the path it contains.
struct WalFileInfo {
explicit WalFileInfo(std::filesystem::path path, std::string uuid, std::string epoch_id)
: path(std::move(path)), uuid(std::move(uuid)), epoch_id(std::move(epoch_id)) {}
std::filesystem::path path;
std::string uuid;
std::string epoch_id;
// NOLINTNEXTLINE(modernize-use-nullptr): bug in clang-tidy
auto operator<=>(const WalFileInfo &) const = default;
};
std::vector<WalFileInfo> wal_files;
for (const auto &item : std::filesystem::directory_iterator(wal_directory, error_code)) {
if (!item.is_regular_file()) continue;
try {
auto info = ReadWalInfo(item.path());
wal_files.emplace_back(item.path(), std::move(info.uuid), std::move(info.epoch_id));
} catch (const RecoveryFailure &e) {
continue;
}
}
MG_ASSERT(!error_code, "Couldn't recover data because an error occurred: {}!", error_code.message());
if (wal_files.empty()) {
spdlog::warn(utils::MessageWithLink("No snapshot or WAL file found.", "https://memgr.ph/durability"));
return std::nullopt;
}
std::sort(wal_files.begin(), wal_files.end());
// UUID used for durability is the UUID of the last WAL file.
// Same for the epoch id.
*uuid = std::move(wal_files.back().uuid);
*epoch_id = std::move(wal_files.back().epoch_id);
}
auto maybe_wal_files = GetWalFiles(wal_directory, *uuid);
if (!maybe_wal_files) {
spdlog::warn(
utils::MessageWithLink("Couldn't get WAL file info from the WAL directory.", "https://memgr.ph/durability"));
return std::nullopt;
}
// Array of all discovered WAL files, ordered by sequence number.
auto &wal_files = *maybe_wal_files;
// By this point we should have recovered from a snapshot, or we should have
// found some WAL files to recover from in the above `else`. This is just a
// sanity check to circumvent the following case: The database didn't recover
// from a snapshot, the above `else` triggered to find the recovery UUID from
// a WAL file. The above `else` has an early exit in case there are no WAL
// files. Because we reached this point there must have been some WAL files
// and we must have some WAL files after this second WAL directory iteration.
MG_ASSERT(snapshot_timestamp || !wal_files.empty(),
"The database didn't recover from a snapshot and didn't find any WAL "
"files that match the last WAL file!");
if (!wal_files.empty()) {
spdlog::info("Checking WAL files.");
{
const auto &first_wal = wal_files[0];
if (first_wal.seq_num != 0) {
// We don't have all WAL files. We need to see whether we need them all.
if (!snapshot_timestamp) {
// We didn't recover from a snapshot and we must have all WAL files
// starting from the first one (seq_num == 0) to be able to recover
// data from them.
LOG_FATAL(
"There are missing prefix WAL files and data can't be "
"recovered without them!");
} else if (first_wal.from_timestamp >= *snapshot_timestamp) {
// We recovered from a snapshot and we must have at least one WAL file
// that has at least one delta that was created before the snapshot in order to
// verify that nothing is missing from the beginning of the WAL chain.
LOG_FATAL(
"You must have at least one WAL file that contains at least one "
"delta that was created before the snapshot file!");
}
}
}
std::optional<uint64_t> previous_seq_num;
auto last_loaded_timestamp = snapshot_timestamp;
spdlog::info("Trying to load WAL files.");
for (auto &wal_file : wal_files) {
if (previous_seq_num && (wal_file.seq_num - *previous_seq_num) > 1) {
LOG_FATAL("You are missing a WAL file with the sequence number {}!", *previous_seq_num + 1);
}
previous_seq_num = wal_file.seq_num;
if (wal_file.epoch_id != *epoch_id) {
// This way we skip WALs finalized only because of role change.
// We can also set the last timestamp to 0 if last loaded timestamp
// is nullopt as this can only happen if the WAL file with seq = 0
// does not contain any deltas and we didn't find any snapshots.
if (last_loaded_timestamp) {
epoch_history->emplace_back(wal_file.epoch_id, *last_loaded_timestamp);
}
*epoch_id = std::move(wal_file.epoch_id);
}
try {
auto info = LoadWal(wal_file.path, &indices_constraints, last_loaded_timestamp, vertices, edges, name_id_mapper,
edge_count, 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);
recovery_info.last_commit_timestamp = info.last_commit_timestamp;
} catch (const RecoveryFailure &e) {
LOG_FATAL("Couldn't recover WAL deltas from {} because of: {}", wal_file.path, e.what());
}
if (recovery_info.next_timestamp != 0) {
last_loaded_timestamp.emplace(recovery_info.next_timestamp - 1);
}
}
// The sequence number needs to be recovered even though `LoadWal` didn't
// load any deltas from that file.
*wal_seq_num = *previous_seq_num + 1;
spdlog::info("All necessary WAL files are loaded successfully.");
}
RecoverIndicesAndConstraints(indices_constraints, indices, constraints, vertices);
return recovery_info;
}
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,114 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <cstdint>
#include <filesystem>
#include <optional>
#include <string>
#include <variant>
#include "storage/v3/config.hpp"
#include "storage/v3/constraints.hpp"
#include "storage/v3/durability/metadata.hpp"
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/indices.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/vertex.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3::durability {
/// Verifies that the owner of the storage directory is the same user that
/// started the current process. If the verification fails, the process is
/// killed (`CHECK` failure).
void VerifyStorageDirectoryOwnerAndProcessUserOrDie(const std::filesystem::path &storage_directory);
// Used to capture the snapshot's data related to durability
struct SnapshotDurabilityInfo {
explicit SnapshotDurabilityInfo(std::filesystem::path path, std::string uuid, const uint64_t start_timestamp)
: path(std::move(path)), uuid(std::move(uuid)), start_timestamp(start_timestamp) {}
std::filesystem::path path;
std::string uuid;
uint64_t start_timestamp;
auto operator<=>(const SnapshotDurabilityInfo &) const = default;
};
/// Get list of snapshot files with their UUID.
/// @param snapshot_directory Directory containing the Snapshot files.
/// @param uuid UUID of the Snapshot files. If not empty, fetch only Snapshot
/// file with the specified UUID. Otherwise, fetch only Snapshot files in the
/// snapshot_directory.
/// @return List of snapshot files defined with its path and UUID.
std::vector<SnapshotDurabilityInfo> GetSnapshotFiles(const std::filesystem::path &snapshot_directory,
std::string_view uuid = "");
/// Used to capture a WAL's data related to durability
struct WalDurabilityInfo {
explicit WalDurabilityInfo(const uint64_t seq_num, const uint64_t from_timestamp, const uint64_t to_timestamp,
std::string uuid, std::string epoch_id, std::filesystem::path path)
: seq_num(seq_num),
from_timestamp(from_timestamp),
to_timestamp(to_timestamp),
uuid(std::move(uuid)),
epoch_id(std::move(epoch_id)),
path(std::move(path)) {}
uint64_t seq_num;
uint64_t from_timestamp;
uint64_t to_timestamp;
std::string uuid;
std::string epoch_id;
std::filesystem::path path;
auto operator<=>(const WalDurabilityInfo &) const = default;
};
/// Get list of WAL files ordered by the sequence number
/// @param wal_directory Directory containing the WAL files.
/// @param uuid UUID of the WAL files. If not empty, fetch only WAL files
/// with the specified UUID. Otherwise, fetch all WAL files in the
/// wal_directory.
/// @param current_seq_num Sequence number of the WAL file which is currently
/// being written. If specified, load only finalized WAL files, i.e. WAL files
/// with seq_num < current_seq_num.
/// @return List of WAL files. Each WAL file is defined with its sequence
/// number, from timestamp, to timestamp and path.
std::optional<std::vector<WalDurabilityInfo>> GetWalFiles(const std::filesystem::path &wal_directory,
std::string_view uuid = "",
std::optional<size_t> current_seq_num = {});
// Helper function used to recover all discovered indices and constraints. The
// indices and constraints must be recovered after the data recovery is done
// to ensure that the indices and constraints are consistent at the end of the
// recovery process.
/// @throw RecoveryFailure
void RecoverIndicesAndConstraints(const RecoveredIndicesAndConstraints &indices_constraints, Indices *indices,
Constraints *constraints, utils::SkipList<Vertex> *vertices);
/// Recovers data either from a snapshot and/or WAL files.
/// @throw RecoveryFailure
/// @throw std::bad_alloc
std::optional<RecoveryInfo> RecoverData(const std::filesystem::path &snapshot_directory,
const std::filesystem::path &wal_directory, std::string *uuid,
std::string *epoch_id,
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,
uint64_t *wal_seq_num);
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,23 @@
// 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 "utils/exceptions.hpp"
namespace memgraph::storage::v3::durability {
/// Exception used to handle errors during recovery.
class RecoveryFailure : public utils::BasicException {
using utils::BasicException::BasicException;
};
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,106 @@
// 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 <cstdint>
namespace memgraph::storage::v3::durability {
/// Markers that are used to indicate crucial parts of the snapshot/WAL.
/// IMPORTANT: Don't forget to update the list of all markers `kMarkersAll` when
/// you add a new Marker.
enum class Marker : uint8_t {
TYPE_NULL = 0x10,
TYPE_BOOL = 0x11,
TYPE_INT = 0x12,
TYPE_DOUBLE = 0x13,
TYPE_STRING = 0x14,
TYPE_LIST = 0x15,
TYPE_MAP = 0x16,
TYPE_PROPERTY_VALUE = 0x17,
TYPE_TEMPORAL_DATA = 0x18,
SECTION_VERTEX = 0x20,
SECTION_EDGE = 0x21,
SECTION_MAPPER = 0x22,
SECTION_METADATA = 0x23,
SECTION_INDICES = 0x24,
SECTION_CONSTRAINTS = 0x25,
SECTION_DELTA = 0x26,
SECTION_EPOCH_HISTORY = 0x27,
SECTION_OFFSETS = 0x42,
DELTA_VERTEX_CREATE = 0x50,
DELTA_VERTEX_DELETE = 0x51,
DELTA_VERTEX_ADD_LABEL = 0x52,
DELTA_VERTEX_REMOVE_LABEL = 0x53,
DELTA_VERTEX_SET_PROPERTY = 0x54,
DELTA_EDGE_CREATE = 0x55,
DELTA_EDGE_DELETE = 0x56,
DELTA_EDGE_SET_PROPERTY = 0x57,
DELTA_TRANSACTION_END = 0x58,
DELTA_LABEL_INDEX_CREATE = 0x59,
DELTA_LABEL_INDEX_DROP = 0x5a,
DELTA_LABEL_PROPERTY_INDEX_CREATE = 0x5b,
DELTA_LABEL_PROPERTY_INDEX_DROP = 0x5c,
DELTA_EXISTENCE_CONSTRAINT_CREATE = 0x5d,
DELTA_EXISTENCE_CONSTRAINT_DROP = 0x5e,
DELTA_UNIQUE_CONSTRAINT_CREATE = 0x5f,
DELTA_UNIQUE_CONSTRAINT_DROP = 0x60,
VALUE_FALSE = 0x00,
VALUE_TRUE = 0xff,
};
/// List of all available markers.
/// IMPORTANT: Don't forget to update this list when you add a new Marker.
static const Marker kMarkersAll[] = {
Marker::TYPE_NULL,
Marker::TYPE_BOOL,
Marker::TYPE_INT,
Marker::TYPE_DOUBLE,
Marker::TYPE_STRING,
Marker::TYPE_LIST,
Marker::TYPE_MAP,
Marker::TYPE_TEMPORAL_DATA,
Marker::TYPE_PROPERTY_VALUE,
Marker::SECTION_VERTEX,
Marker::SECTION_EDGE,
Marker::SECTION_MAPPER,
Marker::SECTION_METADATA,
Marker::SECTION_INDICES,
Marker::SECTION_CONSTRAINTS,
Marker::SECTION_DELTA,
Marker::SECTION_EPOCH_HISTORY,
Marker::SECTION_OFFSETS,
Marker::DELTA_VERTEX_CREATE,
Marker::DELTA_VERTEX_DELETE,
Marker::DELTA_VERTEX_ADD_LABEL,
Marker::DELTA_VERTEX_REMOVE_LABEL,
Marker::DELTA_VERTEX_SET_PROPERTY,
Marker::DELTA_EDGE_CREATE,
Marker::DELTA_EDGE_DELETE,
Marker::DELTA_EDGE_SET_PROPERTY,
Marker::DELTA_TRANSACTION_END,
Marker::DELTA_LABEL_INDEX_CREATE,
Marker::DELTA_LABEL_INDEX_DROP,
Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE,
Marker::DELTA_LABEL_PROPERTY_INDEX_DROP,
Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE,
Marker::DELTA_EXISTENCE_CONSTRAINT_DROP,
Marker::DELTA_UNIQUE_CONSTRAINT_CREATE,
Marker::DELTA_UNIQUE_CONSTRAINT_DROP,
Marker::VALUE_FALSE,
Marker::VALUE_TRUE,
};
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,75 @@
// 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 <algorithm>
#include <optional>
#include <set>
#include <utility>
#include <vector>
#include "storage/v3/durability/exceptions.hpp"
#include "storage/v3/id_types.hpp"
namespace memgraph::storage::v3::durability {
/// Structure used to hold metadata about the recovered snapshot/WAL.
struct RecoveryInfo {
uint64_t next_vertex_id{0};
uint64_t next_edge_id{0};
uint64_t next_timestamp{0};
// last timestamp read from a WAL file
std::optional<uint64_t> last_commit_timestamp;
};
/// Structure used to track indices and constraints during recovery.
struct RecoveredIndicesAndConstraints {
struct {
std::vector<LabelId> label;
std::vector<std::pair<LabelId, PropertyId>> label_property;
} indices;
struct {
std::vector<std::pair<LabelId, PropertyId>> existence;
std::vector<std::pair<LabelId, std::set<PropertyId>>> unique;
} constraints;
};
// Helper function used to insert indices/constraints into the recovered
// indices/constraints object.
// @throw RecoveryFailure
template <typename TObj>
void AddRecoveredIndexConstraint(std::vector<TObj> *list, TObj obj, const char *error_message) {
auto it = std::find(list->begin(), list->end(), obj);
if (it == list->end()) {
list->push_back(obj);
} else {
throw RecoveryFailure(error_message);
}
}
// Helper function used to remove indices/constraints from the recovered
// indices/constraints object.
// @throw RecoveryFailure
template <typename TObj>
void RemoveRecoveredIndexConstraint(std::vector<TObj> *list, TObj obj, const char *error_message) {
auto it = std::find(list->begin(), list->end(), obj);
if (it != list->end()) {
std::swap(*it, list->back());
list->pop_back();
} else {
throw RecoveryFailure(error_message);
}
}
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,50 @@
// 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 <cstdint>
#include <string>
#include "utils/timestamp.hpp"
namespace memgraph::storage::v3::durability {
static const std::string kSnapshotDirectory{"snapshots"};
static const std::string kWalDirectory{"wal"};
static const std::string kBackupDirectory{".backup"};
static const std::string kLockFile{".lock"};
// This is the prefix used for Snapshot and WAL filenames. It is a timestamp
// format that equals to: YYYYmmddHHMMSSffffff
const std::string kTimestampFormat = "{:04d}{:02d}{:02d}{:02d}{:02d}{:02d}{:06d}";
// Generates the name for a snapshot in a well-defined sortable format with the
// start timestamp appended to the file name.
inline std::string MakeSnapshotName(uint64_t start_timestamp) {
std::string date_str = utils::Timestamp::Now().ToString(kTimestampFormat);
return date_str + "_timestamp_" + std::to_string(start_timestamp);
}
// Generates the name for a WAL file in a well-defined sortable format.
inline std::string MakeWalName() {
std::string date_str = utils::Timestamp::Now().ToString(kTimestampFormat);
return date_str + "_current";
}
// Generates the name for a WAL file in a well-defined sortable format with the
// range of timestamps contained [from, to] appended to the name.
inline std::string RemakeWalName(const std::string &current_name, uint64_t from_timestamp, uint64_t to_timestamp) {
return current_name.substr(0, current_name.size() - 8) + "_from_" + std::to_string(from_timestamp) + "_to_" +
std::to_string(to_timestamp);
}
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,468 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/durability/serialization.hpp"
#include "storage/v3/temporal.hpp"
#include "utils/endian.hpp"
namespace memgraph::storage::v3::durability {
//////////////////////////
// Encoder implementation.
//////////////////////////
namespace {
void WriteSize(Encoder *encoder, uint64_t size) {
size = utils::HostToLittleEndian(size);
encoder->Write(reinterpret_cast<const uint8_t *>(&size), sizeof(size));
}
} // namespace
void Encoder::Initialize(const std::filesystem::path &path, const std::string_view &magic, uint64_t version) {
file_.Open(path, utils::OutputFile::Mode::OVERWRITE_EXISTING);
Write(reinterpret_cast<const uint8_t *>(magic.data()), magic.size());
auto version_encoded = utils::HostToLittleEndian(version);
Write(reinterpret_cast<const uint8_t *>(&version_encoded), sizeof(version_encoded));
}
void Encoder::OpenExisting(const std::filesystem::path &path) {
file_.Open(path, utils::OutputFile::Mode::APPEND_TO_EXISTING);
}
void Encoder::Close() {
if (file_.IsOpen()) {
file_.Close();
}
}
void Encoder::Write(const uint8_t *data, uint64_t size) { file_.Write(data, size); }
void Encoder::WriteMarker(Marker marker) {
auto value = static_cast<uint8_t>(marker);
Write(&value, sizeof(value));
}
void Encoder::WriteBool(bool value) {
WriteMarker(Marker::TYPE_BOOL);
if (value) {
WriteMarker(Marker::VALUE_TRUE);
} else {
WriteMarker(Marker::VALUE_FALSE);
}
}
void Encoder::WriteUint(uint64_t value) {
value = utils::HostToLittleEndian(value);
WriteMarker(Marker::TYPE_INT);
Write(reinterpret_cast<const uint8_t *>(&value), sizeof(value));
}
void Encoder::WriteDouble(double value) {
auto value_uint = utils::MemcpyCast<uint64_t>(value);
value_uint = utils::HostToLittleEndian(value_uint);
WriteMarker(Marker::TYPE_DOUBLE);
Write(reinterpret_cast<const uint8_t *>(&value_uint), sizeof(value_uint));
}
void Encoder::WriteString(const std::string_view &value) {
WriteMarker(Marker::TYPE_STRING);
WriteSize(this, value.size());
Write(reinterpret_cast<const uint8_t *>(value.data()), value.size());
}
void Encoder::WritePropertyValue(const PropertyValue &value) {
WriteMarker(Marker::TYPE_PROPERTY_VALUE);
switch (value.type()) {
case PropertyValue::Type::Null: {
WriteMarker(Marker::TYPE_NULL);
break;
}
case PropertyValue::Type::Bool: {
WriteBool(value.ValueBool());
break;
}
case PropertyValue::Type::Int: {
WriteUint(utils::MemcpyCast<uint64_t>(value.ValueInt()));
break;
}
case PropertyValue::Type::Double: {
WriteDouble(value.ValueDouble());
break;
}
case PropertyValue::Type::String: {
WriteString(value.ValueString());
break;
}
case PropertyValue::Type::List: {
const auto &list = value.ValueList();
WriteMarker(Marker::TYPE_LIST);
WriteSize(this, list.size());
for (const auto &item : list) {
WritePropertyValue(item);
}
break;
}
case PropertyValue::Type::Map: {
const auto &map = value.ValueMap();
WriteMarker(Marker::TYPE_MAP);
WriteSize(this, map.size());
for (const auto &item : map) {
WriteString(item.first);
WritePropertyValue(item.second);
}
break;
}
case PropertyValue::Type::TemporalData: {
const auto temporal_data = value.ValueTemporalData();
WriteMarker(Marker::TYPE_TEMPORAL_DATA);
WriteUint(static_cast<uint64_t>(temporal_data.type));
WriteUint(utils::MemcpyCast<uint64_t>(temporal_data.microseconds));
break;
}
}
}
uint64_t Encoder::GetPosition() { return file_.GetPosition(); }
void Encoder::SetPosition(uint64_t position) {
file_.SetPosition(utils::OutputFile::Position::SET, static_cast<ssize_t>(position));
}
void Encoder::Sync() { file_.Sync(); }
void Encoder::Finalize() {
file_.Sync();
file_.Close();
}
void Encoder::DisableFlushing() { file_.DisableFlushing(); }
void Encoder::EnableFlushing() { file_.EnableFlushing(); }
void Encoder::TryFlushing() { file_.TryFlushing(); }
std::pair<const uint8_t *, size_t> Encoder::CurrentFileBuffer() const { return file_.CurrentBuffer(); }
size_t Encoder::GetSize() { return file_.GetSize(); }
//////////////////////////
// Decoder implementation.
//////////////////////////
namespace {
std::optional<Marker> CastToMarker(uint8_t value) {
for (auto marker : kMarkersAll) {
if (static_cast<uint8_t>(marker) == value) {
return marker;
}
}
return std::nullopt;
}
std::optional<uint64_t> ReadSize(Decoder *decoder) {
uint64_t size{0};
if (!decoder->Read(reinterpret_cast<uint8_t *>(&size), sizeof(size))) return std::nullopt;
size = utils::LittleEndianToHost(size);
return size;
}
} // namespace
std::optional<uint64_t> Decoder::Initialize(const std::filesystem::path &path, const std::string &magic) {
if (!file_.Open(path)) return std::nullopt;
std::string file_magic(magic.size(), '\0');
if (!Read(reinterpret_cast<uint8_t *>(file_magic.data()), file_magic.size())) return std::nullopt;
if (file_magic != magic) return std::nullopt;
uint64_t version_encoded{0};
if (!Read(reinterpret_cast<uint8_t *>(&version_encoded), sizeof(version_encoded))) return std::nullopt;
return utils::LittleEndianToHost(version_encoded);
}
bool Decoder::Read(uint8_t *data, size_t size) { return file_.Read(data, size); }
bool Decoder::Peek(uint8_t *data, size_t size) { return file_.Peek(data, size); }
std::optional<Marker> Decoder::PeekMarker() {
uint8_t value{0};
if (!Peek(&value, sizeof(value))) return std::nullopt;
auto marker = CastToMarker(value);
if (!marker) return std::nullopt;
return *marker;
}
std::optional<Marker> Decoder::ReadMarker() {
uint8_t value{0};
if (!Read(&value, sizeof(value))) return std::nullopt;
auto marker = CastToMarker(value);
if (!marker) return std::nullopt;
return *marker;
}
std::optional<bool> Decoder::ReadBool() {
auto marker = ReadMarker();
if (!marker || *marker != Marker::TYPE_BOOL) return std::nullopt;
auto value = ReadMarker();
if (!value || (*value != Marker::VALUE_FALSE && *value != Marker::VALUE_TRUE)) return std::nullopt;
return *value == Marker::VALUE_TRUE;
}
std::optional<uint64_t> Decoder::ReadUint() {
auto marker = ReadMarker();
if (!marker || *marker != Marker::TYPE_INT) return std::nullopt;
uint64_t value{0};
if (!Read(reinterpret_cast<uint8_t *>(&value), sizeof(value))) return std::nullopt;
value = utils::LittleEndianToHost(value);
return value;
}
std::optional<double> Decoder::ReadDouble() {
auto marker = ReadMarker();
if (!marker || *marker != Marker::TYPE_DOUBLE) return std::nullopt;
uint64_t value_int{0};
if (!Read(reinterpret_cast<uint8_t *>(&value_int), sizeof(value_int))) return std::nullopt;
value_int = utils::LittleEndianToHost(value_int);
auto value = utils::MemcpyCast<double>(value_int);
return value;
}
std::optional<std::string> Decoder::ReadString() {
auto marker = ReadMarker();
if (!marker || *marker != Marker::TYPE_STRING) return std::nullopt;
auto size = ReadSize(this);
if (!size) return std::nullopt;
std::string value(*size, '\0');
if (!Read(reinterpret_cast<uint8_t *>(value.data()), *size)) return std::nullopt;
return value;
}
namespace {
std::optional<TemporalData> ReadTemporalData(Decoder &decoder) {
const auto inner_marker = decoder.ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_TEMPORAL_DATA) return std::nullopt;
const auto type = decoder.ReadUint();
if (!type) return std::nullopt;
const auto microseconds = decoder.ReadUint();
if (!microseconds) return std::nullopt;
return TemporalData{static_cast<TemporalType>(*type), utils::MemcpyCast<int64_t>(*microseconds)};
}
} // namespace
std::optional<PropertyValue> Decoder::ReadPropertyValue() {
auto pv_marker = ReadMarker();
if (!pv_marker || *pv_marker != Marker::TYPE_PROPERTY_VALUE) return std::nullopt;
auto marker = PeekMarker();
if (!marker) return std::nullopt;
switch (*marker) {
case Marker::TYPE_NULL: {
auto inner_marker = ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_NULL) return std::nullopt;
return PropertyValue();
}
case Marker::TYPE_BOOL: {
auto value = ReadBool();
if (!value) return std::nullopt;
return PropertyValue(*value);
}
case Marker::TYPE_INT: {
auto value = ReadUint();
if (!value) return std::nullopt;
return PropertyValue(utils::MemcpyCast<int64_t>(*value));
}
case Marker::TYPE_DOUBLE: {
auto value = ReadDouble();
if (!value) return std::nullopt;
return PropertyValue(*value);
}
case Marker::TYPE_STRING: {
auto value = ReadString();
if (!value) return std::nullopt;
return PropertyValue(std::move(*value));
}
case Marker::TYPE_LIST: {
auto inner_marker = ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_LIST) return std::nullopt;
auto size = ReadSize(this);
if (!size) return std::nullopt;
std::vector<PropertyValue> value;
value.reserve(*size);
for (uint64_t i = 0; i < *size; ++i) {
auto item = ReadPropertyValue();
if (!item) return std::nullopt;
value.emplace_back(std::move(*item));
}
return PropertyValue(std::move(value));
}
case Marker::TYPE_MAP: {
auto inner_marker = ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_MAP) return std::nullopt;
auto size = ReadSize(this);
if (!size) return std::nullopt;
std::map<std::string, PropertyValue> value;
for (uint64_t i = 0; i < *size; ++i) {
auto key = ReadString();
if (!key) return std::nullopt;
auto item = ReadPropertyValue();
if (!item) return std::nullopt;
value.emplace(std::move(*key), std::move(*item));
}
return PropertyValue(std::move(value));
}
case Marker::TYPE_TEMPORAL_DATA: {
const auto maybe_temporal_data = ReadTemporalData(*this);
if (!maybe_temporal_data) return std::nullopt;
return PropertyValue(*maybe_temporal_data);
}
case Marker::TYPE_PROPERTY_VALUE:
case Marker::SECTION_VERTEX:
case Marker::SECTION_EDGE:
case Marker::SECTION_MAPPER:
case Marker::SECTION_METADATA:
case Marker::SECTION_INDICES:
case Marker::SECTION_CONSTRAINTS:
case Marker::SECTION_DELTA:
case Marker::SECTION_EPOCH_HISTORY:
case Marker::SECTION_OFFSETS:
case Marker::DELTA_VERTEX_CREATE:
case Marker::DELTA_VERTEX_DELETE:
case Marker::DELTA_VERTEX_ADD_LABEL:
case Marker::DELTA_VERTEX_REMOVE_LABEL:
case Marker::DELTA_VERTEX_SET_PROPERTY:
case Marker::DELTA_EDGE_CREATE:
case Marker::DELTA_EDGE_DELETE:
case Marker::DELTA_EDGE_SET_PROPERTY:
case Marker::DELTA_TRANSACTION_END:
case Marker::DELTA_LABEL_INDEX_CREATE:
case Marker::DELTA_LABEL_INDEX_DROP:
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP:
case Marker::DELTA_UNIQUE_CONSTRAINT_CREATE:
case Marker::DELTA_UNIQUE_CONSTRAINT_DROP:
case Marker::VALUE_FALSE:
case Marker::VALUE_TRUE:
return std::nullopt;
}
}
bool Decoder::SkipString() {
auto marker = ReadMarker();
if (!marker || *marker != Marker::TYPE_STRING) return false;
auto maybe_size = ReadSize(this);
if (!maybe_size) return false;
const uint64_t kBufferSize = 262144;
uint8_t buffer[kBufferSize];
uint64_t size = *maybe_size;
while (size > 0) {
uint64_t to_read = size < kBufferSize ? size : kBufferSize;
if (!Read(reinterpret_cast<uint8_t *>(&buffer), to_read)) return false;
size -= to_read;
}
return true;
}
bool Decoder::SkipPropertyValue() {
auto pv_marker = ReadMarker();
if (!pv_marker || *pv_marker != Marker::TYPE_PROPERTY_VALUE) return false;
auto marker = PeekMarker();
if (!marker) return false;
switch (*marker) {
case Marker::TYPE_NULL: {
auto inner_marker = ReadMarker();
return inner_marker && *inner_marker == Marker::TYPE_NULL;
}
case Marker::TYPE_BOOL: {
return !!ReadBool();
}
case Marker::TYPE_INT: {
return !!ReadUint();
}
case Marker::TYPE_DOUBLE: {
return !!ReadDouble();
}
case Marker::TYPE_STRING: {
return SkipString();
}
case Marker::TYPE_LIST: {
auto inner_marker = ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_LIST) return false;
auto size = ReadSize(this);
if (!size) return false;
for (uint64_t i = 0; i < *size; ++i) {
if (!SkipPropertyValue()) return false;
}
return true;
}
case Marker::TYPE_MAP: {
auto inner_marker = ReadMarker();
if (!inner_marker || *inner_marker != Marker::TYPE_MAP) return false;
auto size = ReadSize(this);
if (!size) return false;
for (uint64_t i = 0; i < *size; ++i) {
if (!SkipString()) return false;
if (!SkipPropertyValue()) return false;
}
return true;
}
case Marker::TYPE_TEMPORAL_DATA: {
return !!ReadTemporalData(*this);
}
case Marker::TYPE_PROPERTY_VALUE:
case Marker::SECTION_VERTEX:
case Marker::SECTION_EDGE:
case Marker::SECTION_MAPPER:
case Marker::SECTION_METADATA:
case Marker::SECTION_INDICES:
case Marker::SECTION_CONSTRAINTS:
case Marker::SECTION_DELTA:
case Marker::SECTION_EPOCH_HISTORY:
case Marker::SECTION_OFFSETS:
case Marker::DELTA_VERTEX_CREATE:
case Marker::DELTA_VERTEX_DELETE:
case Marker::DELTA_VERTEX_ADD_LABEL:
case Marker::DELTA_VERTEX_REMOVE_LABEL:
case Marker::DELTA_VERTEX_SET_PROPERTY:
case Marker::DELTA_EDGE_CREATE:
case Marker::DELTA_EDGE_DELETE:
case Marker::DELTA_EDGE_SET_PROPERTY:
case Marker::DELTA_TRANSACTION_END:
case Marker::DELTA_LABEL_INDEX_CREATE:
case Marker::DELTA_LABEL_INDEX_DROP:
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP:
case Marker::DELTA_UNIQUE_CONSTRAINT_CREATE:
case Marker::DELTA_UNIQUE_CONSTRAINT_DROP:
case Marker::VALUE_FALSE:
case Marker::VALUE_TRUE:
return false;
}
}
std::optional<uint64_t> Decoder::GetSize() { return file_.GetSize(); }
std::optional<uint64_t> Decoder::GetPosition() { return file_.GetPosition(); }
bool Decoder::SetPosition(uint64_t position) {
return !!file_.SetPosition(utils::InputFile::Position::SET, static_cast<ssize_t>(position));
}
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,143 @@
// 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 <cstdint>
#include <filesystem>
#include <string_view>
#include "storage/v3/config.hpp"
#include "storage/v3/durability/marker.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/property_value.hpp"
#include "utils/file.hpp"
namespace memgraph::storage::v3::durability {
/// Encoder interface class. Used to implement streams to different targets
/// (e.g. file and network).
class BaseEncoder {
protected:
BaseEncoder() = default;
~BaseEncoder() = default;
public:
BaseEncoder(const BaseEncoder &) = delete;
BaseEncoder(BaseEncoder &&) = delete;
BaseEncoder &operator=(const BaseEncoder &) = delete;
BaseEncoder &operator=(BaseEncoder &&) = delete;
virtual void WriteMarker(Marker marker) = 0;
virtual void WriteBool(bool value) = 0;
virtual void WriteUint(uint64_t value) = 0;
virtual void WriteDouble(double value) = 0;
virtual void WriteString(const std::string_view &value) = 0;
virtual void WritePropertyValue(const PropertyValue &value) = 0;
};
/// Encoder that is used to generate a snapshot/WAL.
class Encoder final : public BaseEncoder {
public:
void Initialize(const std::filesystem::path &path, const std::string_view &magic, uint64_t version);
void OpenExisting(const std::filesystem::path &path);
void Close();
// Main write function, the only one that is allowed to write to the `file_`
// directly.
void Write(const uint8_t *data, uint64_t size);
void WriteMarker(Marker marker) override;
void WriteBool(bool value) override;
void WriteUint(uint64_t value) override;
void WriteDouble(double value) override;
void WriteString(const std::string_view &value) override;
void WritePropertyValue(const PropertyValue &value) override;
uint64_t GetPosition();
void SetPosition(uint64_t position);
void Sync();
void Finalize();
// Disable flushing of the internal buffer.
void DisableFlushing();
// Enable flushing of the internal buffer.
void EnableFlushing();
// Try flushing the internal buffer.
void TryFlushing();
// Get the current internal buffer with its size.
std::pair<const uint8_t *, size_t> CurrentFileBuffer() const;
// Get the total size of the current file.
size_t GetSize();
private:
utils::OutputFile file_;
};
/// Decoder interface class. Used to implement streams from different sources
/// (e.g. file and network).
class BaseDecoder {
protected:
BaseDecoder() = default;
~BaseDecoder() = default;
public:
BaseDecoder(const BaseDecoder &) = delete;
BaseDecoder(BaseDecoder &&) = delete;
BaseDecoder &operator=(const BaseDecoder &) = delete;
BaseDecoder &operator=(BaseDecoder &&) = delete;
virtual std::optional<Marker> ReadMarker() = 0;
virtual std::optional<bool> ReadBool() = 0;
virtual std::optional<uint64_t> ReadUint() = 0;
virtual std::optional<double> ReadDouble() = 0;
virtual std::optional<std::string> ReadString() = 0;
virtual std::optional<PropertyValue> ReadPropertyValue() = 0;
virtual bool SkipString() = 0;
virtual bool SkipPropertyValue() = 0;
};
/// Decoder that is used to read a generated snapshot/WAL.
class Decoder final : public BaseDecoder {
public:
std::optional<uint64_t> Initialize(const std::filesystem::path &path, const std::string &magic);
// Main read functions, the only one that are allowed to read from the `file_`
// directly.
bool Read(uint8_t *data, size_t size);
bool Peek(uint8_t *data, size_t size);
std::optional<Marker> PeekMarker();
std::optional<Marker> ReadMarker() override;
std::optional<bool> ReadBool() override;
std::optional<uint64_t> ReadUint() override;
std::optional<double> ReadDouble() override;
std::optional<std::string> ReadString() override;
std::optional<PropertyValue> ReadPropertyValue() override;
bool SkipString() override;
bool SkipPropertyValue() override;
std::optional<uint64_t> GetSize();
std::optional<uint64_t> GetPosition();
bool SetPosition(uint64_t position);
private:
utils::InputFile file_;
};
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,983 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/durability/snapshot.hpp"
#include "storage/v3/durability/exceptions.hpp"
#include "storage/v3/durability/paths.hpp"
#include "storage/v3/durability/serialization.hpp"
#include "storage/v3/durability/version.hpp"
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/edge_accessor.hpp"
#include "storage/v3/edge_ref.hpp"
#include "storage/v3/mvcc.hpp"
#include "storage/v3/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/logging.hpp"
#include "utils/message.hpp"
namespace memgraph::storage::v3::durability {
// Snapshot format:
//
// 1) Magic string (non-encoded)
//
// 2) Snapshot version (non-encoded, little-endian)
//
// 3) Section offsets:
// * offset to the first edge in the snapshot (`0` if properties on edges
// are disabled)
// * offset to the first vertex in the snapshot
// * offset to the indices section
// * offset to the constraints section
// * offset to the mapper section
// * offset to the metadata section
//
// 4) Encoded edges (if properties on edges are enabled); each edge is written
// in the following format:
// * gid
// * properties
//
// 5) Encoded vertices; each vertex is written in the following format:
// * gid
// * labels
// * properties
// * in edges
// * edge gid
// * from vertex gid
// * edge type
// * out edges
// * edge gid
// * to vertex gid
// * edge type
//
// 6) Indices
// * label indices
// * label
// * label+property indices
// * label
// * property
//
// 7) Constraints
// * existence constraints
// * label
// * property
// * unique constraints (from version 13)
// * label
// * properties
//
// 8) Name to ID mapper data
// * id to name mappings
// * id
// * name
//
// 9) Metadata
// * storage UUID
// * snapshot transaction start timestamp (required when recovering
// from snapshot combined with WAL to determine what deltas need to be
// applied)
// * number of edges
// * number of vertices
//
// IMPORTANT: When changing snapshot encoding/decoding bump the snapshot/WAL
// version in `version.hpp`.
// Function used to read information about the snapshot file.
SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path) {
// Check magic and version.
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("Invalid snapshot version!");
// Prepare return value.
SnapshotInfo info;
// Read offsets.
{
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_OFFSETS) throw RecoveryFailure("Invalid snapshot data!");
auto snapshot_size = snapshot.GetSize();
if (!snapshot_size) throw RecoveryFailure("Couldn't read data from snapshot!");
auto read_offset = [&snapshot, snapshot_size] {
auto maybe_offset = snapshot.ReadUint();
if (!maybe_offset) throw RecoveryFailure("Invalid snapshot format!");
auto offset = *maybe_offset;
if (offset > *snapshot_size) throw RecoveryFailure("Invalid snapshot format!");
return offset;
};
info.offset_edges = read_offset();
info.offset_vertices = read_offset();
info.offset_indices = read_offset();
info.offset_constraints = read_offset();
info.offset_mapper = read_offset();
info.offset_epoch_history = read_offset();
info.offset_metadata = read_offset();
}
// Read metadata.
{
if (!snapshot.SetPosition(info.offset_metadata)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_METADATA) throw RecoveryFailure("Invalid snapshot data!");
auto maybe_uuid = snapshot.ReadString();
if (!maybe_uuid) throw RecoveryFailure("Invalid snapshot data!");
info.uuid = std::move(*maybe_uuid);
auto maybe_epoch_id = snapshot.ReadString();
if (!maybe_epoch_id) throw RecoveryFailure("Invalid snapshot data!");
info.epoch_id = std::move(*maybe_epoch_id);
auto maybe_timestamp = snapshot.ReadUint();
if (!maybe_timestamp) throw RecoveryFailure("Invalid snapshot data!");
info.start_timestamp = *maybe_timestamp;
auto maybe_edges = snapshot.ReadUint();
if (!maybe_edges) throw RecoveryFailure("Invalid snapshot data!");
info.edges_count = *maybe_edges;
auto maybe_vertices = snapshot.ReadUint();
if (!maybe_vertices) throw RecoveryFailure("Invalid snapshot data!");
info.vertices_count = *maybe_vertices;
}
return info;
}
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) {
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));
// 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);
{
// Recover edges.
auto edge_acc = edges->access();
uint64_t last_edge_gid = 0;
if (snapshot_has_edges) {
spdlog::info("Recovering {} edges.", info.edges_count);
if (!snapshot.SetPosition(info.offset_edges)) throw RecoveryFailure("Couldn't read data from snapshot!");
for (uint64_t i = 0; i < info.edges_count; ++i) {
{
const auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_EDGE) throw RecoveryFailure("Invalid snapshot data!");
}
if (items.properties_on_edges) {
// Insert edge.
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;
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;
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!");
SPDLOG_TRACE("Recovered property \"{}\" with value \"{}\" for edge {}.",
name_id_mapper->IdToName(snapshot_id_map.at(*key)), *value, *gid);
props.SetProperty(get_property_from_id(*key), *value);
}
}
} else {
// 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;
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("Edges are recovered.");
}
// Recover vertices (labels and properties).
if (!snapshot.SetPosition(info.offset_vertices)) throw RecoveryFailure("Couldn't read data from snapshot!");
auto vertex_acc = vertices->access();
uint64_t last_vertex_gid = 0;
spdlog::info("Recovering {} vertices.", info.vertices_count);
for (uint64_t i = 0; i < info.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!");
SPDLOG_TRACE("Recovered label \"{}\" for vertex {}.", name_id_mapper->IdToName(snapshot_id_map.at(*label)),
*gid);
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;
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!");
SPDLOG_TRACE("Recovered property \"{}\" with value \"{}\" for vertex {}.",
name_id_mapper->IdToName(snapshot_id_map.at(*key)), *value, *gid);
props.SetProperty(get_property_from_id(*key), *value);
}
}
// 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("Vertices are recovered.");
// Recover vertices (in/out edges).
spdlog::info("Recovering connectivity.");
if (!snapshot.SetPosition(info.offset_vertices)) throw RecoveryFailure("Couldn't read data from snapshot!");
for (auto &vertex : vertex_acc) {
{
auto marker = snapshot.ReadMarker();
if (!marker || *marker != Marker::SECTION_VERTEX) throw RecoveryFailure("Invalid snapshot data!");
}
spdlog::trace("Recovering connectivity for vertex {}.", vertex.gid.AsUint());
// Check vertex.
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!");
last_edge_gid = std::max(last_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) {
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);
}
}
SPDLOG_TRACE("Recovered inbound edge {} with label \"{}\" from vertex {}.", *edge_gid,
name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), from_vertex->gid.AsUint());
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!");
last_edge_gid = std::max(last_edge_gid, *edge_gid);
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) {
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);
}
}
SPDLOG_TRACE("Recovered outbound edge {} with label \"{}\" to vertex {}.", *edge_gid,
name_id_mapper->IdToName(snapshot_id_map.at(*edge_type)), to_vertex->gid.AsUint());
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->fetch_add(*out_size, std::memory_order_acq_rel);
}
}
spdlog::info("Connectivity is recovered.");
// Set initial values for edge/vertex ID generators.
ret.next_edge_id = last_edge_gid + 1;
ret.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.
ret.next_timestamp = info.start_timestamp + 1;
// Set success flag (to disable cleanup).
success = true;
return {info, ret, 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,
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.
utils::EnsureDirOrDie(snapshot_directory);
// Create snapshot file.
auto path = snapshot_directory / MakeSnapshotName(transaction->start_timestamp);
spdlog::info("Starting snapshot creation to {}", path);
Encoder snapshot;
snapshot.Initialize(path, kSnapshotMagic, kVersion);
// Write placeholder offsets.
uint64_t offset_offsets = 0;
uint64_t offset_edges = 0;
uint64_t offset_vertices = 0;
uint64_t offset_indices = 0;
uint64_t offset_constraints = 0;
uint64_t offset_mapper = 0;
uint64_t offset_metadata = 0;
uint64_t offset_epoch_history = 0;
{
snapshot.WriteMarker(Marker::SECTION_OFFSETS);
offset_offsets = snapshot.GetPosition();
snapshot.WriteUint(offset_edges);
snapshot.WriteUint(offset_vertices);
snapshot.WriteUint(offset_indices);
snapshot.WriteUint(offset_constraints);
snapshot.WriteUint(offset_mapper);
snapshot.WriteUint(offset_epoch_history);
snapshot.WriteUint(offset_metadata);
}
// Object counters.
uint64_t edges_count = 0;
uint64_t vertices_count = 0;
// Mapper data.
std::unordered_set<uint64_t> used_ids;
auto write_mapping = [&snapshot, &used_ids](auto mapping) {
used_ids.insert(mapping.AsUint());
snapshot.WriteUint(mapping.AsUint());
};
// Store all edges.
if (items.properties_on_edges) {
offset_edges = snapshot.GetPosition();
auto acc = edges->access();
for (auto &edge : acc) {
// The edge visibility check must be done here manually because we don't
// allow direct access to the edges through the public API.
bool is_visible = true;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge.lock);
is_visible = !edge.deleted;
delta = edge.delta;
}
ApplyDeltasForRead(transaction, delta, View::OLD, [&is_visible](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
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;
case Delta::Action::RECREATE_OBJECT: {
is_visible = true;
break;
}
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
}
}
});
if (!is_visible) continue;
EdgeRef edge_ref(&edge);
// Here we create an edge accessor that we will use to get the
// properties of the edge. The accessor is created with an invalid
// 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};
// Get edge data.
auto maybe_props = ea.Properties(View::OLD);
MG_ASSERT(maybe_props.HasValue(), "Invalid database state!");
// Store the edge.
{
snapshot.WriteMarker(Marker::SECTION_EDGE);
snapshot.WriteUint(edge.gid.AsUint());
const auto &props = maybe_props.GetValue();
snapshot.WriteUint(props.size());
for (const auto &item : props) {
write_mapping(item.first);
snapshot.WritePropertyValue(item.second);
}
}
++edges_count;
}
}
// Store all vertices.
{
offset_vertices = snapshot.GetPosition();
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);
if (!va) continue;
// Get vertex data.
// TODO (mferencevic): All of these functions could be written into a
// single function so that we traverse the undo deltas only once.
auto maybe_labels = va->Labels(View::OLD);
MG_ASSERT(maybe_labels.HasValue(), "Invalid database state!");
auto maybe_props = va->Properties(View::OLD);
MG_ASSERT(maybe_props.HasValue(), "Invalid database state!");
auto maybe_in_edges = va->InEdges(View::OLD);
MG_ASSERT(maybe_in_edges.HasValue(), "Invalid database state!");
auto maybe_out_edges = va->OutEdges(View::OLD);
MG_ASSERT(maybe_out_edges.HasValue(), "Invalid database state!");
// Store the vertex.
{
snapshot.WriteMarker(Marker::SECTION_VERTEX);
snapshot.WriteUint(vertex.gid.AsUint());
const auto &labels = maybe_labels.GetValue();
snapshot.WriteUint(labels.size());
for (const auto &item : labels) {
write_mapping(item);
}
const auto &props = maybe_props.GetValue();
snapshot.WriteUint(props.size());
for (const auto &item : props) {
write_mapping(item.first);
snapshot.WritePropertyValue(item.second);
}
const auto &in_edges = maybe_in_edges.GetValue();
snapshot.WriteUint(in_edges.size());
for (const auto &item : in_edges) {
snapshot.WriteUint(item.Gid().AsUint());
snapshot.WriteUint(item.FromVertex().Gid().AsUint());
write_mapping(item.EdgeType());
}
const auto &out_edges = maybe_out_edges.GetValue();
snapshot.WriteUint(out_edges.size());
for (const auto &item : out_edges) {
snapshot.WriteUint(item.Gid().AsUint());
snapshot.WriteUint(item.ToVertex().Gid().AsUint());
write_mapping(item.EdgeType());
}
}
++vertices_count;
}
}
// Write indices.
{
offset_indices = snapshot.GetPosition();
snapshot.WriteMarker(Marker::SECTION_INDICES);
// Write label indices.
{
auto label = indices->label_index.ListIndices();
snapshot.WriteUint(label.size());
for (const auto &item : label) {
write_mapping(item);
}
}
// Write label+property indices.
{
auto label_property = indices->label_property_index.ListIndices();
snapshot.WriteUint(label_property.size());
for (const auto &item : label_property) {
write_mapping(item.first);
write_mapping(item.second);
}
}
}
// Write constraints.
{
offset_constraints = snapshot.GetPosition();
snapshot.WriteMarker(Marker::SECTION_CONSTRAINTS);
// Write existence constraints.
{
auto existence = ListExistenceConstraints(*constraints);
snapshot.WriteUint(existence.size());
for (const auto &item : existence) {
write_mapping(item.first);
write_mapping(item.second);
}
}
// Write unique constraints.
{
auto unique = constraints->unique_constraints.ListConstraints();
snapshot.WriteUint(unique.size());
for (const auto &item : unique) {
write_mapping(item.first);
snapshot.WriteUint(item.second.size());
for (const auto &property : item.second) {
write_mapping(property);
}
}
}
}
// Write mapper data.
{
offset_mapper = snapshot.GetPosition();
snapshot.WriteMarker(Marker::SECTION_MAPPER);
snapshot.WriteUint(used_ids.size());
for (auto item : used_ids) {
snapshot.WriteUint(item);
snapshot.WriteString(name_id_mapper->IdToName(item));
}
}
// Write epoch history
{
offset_epoch_history = snapshot.GetPosition();
snapshot.WriteMarker(Marker::SECTION_EPOCH_HISTORY);
snapshot.WriteUint(epoch_history.size());
for (const auto &[epoch_id, last_commit_timestamp] : epoch_history) {
snapshot.WriteString(epoch_id);
snapshot.WriteUint(last_commit_timestamp);
}
}
// Write metadata.
{
offset_metadata = snapshot.GetPosition();
snapshot.WriteMarker(Marker::SECTION_METADATA);
snapshot.WriteString(uuid);
snapshot.WriteString(epoch_id);
snapshot.WriteUint(transaction->start_timestamp);
snapshot.WriteUint(edges_count);
snapshot.WriteUint(vertices_count);
}
// Write true offsets.
{
snapshot.SetPosition(offset_offsets);
snapshot.WriteUint(offset_edges);
snapshot.WriteUint(offset_vertices);
snapshot.WriteUint(offset_indices);
snapshot.WriteUint(offset_constraints);
snapshot.WriteUint(offset_mapper);
snapshot.WriteUint(offset_epoch_history);
snapshot.WriteUint(offset_metadata);
}
// Finalize snapshot file.
snapshot.Finalize();
spdlog::info("Snapshot creation successful!");
// Ensure exactly `snapshot_retention_count` snapshots exist.
std::vector<std::pair<uint64_t, std::filesystem::path>> old_snapshot_files;
{
std::error_code error_code;
for (const auto &item : std::filesystem::directory_iterator(snapshot_directory, error_code)) {
if (!item.is_regular_file()) continue;
if (item.path() == path) continue;
try {
auto info = ReadSnapshotInfo(item.path());
if (info.uuid != uuid) continue;
old_snapshot_files.emplace_back(info.start_timestamp, item.path());
} catch (const RecoveryFailure &e) {
spdlog::warn("Found a corrupt snapshot file {} becuase of: {}", item.path(), e.what());
continue;
}
}
if (error_code) {
spdlog::error(
utils::MessageWithLink("Couldn't ensure that exactly {} snapshots exist because an error occurred: {}.",
snapshot_retention_count, error_code.message(), "https://memgr.ph/snapshots"));
}
std::sort(old_snapshot_files.begin(), old_snapshot_files.end());
if (old_snapshot_files.size() > snapshot_retention_count - 1) {
auto num_to_erase = old_snapshot_files.size() - (snapshot_retention_count - 1);
for (size_t i = 0; i < num_to_erase; ++i) {
const auto &[start_timestamp, snapshot_path] = old_snapshot_files[i];
file_retainer->DeleteFile(snapshot_path);
}
old_snapshot_files.erase(
old_snapshot_files.begin(),
old_snapshot_files.begin() +
static_cast<typename decltype(old_snapshot_files)::iterator::difference_type>(num_to_erase));
}
}
// Ensure that only the absolutely necessary WAL files exist.
if (old_snapshot_files.size() == snapshot_retention_count - 1 && utils::DirExists(wal_directory)) {
std::vector<std::tuple<uint64_t, uint64_t, uint64_t, std::filesystem::path>> wal_files;
std::error_code error_code;
for (const auto &item : std::filesystem::directory_iterator(wal_directory, error_code)) {
if (!item.is_regular_file()) continue;
try {
auto info = ReadWalInfo(item.path());
if (info.uuid != uuid) continue;
wal_files.emplace_back(info.seq_num, info.from_timestamp, info.to_timestamp, item.path());
} catch (const RecoveryFailure &e) {
continue;
}
}
if (error_code) {
spdlog::error(
utils::MessageWithLink("Couldn't ensure that only the absolutely necessary WAL files exist "
"because an error occurred: {}.",
error_code.message(), "https://memgr.ph/snapshots"));
}
std::sort(wal_files.begin(), wal_files.end());
uint64_t snapshot_start_timestamp = transaction->start_timestamp;
if (!old_snapshot_files.empty()) {
snapshot_start_timestamp = old_snapshot_files.front().first;
}
std::optional<uint64_t> pos = 0;
for (uint64_t i = 0; i < wal_files.size(); ++i) {
const auto &[seq_num, from_timestamp, to_timestamp, wal_path] = wal_files[i];
if (from_timestamp <= snapshot_start_timestamp) {
pos = i;
} else {
break;
}
}
if (pos && *pos > 0) {
// We need to leave at least one WAL file that contains deltas that were
// created before the oldest snapshot. Because we always leave at least
// one WAL file that contains deltas before the snapshot, this correctly
// handles the edge case when that one file is the current WAL file that
// is being appended to.
for (uint64_t i = 0; i < *pos; ++i) {
const auto &[seq_num, from_timestamp, to_timestamp, wal_path] = wal_files[i];
file_retainer->DeleteFile(wal_path);
}
}
}
}
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,75 @@
// 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 <cstdint>
#include <filesystem>
#include <string>
#include "storage/v3/config.hpp"
#include "storage/v3/constraints.hpp"
#include "storage/v3/durability/metadata.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/indices.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/vertex.hpp"
#include "utils/file_locker.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3::durability {
/// Structure used to hold information about a snapshot.
struct SnapshotInfo {
uint64_t offset_edges;
uint64_t offset_vertices;
uint64_t offset_indices;
uint64_t offset_constraints;
uint64_t offset_mapper;
uint64_t offset_epoch_history;
uint64_t offset_metadata;
std::string uuid;
std::string epoch_id;
uint64_t start_timestamp;
uint64_t edges_count;
uint64_t vertices_count;
};
/// Structure used to hold information about the snapshot that has been
/// recovered.
struct RecoveredSnapshot {
SnapshotInfo snapshot_info;
RecoveryInfo recovery_info;
RecoveredIndicesAndConstraints indices_constraints;
};
/// Function used to read information about the snapshot file.
/// @throw RecoveryFailure
SnapshotInfo ReadSnapshotInfo(const std::filesystem::path &path);
/// Function used to load the snapshot data into the storage.
/// @throw RecoveryFailure
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);
/// 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,
std::string_view epoch_id, const std::deque<std::pair<std::string, uint64_t>> &epoch_history,
utils::FileRetainer *file_retainer);
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,37 @@
// 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 <cstdint>
#include <string>
#include <type_traits>
namespace memgraph::storage::v3::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 kOldestSupportedVersion{14};
const uint64_t kUniqueConstraintVersion{13};
// Magic values written to the start of a snapshot/WAL file to identify it.
const std::string kSnapshotMagic{"MGsn"};
const std::string kWalMagic{"MGwl"};
static_assert(std::is_same_v<uint8_t, unsigned char>);
// Checks whether the loaded snapshot/WAL version is supported.
inline bool IsVersionSupported(uint64_t version) { return version >= kOldestSupportedVersion && version <= kVersion; }
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,988 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/delta.hpp"
#include "storage/v3/durability/exceptions.hpp"
#include "storage/v3/durability/paths.hpp"
#include "storage/v3/durability/version.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/vertex.hpp"
#include "utils/file_locker.hpp"
#include "utils/logging.hpp"
namespace memgraph::storage::v3::durability {
// WAL format:
//
// 1) Magic string (non-encoded)
//
// 2) WAL version (non-encoded, little-endian)
//
// 3) Section offsets:
// * offset to the metadata section
// * offset to the first delta in the WAL
//
// 4) Metadata
// * storage UUID
// * sequence number (number indicating the sequence position of this WAL
// file)
//
// 5) Encoded deltas; each delta is written in the following format:
// * commit timestamp
// * action (only one of the actions below are encoded)
// * vertex create, vertex delete
// * gid
// * vertex add label, vertex remove label
// * gid
// * label name
// * vertex set property
// * gid
// * property name
// * property value
// * edge create, edge delete
// * gid
// * edge type name
// * from vertex gid
// * to vertex gid
// * edge set property
// * gid
// * property name
// * property value
// * transaction end (marks that the whole transaction is
// stored in the WAL file)
// * label index create, label index drop
// * label name
// * label property index create, label property index drop,
// existence constraint create, existence constraint drop
// * label name
// * property name
// * unique constraint create, unique constraint drop
// * label name
// * property names
//
// IMPORTANT: When changing WAL encoding/decoding bump the snapshot/WAL version
// in `version.hpp`.
namespace {
Marker OperationToMarker(StorageGlobalOperation operation) {
switch (operation) {
case StorageGlobalOperation::LABEL_INDEX_CREATE:
return Marker::DELTA_LABEL_INDEX_CREATE;
case StorageGlobalOperation::LABEL_INDEX_DROP:
return Marker::DELTA_LABEL_INDEX_DROP;
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
return Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE;
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
return Marker::DELTA_LABEL_PROPERTY_INDEX_DROP;
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
return Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE;
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP:
return Marker::DELTA_EXISTENCE_CONSTRAINT_DROP;
case StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
return Marker::DELTA_UNIQUE_CONSTRAINT_CREATE;
case StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP:
return Marker::DELTA_UNIQUE_CONSTRAINT_DROP;
}
}
Marker VertexActionToMarker(Delta::Action action) {
// When converting a Delta to a WAL delta the logic is inverted. That is
// because the Delta's represent undo actions and we want to store redo
// actions.
switch (action) {
case Delta::Action::DELETE_OBJECT:
return Marker::DELTA_VERTEX_CREATE;
case Delta::Action::RECREATE_OBJECT:
return Marker::DELTA_VERTEX_DELETE;
case Delta::Action::SET_PROPERTY:
return Marker::DELTA_VERTEX_SET_PROPERTY;
case Delta::Action::ADD_LABEL:
return Marker::DELTA_VERTEX_REMOVE_LABEL;
case Delta::Action::REMOVE_LABEL:
return Marker::DELTA_VERTEX_ADD_LABEL;
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
return Marker::DELTA_EDGE_DELETE;
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
return Marker::DELTA_EDGE_CREATE;
}
}
// This function convertes a Marker to a WalDeltaData::Type. It checks for the
// validity of the marker and throws if an invalid marker is specified.
// @throw RecoveryFailure
WalDeltaData::Type MarkerToWalDeltaDataType(Marker marker) {
switch (marker) {
case Marker::DELTA_VERTEX_CREATE:
return WalDeltaData::Type::VERTEX_CREATE;
case Marker::DELTA_VERTEX_DELETE:
return WalDeltaData::Type::VERTEX_DELETE;
case Marker::DELTA_VERTEX_ADD_LABEL:
return WalDeltaData::Type::VERTEX_ADD_LABEL;
case Marker::DELTA_VERTEX_REMOVE_LABEL:
return WalDeltaData::Type::VERTEX_REMOVE_LABEL;
case Marker::DELTA_EDGE_CREATE:
return WalDeltaData::Type::EDGE_CREATE;
case Marker::DELTA_EDGE_DELETE:
return WalDeltaData::Type::EDGE_DELETE;
case Marker::DELTA_VERTEX_SET_PROPERTY:
return WalDeltaData::Type::VERTEX_SET_PROPERTY;
case Marker::DELTA_EDGE_SET_PROPERTY:
return WalDeltaData::Type::EDGE_SET_PROPERTY;
case Marker::DELTA_TRANSACTION_END:
return WalDeltaData::Type::TRANSACTION_END;
case Marker::DELTA_LABEL_INDEX_CREATE:
return WalDeltaData::Type::LABEL_INDEX_CREATE;
case Marker::DELTA_LABEL_INDEX_DROP:
return WalDeltaData::Type::LABEL_INDEX_DROP;
case Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE;
case Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
return WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP;
case Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
return WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE;
case Marker::DELTA_EXISTENCE_CONSTRAINT_DROP:
return WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP;
case Marker::DELTA_UNIQUE_CONSTRAINT_CREATE:
return WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE;
case Marker::DELTA_UNIQUE_CONSTRAINT_DROP:
return WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP;
case Marker::TYPE_NULL:
case Marker::TYPE_BOOL:
case Marker::TYPE_INT:
case Marker::TYPE_DOUBLE:
case Marker::TYPE_STRING:
case Marker::TYPE_LIST:
case Marker::TYPE_MAP:
case Marker::TYPE_TEMPORAL_DATA:
case Marker::TYPE_PROPERTY_VALUE:
case Marker::SECTION_VERTEX:
case Marker::SECTION_EDGE:
case Marker::SECTION_MAPPER:
case Marker::SECTION_METADATA:
case Marker::SECTION_INDICES:
case Marker::SECTION_CONSTRAINTS:
case Marker::SECTION_DELTA:
case Marker::SECTION_EPOCH_HISTORY:
case Marker::SECTION_OFFSETS:
case Marker::VALUE_FALSE:
case Marker::VALUE_TRUE:
throw RecoveryFailure("Invalid WAL data!");
}
}
// Function used to either read or skip the current WAL delta data. The WAL
// delta header must be read before calling this function. If the delta data is
// read then the data returned is valid, if the delta data is skipped then the
// returned data is not guaranteed to be set (it could be empty) and shouldn't
// be used.
// @throw RecoveryFailure
template <bool read_data>
WalDeltaData ReadSkipWalDeltaData(BaseDecoder *decoder) {
WalDeltaData delta;
auto action = decoder->ReadMarker();
if (!action) throw RecoveryFailure("Invalid WAL data!");
delta.type = MarkerToWalDeltaDataType(*action);
switch (delta.type) {
case WalDeltaData::Type::VERTEX_CREATE:
case WalDeltaData::Type::VERTEX_DELETE: {
auto gid = decoder->ReadUint();
if (!gid) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_create_delete.gid = Gid::FromUint(*gid);
break;
}
case WalDeltaData::Type::VERTEX_ADD_LABEL:
case WalDeltaData::Type::VERTEX_REMOVE_LABEL: {
auto gid = decoder->ReadUint();
if (!gid) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_add_remove_label.gid = Gid::FromUint(*gid);
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_add_remove_label.label = std::move(*label);
} else {
if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
break;
}
case WalDeltaData::Type::VERTEX_SET_PROPERTY:
case WalDeltaData::Type::EDGE_SET_PROPERTY: {
auto gid = decoder->ReadUint();
if (!gid) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_edge_set_property.gid = Gid::FromUint(*gid);
if constexpr (read_data) {
auto property = decoder->ReadString();
if (!property) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_edge_set_property.property = std::move(*property);
auto value = decoder->ReadPropertyValue();
if (!value) throw RecoveryFailure("Invalid WAL data!");
delta.vertex_edge_set_property.value = std::move(*value);
} else {
if (!decoder->SkipString() || !decoder->SkipPropertyValue()) throw RecoveryFailure("Invalid WAL data!");
}
break;
}
case WalDeltaData::Type::EDGE_CREATE:
case WalDeltaData::Type::EDGE_DELETE: {
auto gid = decoder->ReadUint();
if (!gid) throw RecoveryFailure("Invalid WAL data!");
delta.edge_create_delete.gid = Gid::FromUint(*gid);
if constexpr (read_data) {
auto edge_type = decoder->ReadString();
if (!edge_type) throw RecoveryFailure("Invalid WAL data!");
delta.edge_create_delete.edge_type = std::move(*edge_type);
} else {
if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
auto from_gid = decoder->ReadUint();
if (!from_gid) throw RecoveryFailure("Invalid WAL data!");
delta.edge_create_delete.from_vertex = Gid::FromUint(*from_gid);
auto to_gid = decoder->ReadUint();
if (!to_gid) throw RecoveryFailure("Invalid WAL data!");
delta.edge_create_delete.to_vertex = Gid::FromUint(*to_gid);
break;
}
case WalDeltaData::Type::TRANSACTION_END:
break;
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP: {
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label.label = std::move(*label);
} else {
if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_property.label = std::move(*label);
auto property = decoder->ReadString();
if (!property) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_property.property = std::move(*property);
} else {
if (!decoder->SkipString() || !decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
break;
}
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
if constexpr (read_data) {
auto label = decoder->ReadString();
if (!label) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_properties.label = std::move(*label);
auto properties_count = decoder->ReadUint();
if (!properties_count) throw RecoveryFailure("Invalid WAL data!");
for (uint64_t i = 0; i < *properties_count; ++i) {
auto property = decoder->ReadString();
if (!property) throw RecoveryFailure("Invalid WAL data!");
delta.operation_label_properties.properties.emplace(std::move(*property));
}
} else {
if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
auto properties_count = decoder->ReadUint();
if (!properties_count) throw RecoveryFailure("Invalid WAL data!");
for (uint64_t i = 0; i < *properties_count; ++i) {
if (!decoder->SkipString()) throw RecoveryFailure("Invalid WAL data!");
}
}
}
}
return delta;
}
} // namespace
// Function used to read information about the WAL file.
WalInfo ReadWalInfo(const std::filesystem::path &path) {
// Check magic and version.
Decoder wal;
auto version = wal.Initialize(path, kWalMagic);
if (!version) throw RecoveryFailure("Couldn't read WAL magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure("Invalid WAL version!");
// Prepare return value.
WalInfo info;
// Read offsets.
{
auto marker = wal.ReadMarker();
if (!marker || *marker != Marker::SECTION_OFFSETS) throw RecoveryFailure("Invalid WAL data!");
auto wal_size = wal.GetSize();
if (!wal_size) throw RecoveryFailure("Invalid WAL data!");
auto read_offset = [&wal, wal_size] {
auto maybe_offset = wal.ReadUint();
if (!maybe_offset) throw RecoveryFailure("Invalid WAL format!");
auto offset = *maybe_offset;
if (offset > *wal_size) throw RecoveryFailure("Invalid WAL format!");
return offset;
};
info.offset_metadata = read_offset();
info.offset_deltas = read_offset();
}
// Read metadata.
{
wal.SetPosition(info.offset_metadata);
auto marker = wal.ReadMarker();
if (!marker || *marker != Marker::SECTION_METADATA) throw RecoveryFailure("Invalid WAL data!");
auto maybe_uuid = wal.ReadString();
if (!maybe_uuid) throw RecoveryFailure("Invalid WAL data!");
info.uuid = std::move(*maybe_uuid);
auto maybe_epoch_id = wal.ReadString();
if (!maybe_epoch_id) throw RecoveryFailure("Invalid WAL data!");
info.epoch_id = std::move(*maybe_epoch_id);
auto maybe_seq_num = wal.ReadUint();
if (!maybe_seq_num) throw RecoveryFailure("Invalid WAL data!");
info.seq_num = *maybe_seq_num;
}
// Read deltas.
info.num_deltas = 0;
auto validate_delta = [&wal]() -> std::optional<std::pair<uint64_t, bool>> {
try {
auto timestamp = ReadWalDeltaHeader(&wal);
auto type = SkipWalDeltaData(&wal);
return {{timestamp, IsWalDeltaDataTypeTransactionEnd(type)}};
} catch (const RecoveryFailure &) {
return std::nullopt;
}
};
auto size = wal.GetSize();
// Here we read the whole file and determine the number of valid deltas. A
// delta is valid only if all of its data can be successfully read. This
// allows us to recover data from WAL files that are corrupt at the end (eg.
// because of power loss) but are still valid at the beginning. While reading
// the deltas we only count deltas which are a part of a fully valid
// transaction (indicated by a TRANSACTION_END delta or any other
// non-transactional operation).
std::optional<uint64_t> current_timestamp;
uint64_t num_deltas = 0;
while (wal.GetPosition() != size) {
auto ret = validate_delta();
if (!ret) break;
auto [timestamp, is_end_of_transaction] = *ret;
if (!current_timestamp) current_timestamp = timestamp;
if (*current_timestamp != timestamp) break;
++num_deltas;
if (is_end_of_transaction) {
if (info.num_deltas == 0) {
info.from_timestamp = timestamp;
info.to_timestamp = timestamp;
}
if (timestamp < info.from_timestamp || timestamp < info.to_timestamp) break;
info.to_timestamp = timestamp;
info.num_deltas += num_deltas;
current_timestamp = std::nullopt;
num_deltas = 0;
}
}
if (info.num_deltas == 0) throw RecoveryFailure("Invalid WAL data!");
return info;
}
bool operator==(const WalDeltaData &a, const WalDeltaData &b) {
if (a.type != b.type) return false;
switch (a.type) {
case WalDeltaData::Type::VERTEX_CREATE:
case WalDeltaData::Type::VERTEX_DELETE:
return a.vertex_create_delete.gid == b.vertex_create_delete.gid;
case WalDeltaData::Type::VERTEX_ADD_LABEL:
case WalDeltaData::Type::VERTEX_REMOVE_LABEL:
return a.vertex_add_remove_label.gid == b.vertex_add_remove_label.gid &&
a.vertex_add_remove_label.label == b.vertex_add_remove_label.label;
case WalDeltaData::Type::VERTEX_SET_PROPERTY:
case WalDeltaData::Type::EDGE_SET_PROPERTY:
return a.vertex_edge_set_property.gid == b.vertex_edge_set_property.gid &&
a.vertex_edge_set_property.property == b.vertex_edge_set_property.property &&
a.vertex_edge_set_property.value == b.vertex_edge_set_property.value;
case WalDeltaData::Type::EDGE_CREATE:
case WalDeltaData::Type::EDGE_DELETE:
return a.edge_create_delete.gid == b.edge_create_delete.gid &&
a.edge_create_delete.edge_type == b.edge_create_delete.edge_type &&
a.edge_create_delete.from_vertex == b.edge_create_delete.from_vertex &&
a.edge_create_delete.to_vertex == b.edge_create_delete.to_vertex;
case WalDeltaData::Type::TRANSACTION_END:
return true;
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP:
return a.operation_label.label == b.operation_label.label;
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP:
return a.operation_label_property.label == b.operation_label_property.label &&
a.operation_label_property.property == b.operation_label_property.property;
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP:
return a.operation_label_properties.label == b.operation_label_properties.label &&
a.operation_label_properties.properties == b.operation_label_properties.properties;
}
}
bool operator!=(const WalDeltaData &a, const WalDeltaData &b) { return !(a == b); }
// Function used to read the WAL delta header. The function returns the delta
// timestamp.
uint64_t ReadWalDeltaHeader(BaseDecoder *decoder) {
auto marker = decoder->ReadMarker();
if (!marker || *marker != Marker::SECTION_DELTA) throw RecoveryFailure("Invalid WAL data!");
auto timestamp = decoder->ReadUint();
if (!timestamp) throw RecoveryFailure("Invalid WAL data!");
return *timestamp;
}
// Function used to read the current WAL delta data. The WAL delta header must
// be read before calling this function.
WalDeltaData ReadWalDeltaData(BaseDecoder *decoder) { return ReadSkipWalDeltaData<true>(decoder); }
// Function used to skip the current WAL delta data. The WAL delta header must
// be read before calling this function.
WalDeltaData::Type SkipWalDeltaData(BaseDecoder *decoder) {
auto delta = ReadSkipWalDeltaData<false>(decoder);
return delta.type;
}
void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, Config::Items items, const Delta &delta,
const Vertex &vertex, uint64_t timestamp) {
// When converting a Delta to a WAL delta the logic is inverted. That is
// because the Delta's represent undo actions and we want to store redo
// actions.
encoder->WriteMarker(Marker::SECTION_DELTA);
encoder->WriteUint(timestamp);
std::lock_guard<utils::SpinLock> guard(vertex.lock);
switch (delta.action) {
case Delta::Action::DELETE_OBJECT:
case Delta::Action::RECREATE_OBJECT: {
encoder->WriteMarker(VertexActionToMarker(delta.action));
encoder->WriteUint(vertex.gid.AsUint());
break;
}
case Delta::Action::SET_PROPERTY: {
encoder->WriteMarker(Marker::DELTA_VERTEX_SET_PROPERTY);
encoder->WriteUint(vertex.gid.AsUint());
encoder->WriteString(name_id_mapper->IdToName(delta.property.key.AsUint()));
// The property value is the value that is currently stored in the
// vertex.
// TODO (mferencevic): Mitigate the memory allocation introduced here
// (with the `GetProperty` call). It is the only memory allocation in the
// entire WAL file writing logic.
encoder->WritePropertyValue(vertex.properties.GetProperty(delta.property.key));
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL: {
encoder->WriteMarker(VertexActionToMarker(delta.action));
encoder->WriteUint(vertex.gid.AsUint());
encoder->WriteString(name_id_mapper->IdToName(delta.label.AsUint()));
break;
}
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE: {
encoder->WriteMarker(VertexActionToMarker(delta.action));
if (items.properties_on_edges) {
encoder->WriteUint(delta.vertex_edge.edge.ptr->gid.AsUint());
} else {
encoder->WriteUint(delta.vertex_edge.edge.gid.AsUint());
}
encoder->WriteString(name_id_mapper->IdToName(delta.vertex_edge.edge_type.AsUint()));
encoder->WriteUint(vertex.gid.AsUint());
encoder->WriteUint(delta.vertex_edge.vertex->gid.AsUint());
break;
}
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
// These actions are already encoded in the *_OUT_EDGE actions. This
// function should never be called for this type of deltas.
LOG_FATAL("Invalid delta action!");
}
}
void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, const Delta &delta, const Edge &edge,
uint64_t timestamp) {
// When converting a Delta to a WAL delta the logic is inverted. That is
// because the Delta's represent undo actions and we want to store redo
// actions.
encoder->WriteMarker(Marker::SECTION_DELTA);
encoder->WriteUint(timestamp);
std::lock_guard<utils::SpinLock> guard(edge.lock);
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
encoder->WriteMarker(Marker::DELTA_EDGE_SET_PROPERTY);
encoder->WriteUint(edge.gid.AsUint());
encoder->WriteString(name_id_mapper->IdToName(delta.property.key.AsUint()));
// The property value is the value that is currently stored in the
// edge.
// TODO (mferencevic): Mitigate the memory allocation introduced here
// (with the `GetProperty` call). It is the only memory allocation in the
// entire WAL file writing logic.
encoder->WritePropertyValue(edge.properties.GetProperty(delta.property.key));
break;
}
case Delta::Action::DELETE_OBJECT:
case Delta::Action::RECREATE_OBJECT:
// These actions are already encoded in vertex *_OUT_EDGE actions. Also,
// these deltas don't contain any information about the from vertex, to
// vertex or edge type so they are useless. This function should never
// be called for this type of deltas.
LOG_FATAL("Invalid delta action!");
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
// These deltas shouldn't appear for edges.
LOG_FATAL("Invalid database state!");
}
}
void EncodeTransactionEnd(BaseEncoder *encoder, uint64_t timestamp) {
encoder->WriteMarker(Marker::SECTION_DELTA);
encoder->WriteUint(timestamp);
encoder->WriteMarker(Marker::DELTA_TRANSACTION_END);
}
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties, uint64_t timestamp) {
encoder->WriteMarker(Marker::SECTION_DELTA);
encoder->WriteUint(timestamp);
switch (operation) {
case StorageGlobalOperation::LABEL_INDEX_CREATE:
case StorageGlobalOperation::LABEL_INDEX_DROP: {
MG_ASSERT(properties.empty(), "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
break;
}
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
case StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
case StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP: {
MG_ASSERT(properties.size() == 1, "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
encoder->WriteString(name_id_mapper->IdToName((*properties.begin()).AsUint()));
break;
}
case StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
case StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP: {
MG_ASSERT(!properties.empty(), "Invalid function call!");
encoder->WriteMarker(OperationToMarker(operation));
encoder->WriteString(name_id_mapper->IdToName(label.AsUint()));
encoder->WriteUint(properties.size());
for (const auto &property : properties) {
encoder->WriteString(name_id_mapper->IdToName(property.AsUint()));
}
break;
}
}
}
RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConstraints *indices_constraints,
const std::optional<uint64_t> last_loaded_timestamp, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges, NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count,
Config::Items items) {
spdlog::info("Trying to load WAL file {}.", path);
RecoveryInfo ret;
Decoder wal;
auto version = wal.Initialize(path, kWalMagic);
if (!version) throw RecoveryFailure("Couldn't read WAL magic and/or version!");
if (!IsVersionSupported(*version)) throw RecoveryFailure("Invalid WAL version!");
// Read wal info.
auto info = ReadWalInfo(path);
ret.last_commit_timestamp = info.to_timestamp;
// Check timestamp.
if (last_loaded_timestamp && info.to_timestamp <= *last_loaded_timestamp) {
spdlog::info("Skip loading WAL file because it is too old.");
return ret;
}
// Recover deltas.
wal.SetPosition(info.offset_deltas);
uint64_t deltas_applied = 0;
auto edge_acc = edges->access();
auto vertex_acc = vertices->access();
spdlog::info("WAL file contains {} deltas.", info.num_deltas);
for (uint64_t i = 0; i < info.num_deltas; ++i) {
// Read WAL delta header to find out the delta timestamp.
auto timestamp = ReadWalDeltaHeader(&wal);
if (!last_loaded_timestamp || timestamp > *last_loaded_timestamp) {
// This delta should be loaded.
auto delta = ReadWalDeltaData(&wal);
switch (delta.type) {
case WalDeltaData::Type::VERTEX_CREATE: {
auto [vertex, inserted] = vertex_acc.insert(Vertex{delta.vertex_create_delete.gid, nullptr});
if (!inserted) throw RecoveryFailure("The vertex must be inserted here!");
ret.next_vertex_id = std::max(ret.next_vertex_id, delta.vertex_create_delete.gid.AsUint() + 1);
break;
}
case WalDeltaData::Type::VERTEX_DELETE: {
auto vertex = vertex_acc.find(delta.vertex_create_delete.gid);
if (vertex == vertex_acc.end()) throw RecoveryFailure("The vertex doesn't exist!");
if (!vertex->in_edges.empty() || !vertex->out_edges.empty())
throw RecoveryFailure("The vertex can't be deleted because it still has edges!");
if (!vertex_acc.remove(delta.vertex_create_delete.gid))
throw RecoveryFailure("The vertex must be removed here!");
break;
}
case WalDeltaData::Type::VERTEX_ADD_LABEL:
case WalDeltaData::Type::VERTEX_REMOVE_LABEL: {
auto vertex = vertex_acc.find(delta.vertex_add_remove_label.gid);
if (vertex == vertex_acc.end()) throw RecoveryFailure("The vertex doesn't exist!");
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.vertex_add_remove_label.label));
auto it = std::find(vertex->labels.begin(), vertex->labels.end(), label_id);
if (delta.type == WalDeltaData::Type::VERTEX_ADD_LABEL) {
if (it != vertex->labels.end()) throw RecoveryFailure("The vertex already has the label!");
vertex->labels.push_back(label_id);
} else {
if (it == vertex->labels.end()) throw RecoveryFailure("The vertex doesn't have the label!");
std::swap(*it, vertex->labels.back());
vertex->labels.pop_back();
}
break;
}
case WalDeltaData::Type::VERTEX_SET_PROPERTY: {
auto vertex = vertex_acc.find(delta.vertex_edge_set_property.gid);
if (vertex == vertex_acc.end()) throw RecoveryFailure("The vertex doesn't exist!");
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.vertex_edge_set_property.property));
auto &property_value = delta.vertex_edge_set_property.value;
vertex->properties.SetProperty(property_id, property_value);
break;
}
case WalDeltaData::Type::EDGE_CREATE: {
auto from_vertex = vertex_acc.find(delta.edge_create_delete.from_vertex);
if (from_vertex == vertex_acc.end()) throw RecoveryFailure("The from vertex doesn't exist!");
auto to_vertex = vertex_acc.find(delta.edge_create_delete.to_vertex);
if (to_vertex == vertex_acc.end()) throw RecoveryFailure("The to vertex doesn't exist!");
auto edge_gid = delta.edge_create_delete.gid;
auto edge_type_id = EdgeTypeId::FromUint(name_id_mapper->NameToId(delta.edge_create_delete.edge_type));
EdgeRef edge_ref(edge_gid);
if (items.properties_on_edges) {
auto [edge, inserted] = edge_acc.insert(Edge{edge_gid, nullptr});
if (!inserted) throw RecoveryFailure("The edge must be inserted here!");
edge_ref = EdgeRef(&*edge);
}
{
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{edge_type_id, &*to_vertex, edge_ref};
auto it = std::find(from_vertex->out_edges.begin(), from_vertex->out_edges.end(), link);
if (it != from_vertex->out_edges.end()) throw RecoveryFailure("The from vertex already has this edge!");
from_vertex->out_edges.push_back(link);
}
{
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{edge_type_id, &*from_vertex, edge_ref};
auto it = std::find(to_vertex->in_edges.begin(), to_vertex->in_edges.end(), link);
if (it != to_vertex->in_edges.end()) throw RecoveryFailure("The to vertex already has this edge!");
to_vertex->in_edges.push_back(link);
}
ret.next_edge_id = std::max(ret.next_edge_id, edge_gid.AsUint() + 1);
// Increment edge count.
edge_count->fetch_add(1, std::memory_order_acq_rel);
break;
}
case WalDeltaData::Type::EDGE_DELETE: {
auto from_vertex = vertex_acc.find(delta.edge_create_delete.from_vertex);
if (from_vertex == vertex_acc.end()) throw RecoveryFailure("The from vertex doesn't exist!");
auto to_vertex = vertex_acc.find(delta.edge_create_delete.to_vertex);
if (to_vertex == vertex_acc.end()) throw RecoveryFailure("The to vertex doesn't exist!");
auto edge_gid = delta.edge_create_delete.gid;
auto edge_type_id = EdgeTypeId::FromUint(name_id_mapper->NameToId(delta.edge_create_delete.edge_type));
EdgeRef edge_ref(edge_gid);
if (items.properties_on_edges) {
auto edge = edge_acc.find(edge_gid);
if (edge == edge_acc.end()) throw RecoveryFailure("The edge doesn't exist!");
edge_ref = EdgeRef(&*edge);
}
{
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{edge_type_id, &*to_vertex, edge_ref};
auto it = std::find(from_vertex->out_edges.begin(), from_vertex->out_edges.end(), link);
if (it == from_vertex->out_edges.end()) throw RecoveryFailure("The from vertex doesn't have this edge!");
std::swap(*it, from_vertex->out_edges.back());
from_vertex->out_edges.pop_back();
}
{
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{edge_type_id, &*from_vertex, edge_ref};
auto it = std::find(to_vertex->in_edges.begin(), to_vertex->in_edges.end(), link);
if (it == to_vertex->in_edges.end()) throw RecoveryFailure("The to vertex doesn't have this edge!");
std::swap(*it, to_vertex->in_edges.back());
to_vertex->in_edges.pop_back();
}
if (items.properties_on_edges) {
if (!edge_acc.remove(edge_gid)) throw RecoveryFailure("The edge must be removed here!");
}
// Decrement edge count.
edge_count->fetch_add(-1, std::memory_order_acq_rel);
break;
}
case WalDeltaData::Type::EDGE_SET_PROPERTY: {
if (!items.properties_on_edges)
throw RecoveryFailure(
"The WAL has properties on edges, but the storage is "
"configured without properties on edges!");
auto edge = edge_acc.find(delta.vertex_edge_set_property.gid);
if (edge == edge_acc.end()) throw RecoveryFailure("The edge doesn't exist!");
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.vertex_edge_set_property.property));
auto &property_value = delta.vertex_edge_set_property.value;
edge->properties.SetProperty(property_id, property_value);
break;
}
case WalDeltaData::Type::TRANSACTION_END:
break;
case WalDeltaData::Type::LABEL_INDEX_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label.label));
AddRecoveredIndexConstraint(&indices_constraints->indices.label, label_id, "The label index already exists!");
break;
}
case WalDeltaData::Type::LABEL_INDEX_DROP: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label.label));
RemoveRecoveredIndexConstraint(&indices_constraints->indices.label, label_id,
"The label index doesn't exist!");
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
AddRecoveredIndexConstraint(&indices_constraints->indices.label_property, {label_id, property_id},
"The label property index already exists!");
break;
}
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
RemoveRecoveredIndexConstraint(&indices_constraints->indices.label_property, {label_id, property_id},
"The label property index doesn't exist!");
break;
}
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
AddRecoveredIndexConstraint(&indices_constraints->constraints.existence, {label_id, property_id},
"The existence constraint already exists!");
break;
}
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.label));
auto property_id = PropertyId::FromUint(name_id_mapper->NameToId(delta.operation_label_property.property));
RemoveRecoveredIndexConstraint(&indices_constraints->constraints.existence, {label_id, property_id},
"The existence constraint doesn't exist!");
break;
}
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_properties.label));
std::set<PropertyId> property_ids;
for (const auto &prop : delta.operation_label_properties.properties) {
property_ids.insert(PropertyId::FromUint(name_id_mapper->NameToId(prop)));
}
AddRecoveredIndexConstraint(&indices_constraints->constraints.unique, {label_id, property_ids},
"The unique constraint already exists!");
break;
}
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
auto label_id = LabelId::FromUint(name_id_mapper->NameToId(delta.operation_label_properties.label));
std::set<PropertyId> property_ids;
for (const auto &prop : delta.operation_label_properties.properties) {
property_ids.insert(PropertyId::FromUint(name_id_mapper->NameToId(prop)));
}
RemoveRecoveredIndexConstraint(&indices_constraints->constraints.unique, {label_id, property_ids},
"The unique constraint doesn't exist!");
break;
}
}
ret.next_timestamp = std::max(ret.next_timestamp, timestamp + 1);
++deltas_applied;
} else {
// This delta should be skipped.
SkipWalDeltaData(&wal);
}
}
spdlog::info("Applied {} deltas from WAL. Skipped {} deltas, because they were too old.", deltas_applied,
info.num_deltas - deltas_applied);
return ret;
}
WalFile::WalFile(const std::filesystem::path &wal_directory, const std::string_view uuid,
const std::string_view epoch_id, Config::Items items, NameIdMapper *name_id_mapper, uint64_t seq_num,
utils::FileRetainer *file_retainer)
: items_(items),
name_id_mapper_(name_id_mapper),
path_(wal_directory / MakeWalName()),
from_timestamp_(0),
to_timestamp_(0),
count_(0),
seq_num_(seq_num),
file_retainer_(file_retainer) {
// Ensure that the storage directory exists.
utils::EnsureDirOrDie(wal_directory);
// Initialize the WAL file.
wal_.Initialize(path_, kWalMagic, kVersion);
// Write placeholder offsets.
uint64_t offset_offsets = 0;
uint64_t offset_metadata = 0;
uint64_t offset_deltas = 0;
wal_.WriteMarker(Marker::SECTION_OFFSETS);
offset_offsets = wal_.GetPosition();
wal_.WriteUint(offset_metadata);
wal_.WriteUint(offset_deltas);
// Write metadata.
offset_metadata = wal_.GetPosition();
wal_.WriteMarker(Marker::SECTION_METADATA);
wal_.WriteString(uuid);
wal_.WriteString(epoch_id);
wal_.WriteUint(seq_num);
// Write final offsets.
offset_deltas = wal_.GetPosition();
wal_.SetPosition(offset_offsets);
wal_.WriteUint(offset_metadata);
wal_.WriteUint(offset_deltas);
wal_.SetPosition(offset_deltas);
// Sync the initial data.
wal_.Sync();
}
WalFile::WalFile(std::filesystem::path current_wal_path, Config::Items items, NameIdMapper *name_id_mapper,
uint64_t seq_num, uint64_t from_timestamp, uint64_t to_timestamp, uint64_t count,
utils::FileRetainer *file_retainer)
: items_(items),
name_id_mapper_(name_id_mapper),
path_(std::move(current_wal_path)),
from_timestamp_(from_timestamp),
to_timestamp_(to_timestamp),
count_(count),
seq_num_(seq_num),
file_retainer_(file_retainer) {
wal_.OpenExisting(path_);
}
void WalFile::FinalizeWal() {
if (count_ != 0) {
wal_.Finalize();
// Rename file.
std::filesystem::path new_path(path_);
new_path.replace_filename(RemakeWalName(path_.filename(), from_timestamp_, to_timestamp_));
utils::CopyFile(path_, new_path);
wal_.Close();
file_retainer_->DeleteFile(path_);
path_ = std::move(new_path);
}
}
void WalFile::DeleteWal() {
wal_.Close();
file_retainer_->DeleteFile(path_);
}
WalFile::~WalFile() {
if (count_ == 0) {
// Remove empty WAL file.
utils::DeleteFile(path_);
}
}
void WalFile::AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t timestamp) {
EncodeDelta(&wal_, name_id_mapper_, items_, delta, vertex, timestamp);
UpdateStats(timestamp);
}
void WalFile::AppendDelta(const Delta &delta, const Edge &edge, uint64_t timestamp) {
EncodeDelta(&wal_, name_id_mapper_, delta, edge, timestamp);
UpdateStats(timestamp);
}
void WalFile::AppendTransactionEnd(uint64_t timestamp) {
EncodeTransactionEnd(&wal_, timestamp);
UpdateStats(timestamp);
}
void WalFile::AppendOperation(StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
uint64_t timestamp) {
EncodeOperation(&wal_, name_id_mapper_, operation, label, properties, timestamp);
UpdateStats(timestamp);
}
void WalFile::Sync() { wal_.Sync(); }
uint64_t WalFile::GetSize() { return wal_.GetSize(); }
uint64_t WalFile::SequenceNumber() const { return seq_num_; }
void WalFile::UpdateStats(uint64_t timestamp) {
if (count_ == 0) from_timestamp_ = timestamp;
to_timestamp_ = timestamp;
count_ += 1;
}
void WalFile::DisableFlushing() { wal_.DisableFlushing(); }
void WalFile::EnableFlushing() { wal_.EnableFlushing(); }
void WalFile::TryFlushing() { wal_.TryFlushing(); }
std::pair<const uint8_t *, size_t> WalFile::CurrentFileBuffer() const { return wal_.CurrentFileBuffer(); }
} // namespace memgraph::storage::v3::durability

View File

@ -0,0 +1,261 @@
// 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 <cstdint>
#include <filesystem>
#include <set>
#include <string>
#include "storage/v3/config.hpp"
#include "storage/v3/delta.hpp"
#include "storage/v3/durability/metadata.hpp"
#include "storage/v3/durability/serialization.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/vertex.hpp"
#include "utils/file_locker.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3::durability {
/// Structure used to hold information about a WAL.
struct WalInfo {
uint64_t offset_metadata;
uint64_t offset_deltas;
std::string uuid;
std::string epoch_id;
uint64_t seq_num;
uint64_t from_timestamp;
uint64_t to_timestamp;
uint64_t num_deltas;
};
/// Structure used to return loaded WAL delta data.
struct WalDeltaData {
enum class Type {
VERTEX_CREATE,
VERTEX_DELETE,
VERTEX_ADD_LABEL,
VERTEX_REMOVE_LABEL,
VERTEX_SET_PROPERTY,
EDGE_CREATE,
EDGE_DELETE,
EDGE_SET_PROPERTY,
TRANSACTION_END,
LABEL_INDEX_CREATE,
LABEL_INDEX_DROP,
LABEL_PROPERTY_INDEX_CREATE,
LABEL_PROPERTY_INDEX_DROP,
EXISTENCE_CONSTRAINT_CREATE,
EXISTENCE_CONSTRAINT_DROP,
UNIQUE_CONSTRAINT_CREATE,
UNIQUE_CONSTRAINT_DROP,
};
Type type{Type::TRANSACTION_END};
struct {
Gid gid;
} vertex_create_delete;
struct {
Gid gid;
std::string label;
} vertex_add_remove_label;
struct {
Gid gid;
std::string property;
PropertyValue value;
} vertex_edge_set_property;
struct {
Gid gid;
std::string edge_type;
Gid from_vertex;
Gid to_vertex;
} edge_create_delete;
struct {
std::string label;
} operation_label;
struct {
std::string label;
std::string property;
} operation_label_property;
struct {
std::string label;
std::set<std::string> properties;
} operation_label_properties;
};
bool operator==(const WalDeltaData &a, const WalDeltaData &b);
bool operator!=(const WalDeltaData &a, const WalDeltaData &b);
/// Enum used to indicate a global database operation that isn't transactional.
enum class StorageGlobalOperation {
LABEL_INDEX_CREATE,
LABEL_INDEX_DROP,
LABEL_PROPERTY_INDEX_CREATE,
LABEL_PROPERTY_INDEX_DROP,
EXISTENCE_CONSTRAINT_CREATE,
EXISTENCE_CONSTRAINT_DROP,
UNIQUE_CONSTRAINT_CREATE,
UNIQUE_CONSTRAINT_DROP,
};
constexpr bool IsWalDeltaDataTypeTransactionEnd(const WalDeltaData::Type type) {
switch (type) {
// These delta actions are all found inside transactions so they don't
// indicate a transaction end.
case WalDeltaData::Type::VERTEX_CREATE:
case WalDeltaData::Type::VERTEX_DELETE:
case WalDeltaData::Type::VERTEX_ADD_LABEL:
case WalDeltaData::Type::VERTEX_REMOVE_LABEL:
case WalDeltaData::Type::EDGE_CREATE:
case WalDeltaData::Type::EDGE_DELETE:
case WalDeltaData::Type::VERTEX_SET_PROPERTY:
case WalDeltaData::Type::EDGE_SET_PROPERTY:
return false;
// This delta explicitly indicates that a transaction is done.
case WalDeltaData::Type::TRANSACTION_END:
// These operations aren't transactional and they are encoded only using
// a single delta, so they each individually mark the end of their
// 'transaction'.
case WalDeltaData::Type::LABEL_INDEX_CREATE:
case WalDeltaData::Type::LABEL_INDEX_DROP:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE:
case WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE:
case WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE:
case WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP:
return true;
}
}
/// Function used to read information about the WAL file.
/// @throw RecoveryFailure
WalInfo ReadWalInfo(const std::filesystem::path &path);
/// Function used to read the WAL delta header. The function returns the delta
/// timestamp.
/// @throw RecoveryFailure
uint64_t ReadWalDeltaHeader(BaseDecoder *decoder);
/// Function used to read the current WAL delta data. The function returns the
/// read delta data. The WAL delta header must be read before calling this
/// function.
/// @throw RecoveryFailure
WalDeltaData ReadWalDeltaData(BaseDecoder *decoder);
/// Function used to skip the current WAL delta data. The function returns the
/// skipped delta type. The WAL delta header must be read before calling this
/// function.
/// @throw RecoveryFailure
WalDeltaData::Type SkipWalDeltaData(BaseDecoder *decoder);
/// Function used to encode a `Delta` that originated from a `Vertex`.
void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, Config::Items items, const Delta &delta,
const Vertex &vertex, uint64_t timestamp);
/// Function used to encode a `Delta` that originated from an `Edge`.
void EncodeDelta(BaseEncoder *encoder, NameIdMapper *name_id_mapper, const Delta &delta, const Edge &edge,
uint64_t timestamp);
/// Function used to encode the transaction end.
void EncodeTransactionEnd(BaseEncoder *encoder, uint64_t timestamp);
/// Function used to encode non-transactional operation.
void EncodeOperation(BaseEncoder *encoder, NameIdMapper *name_id_mapper, StorageGlobalOperation operation,
LabelId label, const std::set<PropertyId> &properties, uint64_t timestamp);
/// Function used to load the WAL data into the storage.
/// @throw RecoveryFailure
RecoveryInfo LoadWal(const std::filesystem::path &path, RecoveredIndicesAndConstraints *indices_constraints,
std::optional<uint64_t> last_loaded_timestamp, utils::SkipList<Vertex> *vertices,
utils::SkipList<Edge> *edges, NameIdMapper *name_id_mapper, std::atomic<uint64_t> *edge_count,
Config::Items items);
/// WalFile class used to append deltas and operations to the WAL file.
class WalFile {
public:
WalFile(const std::filesystem::path &wal_directory, std::string_view uuid, std::string_view epoch_id,
Config::Items items, NameIdMapper *name_id_mapper, uint64_t seq_num, utils::FileRetainer *file_retainer);
WalFile(std::filesystem::path current_wal_path, Config::Items items, NameIdMapper *name_id_mapper, uint64_t seq_num,
uint64_t from_timestamp, uint64_t to_timestamp, uint64_t count, utils::FileRetainer *file_retainer);
WalFile(const WalFile &) = delete;
WalFile(WalFile &&) = delete;
WalFile &operator=(const WalFile &) = delete;
WalFile &operator=(WalFile &&) = delete;
~WalFile();
void AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t timestamp);
void AppendDelta(const Delta &delta, const Edge &edge, uint64_t timestamp);
void AppendTransactionEnd(uint64_t timestamp);
void AppendOperation(StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
uint64_t timestamp);
void Sync();
uint64_t GetSize();
uint64_t SequenceNumber() const;
auto FromTimestamp() const { return from_timestamp_; }
auto ToTimestamp() const { return to_timestamp_; }
auto Count() const { return count_; }
// Disable flushing of the internal buffer.
void DisableFlushing();
// Enable flushing of the internal buffer.
void EnableFlushing();
// Try flushing the internal buffer.
void TryFlushing();
// Get the internal buffer with its size.
std::pair<const uint8_t *, size_t> CurrentFileBuffer() const;
// Get the path of the current WAL file.
const auto &Path() const { return path_; }
void FinalizeWal();
void DeleteWal();
private:
void UpdateStats(uint64_t timestamp);
Config::Items items_;
NameIdMapper *name_id_mapper_;
Encoder wal_;
std::filesystem::path path_;
uint64_t from_timestamp_;
uint64_t to_timestamp_;
uint64_t count_;
uint64_t seq_num_;
utils::FileRetainer *file_retainer_;
};
} // namespace memgraph::storage::v3::durability

51
src/storage/v3/edge.hpp Normal file
View File

@ -0,0 +1,51 @@
// 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 <limits>
#include "storage/v3/delta.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/property_store.hpp"
#include "utils/logging.hpp"
#include "utils/spin_lock.hpp"
namespace memgraph::storage::v3 {
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,
"Edge must be created with an initial DELETE_OBJECT delta!");
}
Gid gid;
PropertyStore properties;
mutable utils::SpinLock lock;
bool deleted;
// uint8_t PAD;
// uint16_t PAD;
Delta *delta;
};
static_assert(alignof(Edge) >= 8, "The Edge should be aligned to at least 8!");
inline bool operator==(const Edge &first, const Edge &second) { return first.gid == second.gid; }
inline bool operator<(const Edge &first, const Edge &second) { return first.gid < second.gid; }
inline bool operator==(const Edge &first, const Gid &second) { return first.gid == second; }
inline bool operator<(const Edge &first, const Gid &second) { return first.gid < second; }
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,199 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/edge_accessor.hpp"
#include <memory>
#include "storage/v3/mvcc.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/vertex_accessor.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage::v3 {
bool EdgeAccessor::IsVisible(const View view) const {
bool deleted = true;
bool exists = true;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return exists && (for_deleted_ || !deleted);
}
VertexAccessor EdgeAccessor::FromVertex() const {
return VertexAccessor{from_vertex_, transaction_, indices_, constraints_, config_};
}
VertexAccessor EdgeAccessor::ToVertex() const {
return VertexAccessor{to_vertex_, transaction_, indices_, constraints_, config_};
}
Result<PropertyValue> EdgeAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto current_value = edge_.ptr->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property, current_value);
edge_.ptr->properties.SetProperty(property, value);
return std::move(current_value);
}
Result<std::map<PropertyId, PropertyValue>> EdgeAccessor::ClearProperties() {
if (!config_.properties_on_edges) return Error::PROPERTIES_DISABLED;
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
if (!PrepareForWrite(transaction_, edge_.ptr)) return Error::SERIALIZATION_ERROR;
if (edge_.ptr->deleted) return Error::DELETED_OBJECT;
auto properties = edge_.ptr->properties.Properties();
for (const auto &property : properties) {
CreateAndLinkDelta(transaction_, edge_.ptr, Delta::SetPropertyTag(), property.first, property.second);
}
edge_.ptr->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> EdgeAccessor::GetProperty(PropertyId property, View view) const {
if (!config_.properties_on_edges) return PropertyValue();
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
value = edge_.ptr->properties.GetProperty(property);
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> EdgeAccessor::Properties(View view) const {
if (!config_.properties_on_edges) return std::map<PropertyId, PropertyValue>{};
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge_.ptr->lock);
deleted = edge_.ptr->deleted;
properties = edge_.ptr->properties.Properties();
delta = edge_.ptr->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(properties);
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,111 @@
// 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 "storage/v3/edge.hpp"
#include "storage/v3/edge_ref.hpp"
#include "storage/v3/config.hpp"
#include "storage/v3/result.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/view.hpp"
namespace memgraph::storage::v3 {
struct Vertex;
class VertexAccessor;
struct Indices;
struct Constraints;
class EdgeAccessor final {
private:
friend class Storage;
public:
EdgeAccessor(EdgeRef edge, EdgeTypeId edge_type, Vertex *from_vertex, Vertex *to_vertex, Transaction *transaction,
Indices *indices, Constraints *constraints, Config::Items config, bool for_deleted = false)
: edge_(edge),
edge_type_(edge_type),
from_vertex_(from_vertex),
to_vertex_(to_vertex),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config),
for_deleted_(for_deleted) {}
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const;
VertexAccessor FromVertex() const;
VertexAccessor ToVertex() const;
EdgeTypeId EdgeType() const { return edge_type_; }
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
/// Remove all properties and return old values for each removed property.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const;
Gid Gid() const noexcept {
if (config_.properties_on_edges) {
return edge_.ptr->gid;
}
return edge_.gid;
}
bool IsCycle() const { return from_vertex_ == to_vertex_; }
bool operator==(const EdgeAccessor &other) const noexcept {
return edge_ == other.edge_ && transaction_ == other.transaction_;
}
bool operator!=(const EdgeAccessor &other) const noexcept { return !(*this == other); }
private:
EdgeRef edge_;
EdgeTypeId edge_type_;
Vertex *from_vertex_;
Vertex *to_vertex_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
// if the accessor was created for a deleted edge.
// Accessor behaves differently for some methods based on this
// flag.
// E.g. If this field is set to true, GetProperty will return the property of the edge
// even though the edge is deleted.
// All the write operations will still return an error if it's called for a deleted edge.
bool for_deleted_{false};
};
} // namespace memgraph::storage::v3
namespace std {
template <>
struct hash<memgraph::storage::v3::EdgeAccessor> {
size_t operator()(const memgraph::storage::v3::EdgeAccessor &e) const { return e.Gid().AsUint(); }
};
} // namespace std

View File

@ -0,0 +1,43 @@
// 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 <type_traits>
#include "storage/v3/id_types.hpp"
namespace memgraph::storage::v3 {
// Forward declaration because we only store a pointer here.
struct Edge;
struct EdgeRef {
explicit EdgeRef(Gid gid) : gid(gid) {}
explicit EdgeRef(Edge *ptr) : ptr(ptr) {}
union {
Gid gid;
Edge *ptr;
};
};
// NOLINTNEXTLINE(bugprone-sizeof-expression)
static_assert(sizeof(Gid) == sizeof(Edge *), "The Gid should be the same size as an Edge *!");
static_assert(std::is_standard_layout_v<Gid>, "The Gid must have a standard layout!");
static_assert(std::is_standard_layout_v<Edge *>, "The Edge * must have a standard layout!");
static_assert(std::is_standard_layout_v<EdgeRef>, "The EdgeRef must have a standard layout!");
inline bool operator==(const EdgeRef &a, const EdgeRef &b) noexcept { return a.gid == b.gid; }
inline bool operator!=(const EdgeRef &a, const EdgeRef &b) noexcept { return a.gid != b.gid; }
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,78 @@
// 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 <functional>
#include <type_traits>
#include "utils/cast.hpp"
namespace memgraph::storage::v3 {
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
#define STORAGE_DEFINE_ID_TYPE(name) \
class name final { \
private: \
explicit name(uint64_t id) : id_(id) {} \
\
public: \
/* Default constructor to allow serialization or preallocation. */ \
name() = default; \
\
static name FromUint(uint64_t id) { return (name){id}; } \
static name FromInt(int64_t id) { return (name){utils::MemcpyCast<uint64_t>(id)}; } \
uint64_t AsUint() const { return id_; } \
int64_t AsInt() const { return utils::MemcpyCast<int64_t>(id_); } \
\
private: \
uint64_t id_; \
}; \
static_assert(std::is_trivially_copyable<name>::value, "storage::" #name " must be trivially copyable!"); \
inline bool operator==(const name &first, const name &second) { return first.AsUint() == second.AsUint(); } \
inline bool operator!=(const name &first, const name &second) { return first.AsUint() != second.AsUint(); } \
inline bool operator<(const name &first, const name &second) { return first.AsUint() < second.AsUint(); } \
inline bool operator>(const name &first, const name &second) { return first.AsUint() > second.AsUint(); } \
inline bool operator<=(const name &first, const name &second) { return first.AsUint() <= second.AsUint(); } \
inline bool operator>=(const name &first, const name &second) { return first.AsUint() >= second.AsUint(); }
STORAGE_DEFINE_ID_TYPE(Gid);
STORAGE_DEFINE_ID_TYPE(LabelId);
STORAGE_DEFINE_ID_TYPE(PropertyId);
STORAGE_DEFINE_ID_TYPE(EdgeTypeId);
#undef STORAGE_DEFINE_ID_TYPE
} // namespace memgraph::storage::v3
namespace std {
template <>
struct hash<memgraph::storage::v3::Gid> {
size_t operator()(const memgraph::storage::v3::Gid &id) const noexcept { return id.AsUint(); }
};
template <>
struct hash<memgraph::storage::v3::LabelId> {
size_t operator()(const memgraph::storage::v3::LabelId &id) const noexcept { return id.AsUint(); }
};
template <>
struct hash<memgraph::storage::v3::PropertyId> {
size_t operator()(const memgraph::storage::v3::PropertyId &id) const noexcept { return id.AsUint(); }
};
template <>
struct hash<memgraph::storage::v3::EdgeTypeId> {
size_t operator()(const memgraph::storage::v3::EdgeTypeId &id) const noexcept { return id.AsUint(); }
};
} // namespace std

713
src/storage/v3/indices.cpp Normal file
View File

@ -0,0 +1,713 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "indices.hpp"
#include <limits>
#include "storage/v3/mvcc.hpp"
#include "storage/v3/property_value.hpp"
#include "utils/bound.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage::v3 {
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{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_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{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_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{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_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{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_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;
}
} // 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) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
auto [it, emplaced] = index_.emplace(std::piecewise_construct, std::forward_as_tuple(label), std::forward_as_tuple());
if (!emplaced) {
// Index already exists.
return false;
}
try {
auto acc = it->second.access();
for (Vertex &vertex : vertices) {
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
continue;
}
acc.insert(Entry{&vertex, 0});
}
} catch (const utils::OutOfMemoryException &) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index_.erase(it);
throw;
}
return true;
}
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();
}
}
bool LabelPropertyIndex::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 LabelPropertyIndex::Entry::operator==(const Entry &rhs) const {
return value == rhs.value && vertex == rhs.vertex && timestamp == rhs.timestamp;
}
bool LabelPropertyIndex::Entry::operator<(const PropertyValue &rhs) const { return value < rhs; }
bool LabelPropertyIndex::Entry::operator==(const PropertyValue &rhs) const { 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) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
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;
}
try {
auto acc = it->second.access();
for (Vertex &vertex : vertices) {
if (vertex.deleted || !utils::Contains(vertex.labels, label)) {
continue;
}
auto value = vertex.properties.GetProperty(property);
if (value.IsNull()) {
continue;
}
acc.insert(Entry{std::move(value), &vertex, 0});
}
} catch (const utils::OutOfMemoryException &) {
utils::MemoryTracker::OutOfMemoryExceptionBlocker oom_exception_blocker;
index_.erase(it);
throw;
}
return true;
}
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() < static_cast<double>(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 {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};
}
LabelPropertyIndex::Iterable::Iterator LabelPropertyIndex::Iterable::end() { return {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 static_cast<int64_t>(
acc.estimate_count(value, static_cast<int>(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 static_cast<int64_t>(acc.estimate_average_number_of_equals(
[](const auto &first, const auto &second) { return first.value == second.value; },
static_cast<int>(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 static_cast<int64_t>(
acc.estimate_range_count(lower, upper, static_cast<int>(utils::SkipListLayerForCountEstimation(acc.size()))));
}
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::v3

282
src/storage/v3/indices.hpp Normal file
View File

@ -0,0 +1,282 @@
// 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 <tuple>
#include <utility>
#include "storage/v3/config.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/vertex_accessor.hpp"
#include "utils/bound.hpp"
#include "utils/logging.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3 {
struct Indices;
struct Constraints;
class LabelIndex {
private:
struct Entry {
Vertex *vertex;
uint64_t timestamp;
bool operator<(const Entry &rhs) const {
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; }
};
struct LabelStorage {
LabelId label;
utils::SkipList<Entry> vertices;
bool operator<(const LabelStorage &rhs) const { return label < rhs.label; }
bool operator<(LabelId rhs) const { return label < rhs; }
bool operator==(const LabelStorage &rhs) const { return label == rhs.label; }
bool operator==(LabelId rhs) const { 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);
/// 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 {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::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 {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 static_cast<int64_t>(it->second.size());
}
void Clear() { index_.clear(); }
void RunGC();
private:
std::map<LabelId, utils::SkipList<Entry>> index_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
};
class 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:
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);
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 {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 static_cast<int64_t>(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;
void Clear() { index_.clear(); }
void RunGC();
private:
std::map<std::pair<LabelId, PropertyId>, utils::SkipList<Entry>> index_;
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::v3

View File

@ -0,0 +1,20 @@
// 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 <cstdint>
namespace memgraph::storage::v3 {
enum class IsolationLevel : std::uint8_t { SNAPSHOT_ISOLATION, READ_COMMITTED, READ_UNCOMMITTED };
} // namespace memgraph::storage::v3

134
src/storage/v3/mvcc.hpp Normal file
View File

@ -0,0 +1,134 @@
// 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 "storage/v3/property_value.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/view.hpp"
namespace memgraph::storage::v3 {
/// This function iterates through the undo buffers from an object (starting
/// from the supplied delta) and determines what deltas should be applied to get
/// the currently visible version of the object. When the function finds a delta
/// that should be applied it calls the callback function with the delta that
/// should be applied passed as a parameter to the callback. It is up to the
/// caller to apply the deltas.
template <typename TCallback>
inline void ApplyDeltasForRead(Transaction *transaction, const Delta *delta, View view, const TCallback &callback) {
// if the transaction is not committed, then its deltas have transaction_id for the timestamp, otherwise they have
// its commit timestamp set.
// This allows the transaction to see its changes even though it's committed.
const auto commit_timestamp = transaction->commit_timestamp
? transaction->commit_timestamp->load(std::memory_order_acquire)
: transaction->transaction_id;
while (delta != nullptr) {
auto ts = delta->timestamp->load(std::memory_order_acquire);
auto cid = delta->command_id;
// For SNAPSHOT ISOLATION -> we can only see the changes which were committed before the start of the current
// transaction
//
// For READ COMMITTED -> we can only see the changes which are committed. Commit timestamps of
// uncommitted changes are set to the transaction id of the transaction that made the change. Transaction id is
// always higher than start or commit timestamps so we know if the timestamp is lower than the initial transaction
// id value, that the change is committed.
//
// For READ UNCOMMITTED -> we accept any change.
if ((transaction->isolation_level == IsolationLevel::SNAPSHOT_ISOLATION && ts < transaction->start_timestamp) ||
(transaction->isolation_level == IsolationLevel::READ_COMMITTED && ts < kTransactionInitialId) ||
(transaction->isolation_level == IsolationLevel::READ_UNCOMMITTED)) {
break;
}
// We shouldn't undo our newest changes because the user requested a NEW
// view of the database.
if (view == View::NEW && ts == commit_timestamp && cid <= transaction->command_id) {
break;
}
// 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) {
break;
}
// This delta must be applied, call the callback.
callback(*delta);
// Move to the next delta.
delta = delta->next.load(std::memory_order_acquire);
}
}
/// This function prepares the object for a write. It checks whether there are
/// any serialization errors in the process (eg. the object can't be written to
/// from this transaction because it is being written to from another
/// transaction) and returns a `bool` value indicating whether the caller can
/// proceed with a write operation.
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 || ts < transaction->start_timestamp) {
return true;
}
transaction->must_abort = true;
return false;
}
/// This function creates a `DELETE_OBJECT` delta in the transaction and returns
/// a pointer to the created delta. It doesn't perform any linking of the delta
/// and is primarily used to create the first delta for an object (that must be
/// a `DELETE_OBJECT` delta).
/// @throw std::bad_alloc
inline Delta *CreateDeleteObjectDelta(Transaction *transaction) {
transaction->EnsureCommitTimestampExists();
return &transaction->deltas.emplace_back(Delta::DeleteObjectTag(), transaction->commit_timestamp.get(),
transaction->command_id);
}
/// 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
template <typename TObj, class... Args>
inline void CreateAndLinkDelta(Transaction *transaction, TObj *object, Args &&...args) {
transaction->EnsureCommitTimestampExists();
auto delta = &transaction->deltas.emplace_back(std::forward<Args>(args)..., transaction->commit_timestamp.get(),
transaction->command_id);
// The operations are written in such order so that both `next` and `prev`
// chains are valid at all times. The chains must be valid at all times
// because garbage collection (which traverses the chains) is done
// concurrently (as well as other execution threads).
// 1. We need to set the next delta of the new delta to the existing delta.
delta->next.store(object->delta, std::memory_order_release);
// 2. We need to set the previous delta of the new delta to the object.
delta->prev.Set(object);
// 3. We need to set the previous delta of the existing delta to the new
// delta. After this point the garbage collector will be able to see the new
// delta but won't modify it until we are done with all of our modifications.
if (object->delta) {
object->delta->prev.Set(delta);
}
// 4. Finally, we need to set the object's delta to the new delta. The garbage
// collector and other transactions will acquire the object lock to read the
// delta from the object. Because the lock is held during the whole time this
// modification is being done, everybody else will wait until we are fully
// done with our modification before they read the object's delta value.
object->delta = delta;
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,98 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <string>
#include <string_view>
#include "utils/logging.hpp"
#include "utils/skip_list.hpp"
namespace memgraph::storage::v3 {
class NameIdMapper final {
private:
struct MapNameToId {
std::string name;
uint64_t id;
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; }
};
struct MapIdToName {
uint64_t id;
std::string name;
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) const { return id < other; }
bool operator==(uint64_t other) const { return id == other; }
};
public:
/// @throw std::bad_alloc if unable to insert a new mapping
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{0};
if (found == name_to_id_acc.end()) {
uint64_t new_id = counter_.fetch_add(1, std::memory_order_acq_rel);
// Try to insert the mapping with the `new_id`, but use the id that is in
// the object itself. The object that cointains the mapping is designed to
// be a map, so that if the inserted name already exists `insert` will
// return an iterator to the existing item. This prevents assignment of
// two IDs to the same name when the mapping is being inserted
// concurrently from two threads. One ID is wasted in that case, though.
id = name_to_id_acc.insert({std::string(name), new_id}).first->id;
} else {
id = found->id;
}
auto id_to_name_acc = id_to_name_.access();
// We have to try to insert the ID to name mapping even if we are not the
// one who assigned the ID because we have to make sure that after this
// method returns that both mappings exist.
if (id_to_name_acc.find(id) == id_to_name_acc.end()) {
// We first try to find the `id` in the map to avoid making an unnecessary
// temporary memory allocation when the object already exists.
id_to_name_acc.insert({id, std::string(name)});
}
return id;
}
// NOTE: Currently this function returns a `const std::string &` instead of a
// `std::string` to avoid making unnecessary copies of the string.
// Usually, this wouldn't be correct because the accessor to the
// `utils::SkipList` is destroyed in this function and that removes the
// guarantee that the reference to the value contained in the list will be
// valid.
// 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 {
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!");
return result->name;
}
private:
std::atomic<uint64_t> counter_{0};
utils::SkipList<MapNameToId> name_to_id_;
utils::SkipList<MapIdToName> id_to_name_;
};
} // namespace memgraph::storage::v3

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,72 @@
// 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 <map>
#include "storage/v3/id_types.hpp"
#include "storage/v3/property_value.hpp"
namespace memgraph::storage::v3 {
class PropertyStore {
static_assert(std::endian::native == std::endian::little,
"PropertyStore supports only architectures using little-endian.");
public:
PropertyStore();
PropertyStore(const PropertyStore &) = delete;
PropertyStore(PropertyStore &&other) noexcept;
PropertyStore &operator=(const PropertyStore &) = delete;
PropertyStore &operator=(PropertyStore &&other) noexcept;
~PropertyStore();
/// Returns the currently stored value for property `property`. If the
/// property doesn't exist a Null value is returned. The time complexity of
/// this function is O(n).
/// @throw std::bad_alloc
PropertyValue GetProperty(PropertyId property) const;
/// Checks whether the property `property` exists in the store. The time
/// complexity of this function is O(n).
bool HasProperty(PropertyId property) 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
/// O(n).
bool IsPropertyEqual(PropertyId property, const PropertyValue &value) const;
/// Returns all properties currently stored in the store. The time complexity
/// of this function is O(n).
/// @throw std::bad_alloc
std::map<PropertyId, PropertyValue> Properties() const;
/// Set a property value and return `true` if insertion took place. `false` is
/// returned if assignment took place. The time complexity of this function is
/// O(n).
/// @throw std::bad_alloc
bool SetProperty(PropertyId property, const PropertyValue &value);
/// 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).
/// @throw std::bad_alloc
bool ClearProperties();
private:
uint8_t buffer_[sizeof(uint64_t) + sizeof(uint8_t *)];
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,490 @@
// 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 <iostream>
#include <map>
#include <string>
#include <vector>
#include "storage/v3/temporal.hpp"
#include "utils/algorithm.hpp"
#include "utils/exceptions.hpp"
namespace memgraph::storage::v3 {
/// An exception raised by the PropertyValue. Typically when trying to perform
/// operations (such as addition) on PropertyValues of incompatible Types.
class PropertyValueException : public utils::BasicException {
public:
using utils::BasicException::BasicException;
};
/// Encapsulation of a value and its type in a class that has no compile-time
/// info about the type.
///
/// Values can be of a number of predefined types that are enumerated in
/// PropertyValue::Type. Each such type corresponds to exactly one C++ type.
class PropertyValue {
public:
/// A value type, each type corresponds to exactly one C++ type.
enum class Type : uint8_t {
Null = 0,
Bool = 1,
Int = 2,
Double = 3,
String = 4,
List = 5,
Map = 6,
TemporalData = 7
};
static bool AreComparableTypes(Type a, Type b) {
return (a == b) || (a == Type::Int && b == Type::Double) || (a == Type::Double && b == Type::Int);
}
/// Make a Null value
PropertyValue() {}
// constructors for primitive types
explicit PropertyValue(const bool value) : type_(Type::Bool) { bool_v = value; }
explicit PropertyValue(const int value) : type_(Type::Int) { int_v = value; }
explicit PropertyValue(const int64_t value) : type_(Type::Int) { int_v = value; }
explicit PropertyValue(const double value) : type_(Type::Double) { double_v = value; }
explicit PropertyValue(const TemporalData value) : type_{Type::TemporalData} { temporal_data_v = value; }
// copy constructors for non-primitive types
/// @throw std::bad_alloc
explicit PropertyValue(const std::string &value) : type_(Type::String) { new (&string_v) std::string(value); }
/// @throw std::bad_alloc
/// @throw std::length_error if length of value exceeds
/// std::string::max_length().
explicit PropertyValue(const char *value) : type_(Type::String) { new (&string_v) std::string(value); }
/// @throw std::bad_alloc
explicit PropertyValue(const std::vector<PropertyValue> &value) : type_(Type::List) {
new (&list_v) std::vector<PropertyValue>(value);
}
/// @throw std::bad_alloc
explicit PropertyValue(const std::map<std::string, PropertyValue> &value) : type_(Type::Map) {
new (&map_v) std::map<std::string, PropertyValue>(value);
}
// move constructors for non-primitive types
explicit PropertyValue(std::string &&value) noexcept : type_(Type::String) {
new (&string_v) std::string(std::move(value));
}
explicit PropertyValue(std::vector<PropertyValue> &&value) noexcept : type_(Type::List) {
new (&list_v) std::vector<PropertyValue>(std::move(value));
}
explicit PropertyValue(std::map<std::string, PropertyValue> &&value) noexcept : type_(Type::Map) {
new (&map_v) std::map<std::string, PropertyValue>(std::move(value));
}
// copy constructor
/// @throw std::bad_alloc
PropertyValue(const PropertyValue &other);
// move constructor
PropertyValue(PropertyValue &&other) noexcept;
// copy assignment
/// @throw std::bad_alloc
PropertyValue &operator=(const PropertyValue &other);
// move assignment
PropertyValue &operator=(PropertyValue &&other) noexcept;
// TODO: Implement copy assignment operators for primitive types.
// TODO: Implement copy and move assignment operators for non-primitive types.
~PropertyValue() { DestroyValue(); }
Type type() const { return type_; }
// type checkers
bool IsNull() const { return type_ == Type::Null; }
bool IsBool() const { return type_ == Type::Bool; }
bool IsInt() const { return type_ == Type::Int; }
bool IsDouble() const { return type_ == Type::Double; }
bool IsString() const { return type_ == Type::String; }
bool IsList() const { return type_ == Type::List; }
bool IsMap() const { return type_ == Type::Map; }
bool IsTemporalData() const { return type_ == Type::TemporalData; }
// value getters for primitive types
/// @throw PropertyValueException if value isn't of correct type.
bool ValueBool() const {
if (type_ != Type::Bool) {
throw PropertyValueException("The value isn't a bool!");
}
return bool_v;
}
/// @throw PropertyValueException if value isn't of correct type.
int64_t ValueInt() const {
if (type_ != Type::Int) {
throw PropertyValueException("The value isn't an int!");
}
return int_v;
}
/// @throw PropertyValueException if value isn't of correct type.
double ValueDouble() const {
if (type_ != Type::Double) {
throw PropertyValueException("The value isn't a double!");
}
return double_v;
}
/// @throw PropertyValueException if value isn't of correct type.
TemporalData ValueTemporalData() const {
if (type_ != Type::TemporalData) {
throw PropertyValueException("The value isn't a temporal data!");
}
return temporal_data_v;
}
// const value getters for non-primitive types
/// @throw PropertyValueException if value isn't of correct type.
const std::string &ValueString() const {
if (type_ != Type::String) {
throw PropertyValueException("The value isn't a string!");
}
return string_v;
}
/// @throw PropertyValueException if value isn't of correct type.
const std::vector<PropertyValue> &ValueList() const {
if (type_ != Type::List) {
throw PropertyValueException("The value isn't a list!");
}
return list_v;
}
/// @throw PropertyValueException if value isn't of correct type.
const std::map<std::string, PropertyValue> &ValueMap() const {
if (type_ != Type::Map) {
throw PropertyValueException("The value isn't a map!");
}
return map_v;
}
// reference value getters for non-primitive types
/// @throw PropertyValueException if value isn't of correct type.
std::string &ValueString() {
if (type_ != Type::String) {
throw PropertyValueException("The value isn't a string!");
}
return string_v;
}
/// @throw PropertyValueException if value isn't of correct type.
std::vector<PropertyValue> &ValueList() {
if (type_ != Type::List) {
throw PropertyValueException("The value isn't a list!");
}
return list_v;
}
/// @throw PropertyValueException if value isn't of correct type.
std::map<std::string, PropertyValue> &ValueMap() {
if (type_ != Type::Map) {
throw PropertyValueException("The value isn't a map!");
}
return map_v;
}
private:
void DestroyValue() noexcept;
union {
bool bool_v;
int64_t int_v;
double double_v;
std::string string_v;
std::vector<PropertyValue> list_v;
std::map<std::string, PropertyValue> map_v;
TemporalData temporal_data_v;
};
Type type_{Type::Null};
};
// stream output
/// @throw anything std::ostream::operator<< may throw.
inline std::ostream &operator<<(std::ostream &os, const PropertyValue::Type type) {
switch (type) {
case PropertyValue::Type::Null:
return os << "null";
case PropertyValue::Type::Bool:
return os << "bool";
case PropertyValue::Type::Int:
return os << "int";
case PropertyValue::Type::Double:
return os << "double";
case PropertyValue::Type::String:
return os << "string";
case PropertyValue::Type::List:
return os << "list";
case PropertyValue::Type::Map:
return os << "map";
case PropertyValue::Type::TemporalData:
return os << "temporal data";
}
}
/// @throw anything std::ostream::operator<< may throw.
inline std::ostream &operator<<(std::ostream &os, const PropertyValue &value) {
switch (value.type()) {
case PropertyValue::Type::Null:
return os << "null";
case PropertyValue::Type::Bool:
return os << (value.ValueBool() ? "true" : "false");
case PropertyValue::Type::Int:
return os << value.ValueInt();
case PropertyValue::Type::Double:
return os << value.ValueDouble();
case PropertyValue::Type::String:
return os << value.ValueString();
case PropertyValue::Type::List:
os << "[";
utils::PrintIterable(os, value.ValueList());
return os << "]";
case PropertyValue::Type::Map:
os << "{";
utils::PrintIterable(os, value.ValueMap(), ", ",
[](auto &stream, const auto &pair) { stream << pair.first << ": " << pair.second; });
return os << "}";
case PropertyValue::Type::TemporalData:
return os << fmt::format("type: {}, microseconds: {}", TemporalTypeTostring(value.ValueTemporalData().type),
value.ValueTemporalData().microseconds);
}
}
// NOTE: The logic in this function *MUST* be equal to the logic in
// `PropertyStore::ComparePropertyValue`. If you change this operator make sure
// to change the function so that they have identical functionality.
inline bool operator==(const PropertyValue &first, const PropertyValue &second) {
if (!PropertyValue::AreComparableTypes(first.type(), second.type())) return false;
switch (first.type()) {
case PropertyValue::Type::Null:
return true;
case PropertyValue::Type::Bool:
return first.ValueBool() == second.ValueBool();
case PropertyValue::Type::Int:
if (second.type() == PropertyValue::Type::Double) {
return static_cast<double>(first.ValueInt()) == second.ValueDouble();
} else {
return first.ValueInt() == second.ValueInt();
}
case PropertyValue::Type::Double:
if (second.type() == PropertyValue::Type::Double) {
return first.ValueDouble() == second.ValueDouble();
} else {
return first.ValueDouble() == static_cast<double>(second.ValueInt());
}
case PropertyValue::Type::String:
return first.ValueString() == second.ValueString();
case PropertyValue::Type::List:
return first.ValueList() == second.ValueList();
case PropertyValue::Type::Map:
return first.ValueMap() == second.ValueMap();
case PropertyValue::Type::TemporalData:
return first.ValueTemporalData() == second.ValueTemporalData();
}
}
inline bool operator<(const PropertyValue &first, const PropertyValue &second) {
if (!PropertyValue::AreComparableTypes(first.type(), second.type())) return first.type() < second.type();
switch (first.type()) {
case PropertyValue::Type::Null:
return false;
case PropertyValue::Type::Bool:
return first.ValueBool() < second.ValueBool();
case PropertyValue::Type::Int:
if (second.type() == PropertyValue::Type::Double) {
return static_cast<double>(first.ValueInt()) < second.ValueDouble();
} else {
return first.ValueInt() < second.ValueInt();
}
case PropertyValue::Type::Double:
if (second.type() == PropertyValue::Type::Double) {
return first.ValueDouble() < second.ValueDouble();
} else {
return first.ValueDouble() < static_cast<double>(second.ValueInt());
}
case PropertyValue::Type::String:
return first.ValueString() < second.ValueString();
case PropertyValue::Type::List:
return first.ValueList() < second.ValueList();
case PropertyValue::Type::Map:
return first.ValueMap() < second.ValueMap();
case PropertyValue::Type::TemporalData:
return first.ValueTemporalData() < second.ValueTemporalData();
}
}
inline PropertyValue::PropertyValue(const PropertyValue &other) : type_(other.type_) {
switch (other.type_) {
case Type::Null:
return;
case Type::Bool:
this->bool_v = other.bool_v;
return;
case Type::Int:
this->int_v = other.int_v;
return;
case Type::Double:
this->double_v = other.double_v;
return;
case Type::String:
new (&string_v) std::string(other.string_v);
return;
case Type::List:
new (&list_v) std::vector<PropertyValue>(other.list_v);
return;
case Type::Map:
new (&map_v) std::map<std::string, PropertyValue>(other.map_v);
return;
case Type::TemporalData:
this->temporal_data_v = other.temporal_data_v;
return;
}
}
inline PropertyValue::PropertyValue(PropertyValue &&other) noexcept : type_(other.type_) {
switch (other.type_) {
case Type::Null:
break;
case Type::Bool:
this->bool_v = other.bool_v;
break;
case Type::Int:
this->int_v = other.int_v;
break;
case Type::Double:
this->double_v = other.double_v;
break;
case Type::String:
new (&string_v) std::string(std::move(other.string_v));
break;
case Type::List:
new (&list_v) std::vector<PropertyValue>(std::move(other.list_v));
break;
case Type::Map:
new (&map_v) std::map<std::string, PropertyValue>(std::move(other.map_v));
break;
case Type::TemporalData:
this->temporal_data_v = other.temporal_data_v;
break;
}
// reset the type of other
other.DestroyValue();
other.type_ = Type::Null;
}
inline PropertyValue &PropertyValue::operator=(const PropertyValue &other) {
if (this == &other) return *this;
DestroyValue();
type_ = other.type_;
switch (other.type_) {
case Type::Null:
break;
case Type::Bool:
this->bool_v = other.bool_v;
break;
case Type::Int:
this->int_v = other.int_v;
break;
case Type::Double:
this->double_v = other.double_v;
break;
case Type::String:
new (&string_v) std::string(other.string_v);
break;
case Type::List:
new (&list_v) std::vector<PropertyValue>(other.list_v);
break;
case Type::Map:
new (&map_v) std::map<std::string, PropertyValue>(other.map_v);
break;
case Type::TemporalData:
this->temporal_data_v = other.temporal_data_v;
break;
}
return *this;
}
inline PropertyValue &PropertyValue::operator=(PropertyValue &&other) noexcept {
if (this == &other) return *this;
DestroyValue();
type_ = other.type_;
switch (other.type_) {
case Type::Null:
break;
case Type::Bool:
this->bool_v = other.bool_v;
break;
case Type::Int:
this->int_v = other.int_v;
break;
case Type::Double:
this->double_v = other.double_v;
break;
case Type::String:
new (&string_v) std::string(std::move(other.string_v));
break;
case Type::List:
new (&list_v) std::vector<PropertyValue>(std::move(other.list_v));
break;
case Type::Map:
new (&map_v) std::map<std::string, PropertyValue>(std::move(other.map_v));
break;
case Type::TemporalData:
this->temporal_data_v = other.temporal_data_v;
break;
}
// reset the type of other
other.DestroyValue();
other.type_ = Type::Null;
return *this;
}
inline void PropertyValue::DestroyValue() noexcept {
switch (type_) {
// destructor for primitive types does nothing
case Type::Null:
case Type::Bool:
case Type::Int:
case Type::Double:
case Type::TemporalData:
return;
// destructor for non primitive types since we used placement new
case Type::String:
std::destroy_at(&string_v);
return;
case Type::List:
std::destroy_at(&list_v);
return;
case Type::Map:
std::destroy_at(&map_v);
return;
}
}
} // namespace memgraph::storage::v3

2
src/storage/v3/replication/.gitignore vendored Normal file
View File

@ -0,0 +1,2 @@
# autogenerated files
rpc.hpp

View File

@ -0,0 +1,44 @@
// 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 <chrono>
#include <optional>
#include <string>
namespace memgraph::storage::v3::replication {
struct ReplicationClientConfig {
std::optional<double> timeout;
// The default delay between main checking/pinging replicas is 1s because
// that seems like a reasonable timeframe in which main should notice a
// replica is down.
std::chrono::seconds replica_check_frequency{1};
struct SSL {
std::string key_file;
std::string cert_file;
};
std::optional<SSL> ssl;
};
struct ReplicationServerConfig {
struct SSL {
std::string key_file;
std::string cert_file;
std::string ca_file;
bool verify_peer;
};
std::optional<SSL> ssl;
};
} // namespace memgraph::storage::v3::replication

View File

@ -0,0 +1,19 @@
// 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 <cstdint>
namespace memgraph::storage::v3::replication {
enum class ReplicationMode : std::uint8_t { SYNC, ASYNC };
enum class ReplicaState : std::uint8_t { READY, REPLICATING, RECOVERY, INVALID };
} // namespace memgraph::storage::v3::replication

View File

@ -0,0 +1,626 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/replication/replication_client.hpp"
#include <algorithm>
#include <type_traits>
#include "storage/v3/durability/durability.hpp"
#include "storage/v3/replication/config.hpp"
#include "storage/v3/replication/enums.hpp"
#include "storage/v3/transaction.hpp"
#include "utils/file_locker.hpp"
#include "utils/logging.hpp"
#include "utils/message.hpp"
namespace memgraph::storage::v3 {
namespace {
template <typename>
[[maybe_unused]] inline constexpr bool always_false_v = false;
} // namespace
////// ReplicationClient //////
Storage::ReplicationClient::ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
const replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
: name_(std::move(name)), storage_(storage), mode_(mode) {
if (config.ssl) {
rpc_context_.emplace(config.ssl->key_file, config.ssl->cert_file);
} else {
rpc_context_.emplace();
}
rpc_client_.emplace(endpoint, &*rpc_context_);
TryInitializeClientSync();
if (config.timeout && replica_state_ != replication::ReplicaState::INVALID) {
timeout_.emplace(*config.timeout);
timeout_dispatcher_.emplace();
}
// Help the user to get the most accurate replica state possible.
if (config.replica_check_frequency > std::chrono::seconds(0)) {
replica_checker_.Run("Replica Checker", config.replica_check_frequency, [&] { FrequentCheck(); });
}
}
void Storage::ReplicationClient::TryInitializeClientAsync() {
thread_pool_.AddTask([this] {
rpc_client_->Abort();
this->TryInitializeClientSync();
});
}
void Storage::ReplicationClient::FrequentCheck() {
const auto is_success = std::invoke([this]() {
try {
auto stream{rpc_client_->Stream<replication::FrequentHeartbeatRpc>()};
const auto response = stream.AwaitResponse();
return response.success;
} catch (const rpc::RpcFailedException &) {
return false;
}
});
// States: READY, REPLICATING, RECOVERY, INVALID
// If success && ready, replicating, recovery -> stay the same because something good is going on.
// If success && INVALID -> [it's possible that replica came back to life] -> TryInitializeClient.
// If fail -> [replica is not reachable at all] -> INVALID state.
// NOTE: TryInitializeClient might return nothing if there is a branching point.
// NOTE: The early return pattern simplified the code, but the behavior should be as explained.
if (!is_success) {
replica_state_.store(replication::ReplicaState::INVALID);
return;
}
if (replica_state_.load() == replication::ReplicaState::INVALID) {
TryInitializeClientAsync();
}
}
/// @throws rpc::RpcFailedException
void Storage::ReplicationClient::InitializeClient() {
uint64_t current_commit_timestamp{kTimestampInitialId};
std::optional<std::string> epoch_id;
{
// epoch_id_ can be changed if we don't take this lock
std::unique_lock engine_guard(storage_->engine_lock_);
epoch_id.emplace(storage_->epoch_id_);
}
auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(storage_->last_commit_timestamp_, std::move(*epoch_id))};
const auto response = stream.AwaitResponse();
std::optional<uint64_t> branching_point;
if (response.epoch_id != storage_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) {
const auto &epoch_history = storage_->epoch_history_;
const auto epoch_info_iter =
std::find_if(epoch_history.crbegin(), epoch_history.crend(),
[&](const auto &epoch_info) { return epoch_info.first == response.epoch_id; });
if (epoch_info_iter == epoch_history.crend()) {
branching_point = 0;
} else if (epoch_info_iter->second != response.current_commit_timestamp) {
branching_point = epoch_info_iter->second;
}
}
if (branching_point) {
spdlog::error(
"Replica {} cannot be used with this instance. Please start a clean "
"instance of Memgraph server on the specified endpoint.",
name_);
return;
}
current_commit_timestamp = response.current_commit_timestamp;
spdlog::trace("Current timestamp on replica: {}", current_commit_timestamp);
spdlog::trace("Current timestamp on main: {}", storage_->last_commit_timestamp_.load());
if (current_commit_timestamp == storage_->last_commit_timestamp_.load()) {
spdlog::debug("Replica '{}' up to date", name_);
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::READY);
} else {
spdlog::debug("Replica '{}' is behind", name_);
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::RECOVERY);
}
thread_pool_.AddTask([=, this] { this->RecoverReplica(current_commit_timestamp); });
}
}
void Storage::ReplicationClient::TryInitializeClientSync() {
try {
InitializeClient();
} catch (const rpc::RpcFailedException &) {
std::unique_lock client_guarde{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
spdlog::error(utils::MessageWithLink("Failed to connect to replica {} at the endpoint {}.", name_,
rpc_client_->Endpoint(), "https://memgr.ph/replication"));
}
}
void Storage::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) {
auto stream{rpc_client_->Stream<replication::SnapshotRpc>()};
replication::Encoder encoder(stream.GetBuilder());
encoder.WriteFile(path);
return stream.AwaitResponse();
}
replication::WalFilesRes Storage::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())};
replication::Encoder encoder(stream.GetBuilder());
for (const auto &wal : wal_files) {
spdlog::debug("Sending wal file: {}", wal);
encoder.WriteFile(wal);
}
return stream.AwaitResponse();
}
void Storage::ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
std::unique_lock guard(client_lock_);
const auto status = replica_state_.load();
switch (status) {
case replication::ReplicaState::RECOVERY:
spdlog::debug("Replica {} is behind MAIN instance", name_);
return;
case replication::ReplicaState::REPLICATING:
spdlog::debug("Replica {} missed a transaction", name_);
// We missed a transaction because we're still replicating
// the previous transaction so we need to go to RECOVERY
// state to catch up with the missing transaction
// We cannot queue the recovery process here because
// an error can happen while we're replicating the previous
// transaction after which the client should go to
// INVALID state before starting the recovery process
replica_state_.store(replication::ReplicaState::RECOVERY);
return;
case replication::ReplicaState::INVALID:
HandleRpcFailure();
return;
case replication::ReplicaState::READY:
MG_ASSERT(!replica_stream_);
try {
replica_stream_.emplace(ReplicaStream{this, storage_->last_commit_timestamp_.load(), current_wal_seq_num});
replica_state_.store(replication::ReplicaState::REPLICATING);
} catch (const rpc::RpcFailedException &) {
replica_state_.store(replication::ReplicaState::INVALID);
HandleRpcFailure();
}
return;
}
}
void Storage::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
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return;
}
try {
callback(*replica_stream_);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
}
void Storage::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
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return;
}
if (mode_ == replication::ReplicationMode::ASYNC) {
thread_pool_.AddTask([this] { this->FinalizeTransactionReplicationInternal(); });
} else if (timeout_) {
MG_ASSERT(mode_ == replication::ReplicationMode::SYNC, "Only SYNC replica can have a timeout.");
MG_ASSERT(timeout_dispatcher_, "Timeout thread is missing");
timeout_dispatcher_->WaitForTaskToFinish();
timeout_dispatcher_->active = true;
thread_pool_.AddTask([&, this] {
this->FinalizeTransactionReplicationInternal();
std::unique_lock main_guard(timeout_dispatcher_->main_lock);
// TimerThread can finish waiting for timeout
timeout_dispatcher_->active = false;
// Notify the main thread
timeout_dispatcher_->main_cv.notify_one();
});
timeout_dispatcher_->StartTimeoutTask(*timeout_);
// Wait until one of the threads notifies us that they finished executing
// Both threads should first set the active flag to false
{
std::unique_lock main_guard(timeout_dispatcher_->main_lock);
timeout_dispatcher_->main_cv.wait(main_guard, [&] { return !timeout_dispatcher_->active.load(); });
}
// TODO (antonio2368): Document and/or polish SEMI-SYNC to ASYNC fallback.
if (replica_state_ == replication::ReplicaState::REPLICATING) {
mode_ = replication::ReplicationMode::ASYNC;
timeout_.reset();
// This can only happen if we timeouted so we are sure that
// Timeout task finished
// We need to delete timeout dispatcher AFTER the replication
// finished because it tries to acquire the timeout lock
// and acces the `active` variable`
thread_pool_.AddTask([this] { timeout_dispatcher_.reset(); });
}
} else {
FinalizeTransactionReplicationInternal();
}
}
void Storage::ReplicationClient::FinalizeTransactionReplicationInternal() {
MG_ASSERT(replica_stream_, "Missing stream for transaction deltas");
try {
auto response = replica_stream_->Finalize();
replica_stream_.reset();
std::unique_lock client_guard(client_lock_);
if (!response.success || replica_state_ == replication::ReplicaState::RECOVERY) {
replica_state_.store(replication::ReplicaState::RECOVERY);
thread_pool_.AddTask([&, this] { this->RecoverReplica(response.current_commit_timestamp); });
} else {
replica_state_.store(replication::ReplicaState::READY);
}
} catch (const rpc::RpcFailedException &) {
replica_stream_.reset();
{
std::unique_lock client_guard(client_lock_);
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
}
void Storage::ReplicationClient::RecoverReplica(uint64_t replica_commit) {
while (true) {
auto file_locker = storage_->file_retainer_.AddLocker();
const auto steps = GetRecoverySteps(replica_commit, &file_locker);
for (const auto &recovery_step : steps) {
try {
std::visit(
[&, this]<typename T>(T &&arg) {
using StepType = std::remove_cvref_t<T>;
if constexpr (std::is_same_v<StepType, RecoverySnapshot>) {
spdlog::debug("Sending the latest snapshot file: {}", arg);
auto response = TransferSnapshot(arg);
replica_commit = response.current_commit_timestamp;
} else if constexpr (std::is_same_v<StepType, RecoveryWals>) {
spdlog::debug("Sending the latest wal files");
auto response = TransferWalFiles(arg);
replica_commit = response.current_commit_timestamp;
} else if constexpr (std::is_same_v<StepType, RecoveryCurrentWal>) {
std::unique_lock transaction_guard(storage_->engine_lock_);
if (storage_->wal_file_ && storage_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) {
storage_->wal_file_->DisableFlushing();
transaction_guard.unlock();
spdlog::debug("Sending current wal file");
replica_commit = ReplicateCurrentWal();
storage_->wal_file_->EnableFlushing();
}
} else {
static_assert(always_false_v<T>, "Missing type from variant visitor");
}
},
recovery_step);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
return;
}
}
spdlog::trace("Current timestamp on replica: {}", replica_commit);
// To avoid the situation where we read a correct commit timestamp in
// one thread, and after that another thread commits a different a
// transaction and THEN we set the state to READY in the first thread,
// we set this lock before checking the timestamp.
// We will detect that the state is invalid during the next commit,
// because replication::AppendDeltasRpc sends the last commit timestamp which
// replica checks if it's the same last commit timestamp it received
// and we will go to recovery.
// By adding this lock, we can avoid that, and go to RECOVERY immediately.
std::unique_lock client_guard{client_lock_};
SPDLOG_INFO("Replica timestamp: {}", replica_commit);
SPDLOG_INFO("Last commit: {}", storage_->last_commit_timestamp_);
if (storage_->last_commit_timestamp_.load() == replica_commit) {
replica_state_.store(replication::ReplicaState::READY);
return;
}
}
}
uint64_t Storage::ReplicationClient::ReplicateCurrentWal() {
const auto &wal_file = storage_->wal_file_;
auto stream = TransferCurrentWalFile();
stream.AppendFilename(wal_file->Path().filename());
utils::InputFile file;
MG_ASSERT(file.Open(storage_->wal_file_->Path()), "Failed to open current WAL file!");
const auto [buffer, buffer_size] = wal_file->CurrentFileBuffer();
stream.AppendSize(file.GetSize() + buffer_size);
stream.AppendFileData(&file);
stream.AppendBufferData(buffer, buffer_size);
auto response = stream.Finalize();
return response.current_commit_timestamp;
}
/// This method tries to find the optimal path for recoverying a single replica.
/// Based on the last commit transfered to replica it tries to update the
/// replica using durability files - WALs and Snapshots. WAL files are much
/// smaller in size as they contain only the Deltas (changes) made during the
/// transactions while Snapshots contain all the data. For that reason we prefer
/// WALs as much as possible. As the WAL file that is currently being updated
/// can change during the process we ignore it as much as possible. Also, it
/// uses the transaction lock so lokcing it can be really expensive. After we
/// fetch the list of finalized WALs, we try to find the longest chain of
/// sequential WALs, starting from the latest one, that will update the recovery
/// with the all missed updates. If the WAL chain cannot be created, replica is
/// behind by a lot, so we use the regular recovery process, we send the latest
/// snapshot and all the necessary WAL files, starting from the newest WAL that
/// contains a timestamp before the snapshot. If we registered the existence of
/// the current WAL, we add the sequence number we read from it to the recovery
/// process. After all the other steps are finished, if the current WAL contains
/// the same sequence number, it's the same WAL we read while fetching the
/// 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(
const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) {
// First check if we can recover using the current wal file only
// otherwise save the seq_num of the current wal file
// This lock is also necessary to force the missed transaction to finish.
std::optional<uint64_t> current_wal_seq_num;
std::optional<uint64_t> current_wal_from_timestamp;
if (std::unique_lock transtacion_guard(storage_->engine_lock_); storage_->wal_file_) {
current_wal_seq_num.emplace(storage_->wal_file_->SequenceNumber());
current_wal_from_timestamp.emplace(storage_->wal_file_->FromTimestamp());
}
auto locker_acc = file_locker->Access();
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_, current_wal_seq_num);
MG_ASSERT(wal_files, "Wal files could not be loaded");
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
std::optional<durability::SnapshotDurabilityInfo> latest_snapshot;
if (!snapshot_files.empty()) {
std::sort(snapshot_files.begin(), snapshot_files.end());
latest_snapshot.emplace(std::move(snapshot_files.back()));
}
std::vector<RecoveryStep> recovery_steps;
// No finalized WAL files were found. This means the difference is contained
// inside the current WAL or the snapshot.
if (wal_files->empty()) {
if (current_wal_from_timestamp && replica_commit >= *current_wal_from_timestamp) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Without the finalized WAL containing the current timestamp of replica,
// we cannot know if the difference is only in the current WAL or we need
// to send the snapshot.
if (latest_snapshot) {
locker_acc.AddPath(latest_snapshot->path);
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
}
// if there are no finalized WAL files, snapshot left the current WAL
// as the WAL file containing a transaction before snapshot creation
// so we can be sure that the current WAL is present
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Find the longest chain of WALs for recovery.
// The chain consists ONLY of sequential WALs.
auto rwal_it = wal_files->rbegin();
// if the last finalized WAL is before the replica commit
// then we can recovery only from current WAL
if (rwal_it->to_timestamp <= replica_commit) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
uint64_t previous_seq_num{rwal_it->seq_num};
for (; rwal_it != wal_files->rend(); ++rwal_it) {
// If the difference between two consecutive wal files is not 0 or 1
// we have a missing WAL in our chain
if (previous_seq_num - rwal_it->seq_num > 1) {
break;
}
// Find first WAL that contains up to replica commit, i.e. WAL
// that is before the replica commit or conatins the replica commit
// as the last committed transaction OR we managed to find the first WAL
// file.
if (replica_commit >= rwal_it->from_timestamp || rwal_it->seq_num == 0) {
if (replica_commit >= rwal_it->to_timestamp) {
// We want the WAL after because the replica already contains all the
// commits from this WAL
--rwal_it;
}
std::vector<std::filesystem::path> wal_chain;
auto distance_from_first = std::distance(rwal_it, wal_files->rend() - 1);
// We have managed to create WAL chain
// We need to lock these files and add them to the chain
for (auto result_wal_it = wal_files->begin() + distance_from_first; result_wal_it != wal_files->end();
++result_wal_it) {
locker_acc.AddPath(result_wal_it->path);
wal_chain.push_back(std::move(result_wal_it->path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(wal_chain));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
previous_seq_num = rwal_it->seq_num;
}
MG_ASSERT(latest_snapshot, "Invalid durability state, missing snapshot");
// We didn't manage to find a WAL chain, we need to send the latest snapshot
// with its WALs
locker_acc.AddPath(latest_snapshot->path);
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
std::vector<std::filesystem::path> recovery_wal_files;
auto wal_it = wal_files->begin();
for (; wal_it != wal_files->end(); ++wal_it) {
// Assuming recovery process is correct the snashpot should
// always retain a single WAL that contains a transaction
// before its creation
if (latest_snapshot->start_timestamp < wal_it->to_timestamp) {
if (latest_snapshot->start_timestamp < wal_it->from_timestamp) {
MG_ASSERT(wal_it != wal_files->begin(), "Invalid durability files state");
--wal_it;
}
break;
}
}
for (; wal_it != wal_files->end(); ++wal_it) {
locker_acc.AddPath(wal_it->path);
recovery_wal_files.push_back(std::move(wal_it->path));
}
// We only have a WAL before the snapshot
if (recovery_wal_files.empty()) {
locker_acc.AddPath(wal_files->back().path);
recovery_wal_files.push_back(std::move(wal_files->back().path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(recovery_wal_files));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
////// TimeoutDispatcher //////
void Storage::ReplicationClient::TimeoutDispatcher::WaitForTaskToFinish() {
// Wait for the previous timeout task to finish
std::unique_lock main_guard(main_lock);
main_cv.wait(main_guard, [&] { return finished; });
}
void Storage::ReplicationClient::TimeoutDispatcher::StartTimeoutTask(const double timeout) {
timeout_pool.AddTask([timeout, this] {
finished = false;
using std::chrono::steady_clock;
const auto timeout_duration =
std::chrono::duration_cast<steady_clock::duration>(std::chrono::duration<double>(timeout));
const auto end_time = steady_clock::now() + timeout_duration;
while (active && (steady_clock::now() < end_time)) {
std::this_thread::sleep_for(std::chrono::milliseconds(10));
}
std::unique_lock main_guard(main_lock);
finished = true;
active = false;
main_cv.notify_one();
});
}
////// ReplicaStream //////
Storage::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) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, 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) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, &self_->storage_->name_id_mapper_, delta, edge, final_commit_timestamp);
}
void Storage::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) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeOperation(&encoder, &self_->storage_->name_id_mapper_, operation, label, properties, timestamp);
}
replication::AppendDeltasRes Storage::ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
////// CurrentWalHandler //////
Storage::ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
: self_(self), stream_(self_->rpc_client_->Stream<replication::CurrentWalRpc>()) {}
void Storage::ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteString(filename);
}
void Storage::ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteUint(size);
}
void Storage::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) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteBuffer(buffer, buffer_size);
}
replication::CurrentWalRes Storage::ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,203 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <chrono>
#include <thread>
#include <variant>
#include "rpc/client.hpp"
#include "storage/v3/config.hpp"
#include "storage/v3/delta.hpp"
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/mvcc.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/replication/config.hpp"
#include "storage/v3/replication/enums.hpp"
#include "storage/v3/replication/rpc.hpp"
#include "storage/v3/replication/serialization.hpp"
#include "storage/v3/storage.hpp"
#include "utils/file.hpp"
#include "utils/file_locker.hpp"
#include "utils/spin_lock.hpp"
#include "utils/synchronized.hpp"
#include "utils/thread_pool.hpp"
namespace memgraph::storage::v3 {
class Storage::ReplicationClient {
public:
ReplicationClient(std::string name, Storage *storage, const io::network::Endpoint &endpoint,
replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {});
// Handler used for transfering the current transaction.
class ReplicaStream {
private:
friend class ReplicationClient;
explicit ReplicaStream(ReplicationClient *self, uint64_t previous_commit_timestamp, uint64_t current_seq_num);
public:
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Edge &edge, uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendTransactionEnd(uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp);
private:
/// @throw rpc::RpcFailedException
replication::AppendDeltasRes Finalize();
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::AppendDeltasRpc> stream_;
};
// Handler for transfering the current WAL file whose data is
// contained in the internal buffer and the file.
class CurrentWalHandler {
private:
friend class ReplicationClient;
explicit CurrentWalHandler(ReplicationClient *self);
public:
void AppendFilename(const std::string &filename);
void AppendSize(size_t size);
void AppendFileData(utils::InputFile *file);
void AppendBufferData(const uint8_t *buffer, size_t buffer_size);
/// @throw rpc::RpcFailedException
replication::CurrentWalRes Finalize();
private:
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::CurrentWalRpc> stream_;
};
void StartTransactionReplication(uint64_t current_wal_seq_num);
// Replication clients can be removed at any point
// so to avoid any complexity of checking if the client was removed whenever
// we want to send part of transaction and to avoid adding some GC logic this
// function will run a callback if, after previously callling
// StartTransactionReplication, stream is created.
void IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback);
void FinalizeTransactionReplication();
// Transfer the snapshot file.
// @param path Path of the snapshot file.
replication::SnapshotRes TransferSnapshot(const std::filesystem::path &path);
CurrentWalHandler TransferCurrentWalFile() { return CurrentWalHandler{this}; }
// Transfer the WAL files
replication::WalFilesRes TransferWalFiles(const std::vector<std::filesystem::path> &wal_files);
const auto &Name() const { return name_; }
auto State() const { return replica_state_.load(); }
auto Mode() const { return mode_; }
auto Timeout() const { return timeout_; }
const auto &Endpoint() const { return rpc_client_->Endpoint(); }
private:
void FinalizeTransactionReplicationInternal();
void RecoverReplica(uint64_t replica_commit);
uint64_t ReplicateCurrentWal();
using RecoveryWals = std::vector<std::filesystem::path>;
struct RecoveryCurrentWal {
uint64_t current_wal_seq_num;
explicit RecoveryCurrentWal(const uint64_t current_wal_seq_num) : current_wal_seq_num(current_wal_seq_num) {}
};
using RecoverySnapshot = std::filesystem::path;
using RecoveryStep = std::variant<RecoverySnapshot, RecoveryWals, RecoveryCurrentWal>;
std::vector<RecoveryStep> GetRecoverySteps(uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker);
void FrequentCheck();
void InitializeClient();
void TryInitializeClientSync();
void TryInitializeClientAsync();
void HandleRpcFailure();
std::string name_;
Storage *storage_;
std::optional<communication::ClientContext> rpc_context_;
std::optional<rpc::Client> rpc_client_;
std::optional<ReplicaStream> replica_stream_;
replication::ReplicationMode mode_{replication::ReplicationMode::SYNC};
// Dispatcher class for timeout tasks
struct TimeoutDispatcher {
explicit TimeoutDispatcher(){};
void WaitForTaskToFinish();
void StartTimeoutTask(double timeout);
// If the Timeout task should continue waiting
std::atomic<bool> active{false};
std::mutex main_lock;
std::condition_variable main_cv;
private:
// if the Timeout task finished executing
bool finished{true};
utils::ThreadPool timeout_pool{1};
};
std::optional<double> timeout_;
std::optional<TimeoutDispatcher> timeout_dispatcher_;
utils::SpinLock client_lock_;
// This thread pool is used for background tasks so we don't
// block the main storage thread
// We use only 1 thread for 2 reasons:
// - background tasks ALWAYS contain some kind of RPC communication.
// We can't have multiple RPC communication from a same client
// because that's not logically valid (e.g. you cannot send a snapshot
// and WAL at a same time because WAL will arrive earlier and be applied
// before the snapshot which is not correct)
// - the implementation is simplified as we have a total control of what
// this pool is executing. Also, we can simply queue multiple tasks
// and be sure of the execution order.
// Not having mulitple possible threads in the same client allows us
// to ignore concurrency problems inside the client.
utils::ThreadPool thread_pool_{1};
std::atomic<replication::ReplicaState> replica_state_{replication::ReplicaState::INVALID};
utils::Scheduler replica_checker_;
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,573 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/replication/replication_server.hpp"
#include <atomic>
#include <filesystem>
#include "storage/v3/durability/durability.hpp"
#include "storage/v3/durability/paths.hpp"
#include "storage/v3/durability/serialization.hpp"
#include "storage/v3/durability/snapshot.hpp"
#include "storage/v3/durability/version.hpp"
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/replication/config.hpp"
#include "storage/v3/transaction.hpp"
#include "utils/exceptions.hpp"
namespace memgraph::storage::v3 {
namespace {
std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder *decoder) {
try {
auto timestamp = ReadWalDeltaHeader(decoder);
SPDLOG_INFO(" Timestamp {}", timestamp);
auto delta = ReadWalDeltaData(decoder);
return {timestamp, delta};
} catch (const slk::SlkReaderException &) {
throw utils::BasicException("Missing data!");
} catch (const durability::RecoveryFailure &) {
throw utils::BasicException("Invalid data!");
}
};
} // namespace
Storage::ReplicationServer::ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config)
: storage_(storage) {
// Create RPC server.
if (config.ssl) {
rpc_server_context_.emplace(config.ssl->key_file, config.ssl->cert_file, config.ssl->ca_file,
config.ssl->verify_peer);
} else {
rpc_server_context_.emplace();
}
// NOTE: The replication server must have a single thread for processing
// because there is no need for more processing threads - each replica can
// have only a single main server. Also, the single-threaded guarantee
// simplifies the rest of the implementation.
rpc_server_.emplace(std::move(endpoint), &*rpc_server_context_,
/* workers_count = */ 1);
rpc_server_->Register<replication::HeartbeatRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received HeartbeatRpc");
this->HeartbeatHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
spdlog::debug("Received FrequentHeartbeatRpc");
FrequentHeartbeatHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::AppendDeltasRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received AppendDeltasRpc");
this->AppendDeltasHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::SnapshotRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received SnapshotRpc");
this->SnapshotHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::WalFilesRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received WalFilesRpc");
this->WalFilesHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::CurrentWalRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received CurrentWalRpc");
this->CurrentWalHandler(req_reader, res_builder);
});
rpc_server_->Start();
}
void Storage::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) {
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) {
replication::AppendDeltasReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
auto maybe_epoch_id = decoder.ReadString();
MG_ASSERT(maybe_epoch_id, "Invalid replication message");
if (*maybe_epoch_id != storage_->epoch_id_) {
storage_->epoch_history_.emplace_back(std::move(storage_->epoch_id_), storage_->last_commit_timestamp_);
storage_->epoch_id_ = std::move(*maybe_epoch_id);
}
if (storage_->wal_file_) {
if (req.seq_num > storage_->wal_file_->SequenceNumber() || *maybe_epoch_id != storage_->epoch_id_) {
storage_->wal_file_->FinalizeWal();
storage_->wal_file_.reset();
storage_->wal_seq_num_ = req.seq_num;
} else {
MG_ASSERT(storage_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file");
storage_->wal_seq_num_ = req.seq_num + 1;
}
} else {
storage_->wal_seq_num_ = req.seq_num;
}
if (req.previous_commit_timestamp != storage_->last_commit_timestamp_.load()) {
// Empty the stream
bool transaction_complete = false;
while (!transaction_complete) {
SPDLOG_INFO("Skipping delta");
const auto [timestamp, delta] = ReadDelta(&decoder);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
}
replication::AppendDeltasRes res{false, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
return;
}
ReadAndApplyDelta(&decoder);
replication::AppendDeltasRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
}
void Storage::ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::SnapshotReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->snapshot_directory_);
const auto maybe_snapshot_path = decoder.ReadFile(storage_->snapshot_directory_);
MG_ASSERT(maybe_snapshot_path, "Failed to load snapshot!");
spdlog::info("Received snapshot saved to {}", *maybe_snapshot_path);
std::unique_lock<utils::RWLock> storage_guard(storage_->main_lock_);
// Clear the database
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);
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_->edge_count_, storage_->config_.items);
spdlog::debug("Snapshot loaded successfully");
// If this step is present it should always be the first step of
// the recovery so we use the UUID we read from snasphost
storage_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
storage_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id);
const auto &recovery_info = recovered_snapshot.recovery_info;
storage_->vertex_id_ = recovery_info.next_vertex_id;
storage_->edge_id_ = recovery_info.next_edge_id;
storage_->timestamp_ = std::max(storage_->timestamp_, recovery_info.next_timestamp);
durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &storage_->indices_,
&storage_->constraints_, &storage_->vertices_);
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't load the snapshot because of: {}", e.what());
}
storage_guard.unlock();
replication::SnapshotRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
// Delete other durability files
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
for (const auto &[path, uuid, _] : snapshot_files) {
if (path != *maybe_snapshot_path) {
storage_->file_retainer_.DeleteFile(path);
}
}
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_);
if (wal_files) {
for (const auto &wal_file : *wal_files) {
storage_->file_retainer_.DeleteFile(wal_file.path);
}
storage_->wal_file_.reset();
}
}
void Storage::ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::WalFilesReq req;
slk::Load(&req, req_reader);
const auto wal_file_number = req.file_number;
spdlog::debug("Received WAL files: {}", wal_file_number);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
for (auto i = 0; i < wal_file_number; ++i) {
LoadWal(&decoder);
}
replication::WalFilesRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
}
void Storage::ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::CurrentWalReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
LoadWal(&decoder);
replication::CurrentWalRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
}
void Storage::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);
MG_ASSERT(maybe_wal_path, "Failed to load WAL!");
spdlog::trace("Received WAL saved to {}", *maybe_wal_path);
try {
auto wal_info = durability::ReadWalInfo(*maybe_wal_path);
if (wal_info.seq_num == 0) {
storage_->uuid_ = wal_info.uuid;
}
if (wal_info.epoch_id != storage_->epoch_id_) {
storage_->epoch_history_.emplace_back(wal_info.epoch_id, storage_->last_commit_timestamp_);
storage_->epoch_id_ = std::move(wal_info.epoch_id);
}
if (storage_->wal_file_) {
if (storage_->wal_file_->SequenceNumber() != wal_info.seq_num) {
storage_->wal_file_->FinalizeWal();
storage_->wal_seq_num_ = wal_info.seq_num;
storage_->wal_file_.reset();
}
} else {
storage_->wal_seq_num_ = wal_info.seq_num;
}
durability::Decoder wal;
const auto version = wal.Initialize(*maybe_wal_path, durability::kWalMagic);
if (!version) throw durability::RecoveryFailure("Couldn't read WAL magic and/or version!");
if (!durability::IsVersionSupported(*version)) throw durability::RecoveryFailure("Invalid WAL version!");
wal.SetPosition(wal_info.offset_deltas);
for (size_t i = 0; i < wal_info.num_deltas;) {
i += ReadAndApplyDelta(&wal);
}
spdlog::debug("{} loaded successfully", *maybe_wal_path);
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't recover WAL deltas from {} because of: {}", *maybe_wal_path, e.what());
}
}
Storage::ReplicationServer::~ReplicationServer() {
if (rpc_server_) {
rpc_server_->Shutdown();
rpc_server_->AwaitShutdown();
}
}
uint64_t Storage::ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
auto edge_acc = storage_->edges_.access();
auto vertex_acc = storage_->vertices_.access();
std::optional<std::pair<uint64_t, 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());
} else if (commit_timestamp_and_accessor->first != commit_timestamp) {
throw utils::BasicException("Received more than one transaction!");
}
return &commit_timestamp_and_accessor->second;
};
uint64_t applied_deltas = 0;
auto max_commit_timestamp = storage_->last_commit_timestamp_.load();
for (bool transaction_complete = false; !transaction_complete; ++applied_deltas) {
const auto [timestamp, delta] = ReadDelta(decoder);
if (timestamp > max_commit_timestamp) {
max_commit_timestamp = timestamp;
}
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
if (timestamp < storage_->timestamp_) {
continue;
}
SPDLOG_INFO(" Delta {}", applied_deltas);
switch (delta.type) {
case durability::WalDeltaData::Type::VERTEX_CREATE: {
spdlog::trace(" Create vertex {}", delta.vertex_create_delete.gid.AsUint());
auto *transaction = get_transaction(timestamp);
transaction->CreateVertex(delta.vertex_create_delete.gid);
break;
}
case durability::WalDeltaData::Type::VERTEX_DELETE: {
spdlog::trace(" Delete vertex {}", delta.vertex_create_delete.gid.AsUint());
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_create_delete.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = transaction->DeleteVertex(&*vertex);
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_ADD_LABEL: {
spdlog::trace(" Vertex {} add label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->AddLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_REMOVE_LABEL: {
spdlog::trace(" Vertex {} remove label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->RemoveLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_SET_PROPERTY: {
spdlog::trace(" Vertex {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_edge_set_property.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_CREATE: {
spdlog::trace(" Create edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto *transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edge = transaction->CreateEdge(&*from_vertex, &*to_vertex,
transaction->NameToEdgeType(delta.edge_create_delete.edge_type),
delta.edge_create_delete.gid);
if (edge.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_DELETE: {
spdlog::trace(" Delete edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto *transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edges = from_vertex->OutEdges(View::NEW, {transaction->NameToEdgeType(delta.edge_create_delete.edge_type)},
&*to_vertex);
if (edges.HasError()) throw utils::BasicException("Invalid transaction!");
if (edges->size() != 1) throw utils::BasicException("Invalid transaction!");
auto &edge = (*edges)[0];
auto ret = transaction->DeleteEdge(&edge);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_SET_PROPERTY: {
spdlog::trace(" Edge {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
if (!storage_->config_.items.properties_on_edges)
throw utils::BasicException(
"Can't set properties on edges because properties on edges "
"are disabled!");
auto *transaction = get_transaction(timestamp);
// The following block of code effectively implements `FindEdge` and
// yields an accessor that is only valid for managing the edge's
// properties.
auto edge = edge_acc.find(delta.vertex_edge_set_property.gid);
if (edge == edge_acc.end()) throw utils::BasicException("Invalid transaction!");
// The edge visibility check must be done here manually because we
// don't allow direct access to the edges through the public API.
{
bool is_visible = true;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge->lock);
is_visible = !edge->deleted;
delta = edge->delta;
}
ApplyDeltasForRead(&transaction->transaction_, delta, View::NEW, [&is_visible](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
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;
case Delta::Action::RECREATE_OBJECT: {
is_visible = true;
break;
}
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
}
}
});
if (!is_visible) throw utils::BasicException("Invalid transaction!");
}
EdgeRef edge_ref(&*edge);
// Here we create an edge accessor that we will use to get the
// properties of the edge. The accessor is created with an invalid
// 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->transaction_,
&storage_->indices_,
&storage_->constraints_,
storage_->config_.items};
auto ret = ea.SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::TRANSACTION_END: {
spdlog::trace(" Transaction end");
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
throw utils::BasicException("Invalid data!");
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;
}
case durability::WalDeltaData::Type::LABEL_INDEX_CREATE: {
spdlog::trace(" Create label index on :{}", delta.operation_label.label);
// Need to send the timestamp
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label.label), timestamp))
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_DROP: {
spdlog::trace(" Drop label index on :{}", delta.operation_label.label);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label.label), timestamp))
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
spdlog::trace(" Create label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (!storage_->CreateIndex(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp))
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: {
spdlog::trace(" Drop label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (!storage_->DropIndex(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp))
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
spdlog::trace(" Create existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
auto ret = storage_->CreateExistenceConstraint(
storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp);
if (!ret.HasValue() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
spdlog::trace(" Drop existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (!storage_->DropExistenceConstraint(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property),
timestamp))
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Create unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage_->NameToProperty(prop));
}
auto ret = storage_->CreateUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Drop unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage_->NameToProperty(prop));
}
auto ret = storage_->DropUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (ret != UniqueConstraints::DeletionStatus::SUCCESS) throw utils::BasicException("Invalid transaction!");
break;
}
}
}
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!");
storage_->last_commit_timestamp_ = max_commit_timestamp;
return applied_deltas;
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,47 @@
// 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 "storage/v3/storage.hpp"
namespace memgraph::storage::v3 {
class Storage::ReplicationServer {
public:
explicit ReplicationServer(Storage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config);
ReplicationServer(const ReplicationServer &) = delete;
ReplicationServer(ReplicationServer &&) = delete;
ReplicationServer &operator=(const ReplicationServer &) = delete;
ReplicationServer &operator=(ReplicationServer &&) = delete;
~ReplicationServer();
private:
// RPC handlers
void HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
static void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void LoadWal(replication::Decoder *decoder);
uint64_t ReadAndApplyDelta(durability::BaseDecoder *decoder);
std::optional<communication::ServerContext> rpc_server_context_;
std::optional<rpc::Server> rpc_server_;
Storage *storage_;
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,74 @@
;; 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.
#>cpp
#pragma once
#include <cstdint>
#include <cstring>
#include <string>
#include "rpc/messages.hpp"
#include "slk/serialization.hpp"
#include "slk/streams.hpp"
cpp<#
(lcp:namespace memgraph)
(lcp:namespace storage)
(lcp:namespace v3)
(lcp:namespace replication)
(lcp:define-rpc append-deltas
;; The actual deltas are sent as additional data using the RPC client's
;; streaming API for additional data.
(:request
((previous-commit-timestamp :uint64_t)
(seq-num :uint64_t)))
(:response
((success :bool)
(current-commit-timestamp :uint64_t))))
(lcp:define-rpc heartbeat
(:request
((main-commit-timestamp :uint64_t)
(epoch-id "std::string")))
(:response
((success :bool)
(current-commit-timestamp :uint64_t)
(epoch-id "std::string"))))
;; FrequentHearthbeat is required because calling Heartbeat takes the storage lock.
;; Configured by `replication_replica_check_delay`.
(lcp:define-rpc frequent-heartbeat
(:request ())
(:response ((success :bool))))
(lcp:define-rpc snapshot
(:request ())
(:response
((success :bool)
(current-commit-timestamp :uint64_t))))
(lcp:define-rpc wal-files
(:request ((file-number :uint64_t)))
(:response
((success :bool)
(current-commit-timestamp :uint64_t))))
(lcp:define-rpc current-wal
(:request ())
(:response
((success :bool)
(current-commit-timestamp :uint64_t))))
(lcp:pop-namespace) ;; replication
(lcp:pop-namespace) ;; v3
(lcp:pop-namespace) ;; storage
(lcp:pop-namespace) ;; memgraph

View File

@ -0,0 +1,149 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/replication/serialization.hpp"
namespace memgraph::storage::v3::replication {
////// Encoder //////
void Encoder::WriteMarker(durability::Marker marker) { slk::Save(marker, builder_); }
void Encoder::WriteBool(bool value) {
WriteMarker(durability::Marker::TYPE_BOOL);
slk::Save(value, builder_);
}
void Encoder::WriteUint(uint64_t value) {
WriteMarker(durability::Marker::TYPE_INT);
slk::Save(value, builder_);
}
void Encoder::WriteDouble(double value) {
WriteMarker(durability::Marker::TYPE_DOUBLE);
slk::Save(value, builder_);
}
void Encoder::WriteString(const std::string_view &value) {
WriteMarker(durability::Marker::TYPE_STRING);
slk::Save(value, builder_);
}
void Encoder::WritePropertyValue(const PropertyValue &value) {
WriteMarker(durability::Marker::TYPE_PROPERTY_VALUE);
slk::Save(value, builder_);
}
void Encoder::WriteBuffer(const uint8_t *buffer, const size_t buffer_size) { builder_->Save(buffer, buffer_size); }
void Encoder::WriteFileData(utils::InputFile *file) {
auto file_size = file->GetSize();
uint8_t buffer[utils::kFileBufferSize];
while (file_size > 0) {
const auto chunk_size = std::min(file_size, utils::kFileBufferSize);
file->Read(buffer, chunk_size);
WriteBuffer(buffer, chunk_size);
file_size -= chunk_size;
}
}
void Encoder::WriteFile(const std::filesystem::path &path) {
utils::InputFile file;
MG_ASSERT(file.Open(path), "Failed to open file {}", path);
MG_ASSERT(path.has_filename(), "Path does not have a filename!");
const auto &filename = path.filename().generic_string();
WriteString(filename);
auto file_size = file.GetSize();
WriteUint(file_size);
WriteFileData(&file);
file.Close();
}
////// Decoder //////
std::optional<durability::Marker> Decoder::ReadMarker() {
durability::Marker marker{durability::Marker::TYPE_NULL};
slk::Load(&marker, reader_);
return marker;
}
std::optional<bool> Decoder::ReadBool() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_BOOL) return std::nullopt;
bool value{false};
slk::Load(&value, reader_);
return value;
}
std::optional<uint64_t> Decoder::ReadUint() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_INT) return std::nullopt;
uint64_t value{0};
slk::Load(&value, reader_);
return value;
}
std::optional<double> Decoder::ReadDouble() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_DOUBLE) return std::nullopt;
double value{0.0};
slk::Load(&value, reader_);
return value;
}
std::optional<std::string> Decoder::ReadString() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_STRING) return std::nullopt;
std::string value;
slk::Load(&value, reader_);
return std::move(value);
}
std::optional<PropertyValue> Decoder::ReadPropertyValue() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_PROPERTY_VALUE)
return std::nullopt;
PropertyValue value;
slk::Load(&value, reader_);
return std::move(value);
}
bool Decoder::SkipString() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_STRING) return false;
std::string value;
slk::Load(&value, reader_);
return true;
}
bool Decoder::SkipPropertyValue() {
if (const auto marker = ReadMarker(); !marker || marker != durability::Marker::TYPE_PROPERTY_VALUE) return false;
PropertyValue value;
slk::Load(&value, reader_);
return true;
}
std::optional<std::filesystem::path> Decoder::ReadFile(const std::filesystem::path &directory,
const std::string &suffix) {
MG_ASSERT(std::filesystem::exists(directory) && std::filesystem::is_directory(directory),
"Sent path for streamed files should be a valid directory!");
utils::OutputFile file;
const auto maybe_filename = ReadString();
MG_ASSERT(maybe_filename, "Filename missing for the file");
const auto filename = *maybe_filename + suffix;
auto path = directory / filename;
file.Open(path, utils::OutputFile::Mode::OVERWRITE_EXISTING);
std::optional<size_t> maybe_file_size = ReadUint();
MG_ASSERT(maybe_file_size, "File size missing");
auto file_size = *maybe_file_size;
uint8_t buffer[utils::kFileBufferSize];
while (file_size > 0) {
const auto chunk_size = std::min(file_size, utils::kFileBufferSize);
reader_->Load(buffer, chunk_size);
file.Write(buffer, chunk_size);
file_size -= chunk_size;
}
file.Close();
return std::move(path);
}
} // namespace memgraph::storage::v3::replication

View File

@ -0,0 +1,80 @@
// 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 <filesystem>
#include "slk/streams.hpp"
#include "storage/v3/durability/serialization.hpp"
#include "storage/v3/replication/slk.hpp"
#include "utils/cast.hpp"
#include "utils/file.hpp"
namespace memgraph::storage::v3::replication {
class Encoder final : public durability::BaseEncoder {
public:
explicit Encoder(slk::Builder *builder) : builder_(builder) {}
void WriteMarker(durability::Marker marker) override;
void WriteBool(bool value) override;
void WriteUint(uint64_t value) override;
void WriteDouble(double value) override;
void WriteString(const std::string_view &value) override;
void WritePropertyValue(const PropertyValue &value) override;
void WriteBuffer(const uint8_t *buffer, size_t buffer_size);
void WriteFileData(utils::InputFile *file);
void WriteFile(const std::filesystem::path &path);
private:
slk::Builder *builder_;
};
class Decoder final : public durability::BaseDecoder {
public:
explicit Decoder(slk::Reader *reader) : reader_(reader) {}
std::optional<durability::Marker> ReadMarker() override;
std::optional<bool> ReadBool() override;
std::optional<uint64_t> ReadUint() override;
std::optional<double> ReadDouble() override;
std::optional<std::string> ReadString() override;
std::optional<PropertyValue> ReadPropertyValue() override;
bool SkipString() override;
bool SkipPropertyValue() override;
/// Read the file and save it inside the specified directory.
/// @param directory Directory which will contain the read file.
/// @param suffix Suffix to be added to the received file's filename.
/// @return If the read was successful, path to the read file.
std::optional<std::filesystem::path> ReadFile(const std::filesystem::path &directory, const std::string &suffix = "");
private:
slk::Reader *reader_;
};
} // namespace memgraph::storage::v3::replication

View File

@ -0,0 +1,169 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/replication/slk.hpp"
#include <type_traits>
#include "storage/v3/property_value.hpp"
#include "storage/v3/temporal.hpp"
#include "utils/cast.hpp"
namespace memgraph::slk {
void Save(const storage::v3::Gid &gid, slk::Builder *builder) { slk::Save(gid.AsUint(), builder); }
void Load(storage::v3::Gid *gid, slk::Reader *reader) {
uint64_t value{0};
slk::Load(&value, reader);
*gid = storage::v3::Gid::FromUint(value);
}
void Load(storage::v3::PropertyValue::Type *type, slk::Reader *reader) {
using PVTypeUnderlyingType = std::underlying_type_t<storage::v3::PropertyValue::Type>;
PVTypeUnderlyingType value{};
slk::Load(&value, reader);
bool valid{false};
switch (value) {
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::Null):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::Bool):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::Int):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::Double):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::String):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::List):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::Map):
case utils::UnderlyingCast(storage::v3::PropertyValue::Type::TemporalData):
valid = true;
break;
default:
valid = false;
break;
}
if (!valid) throw slk::SlkDecodeException("Trying to load unknown storage::v3::PropertyValue!");
*type = static_cast<storage::v3::PropertyValue::Type>(value);
}
void Save(const storage::v3::PropertyValue &value, slk::Builder *builder) {
switch (value.type()) {
case storage::v3::PropertyValue::Type::Null:
slk::Save(storage::v3::PropertyValue::Type::Null, builder);
return;
case storage::v3::PropertyValue::Type::Bool:
slk::Save(storage::v3::PropertyValue::Type::Bool, builder);
slk::Save(value.ValueBool(), builder);
return;
case storage::v3::PropertyValue::Type::Int:
slk::Save(storage::v3::PropertyValue::Type::Int, builder);
slk::Save(value.ValueInt(), builder);
return;
case storage::v3::PropertyValue::Type::Double:
slk::Save(storage::v3::PropertyValue::Type::Double, builder);
slk::Save(value.ValueDouble(), builder);
return;
case storage::v3::PropertyValue::Type::String:
slk::Save(storage::v3::PropertyValue::Type::String, builder);
slk::Save(value.ValueString(), builder);
return;
case storage::v3::PropertyValue::Type::List: {
slk::Save(storage::v3::PropertyValue::Type::List, builder);
const auto &values = value.ValueList();
size_t size = values.size();
slk::Save(size, builder);
for (const auto &v : values) {
slk::Save(v, builder);
}
return;
}
case storage::v3::PropertyValue::Type::Map: {
slk::Save(storage::v3::PropertyValue::Type::Map, builder);
const auto &map = value.ValueMap();
size_t size = map.size();
slk::Save(size, builder);
for (const auto &kv : map) {
slk::Save(kv, builder);
}
return;
}
case storage::v3::PropertyValue::Type::TemporalData: {
slk::Save(storage::v3::PropertyValue::Type::TemporalData, builder);
const auto temporal_data = value.ValueTemporalData();
slk::Save(temporal_data.type, builder);
slk::Save(temporal_data.microseconds, builder);
return;
}
}
}
void Load(storage::v3::PropertyValue *value, slk::Reader *reader) {
storage::v3::PropertyValue::Type type{};
slk::Load(&type, reader);
switch (type) {
case storage::v3::PropertyValue::Type::Null:
*value = storage::v3::PropertyValue();
return;
case storage::v3::PropertyValue::Type::Bool: {
bool v{false};
slk::Load(&v, reader);
*value = storage::v3::PropertyValue(v);
return;
}
case storage::v3::PropertyValue::Type::Int: {
int64_t v{0};
slk::Load(&v, reader);
*value = storage::v3::PropertyValue(v);
return;
}
case storage::v3::PropertyValue::Type::Double: {
double v{0.0};
slk::Load(&v, reader);
*value = storage::v3::PropertyValue(v);
return;
}
case storage::v3::PropertyValue::Type::String: {
std::string v;
slk::Load(&v, reader);
*value = storage::v3::PropertyValue(std::move(v));
return;
}
case storage::v3::PropertyValue::Type::List: {
size_t size{0};
slk::Load(&size, reader);
std::vector<storage::v3::PropertyValue> list(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&list[i], reader);
}
*value = storage::v3::PropertyValue(std::move(list));
return;
}
case storage::v3::PropertyValue::Type::Map: {
size_t size{0};
slk::Load(&size, reader);
std::map<std::string, storage::v3::PropertyValue> map;
for (size_t i = 0; i < size; ++i) {
std::pair<std::string, storage::v3::PropertyValue> kv;
slk::Load(&kv, reader);
map.insert(kv);
}
*value = storage::v3::PropertyValue(std::move(map));
return;
}
case storage::v3::PropertyValue::Type::TemporalData: {
storage::v3::TemporalType temporal_type{};
slk::Load(&temporal_type, reader);
int64_t microseconds{0};
slk::Load(&microseconds, reader);
*value = storage::v3::PropertyValue(storage::v3::TemporalData{temporal_type, microseconds});
return;
}
}
}
} // namespace memgraph::slk

View File

@ -0,0 +1,41 @@
// 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 "slk/serialization.hpp"
#include "storage/v3/durability/marker.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/property_value.hpp"
#include "utils/concepts.hpp"
namespace memgraph::slk {
void Save(const storage::v3::Gid &gid, slk::Builder *builder);
void Load(storage::v3::Gid *gid, slk::Reader *reader);
void Save(const storage::v3::PropertyValue &value, slk::Builder *builder);
void Load(storage::v3::PropertyValue *value, slk::Reader *reader);
template <utils::Enum T>
void Save(const T &enum_value, slk::Builder *builder) {
slk::Save(utils::UnderlyingCast(enum_value), builder);
}
template <utils::Enum T>
void Load(T *enum_value, slk::Reader *reader) {
using UnderlyingType = std::underlying_type_t<T>;
UnderlyingType value;
slk::Load(&value, reader);
*enum_value = static_cast<T>(value);
}
} // namespace memgraph::slk

33
src/storage/v3/result.hpp Normal file
View File

@ -0,0 +1,33 @@
// 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 <type_traits>
#include "utils/result.hpp"
namespace memgraph::storage::v3 {
static_assert(std::is_same_v<uint8_t, unsigned char>);
enum class Error : uint8_t {
SERIALIZATION_ERROR,
NONEXISTENT_OBJECT,
DELETED_OBJECT,
VERTEX_HAS_EDGES,
PROPERTIES_DISABLED,
};
template <class TValue>
using Result = utils::BasicResult<Error, TValue>;
} // namespace memgraph::storage::v3

1970
src/storage/v3/storage.cpp Normal file

File diff suppressed because it is too large Load Diff

596
src/storage/v3/storage.hpp Normal file
View File

@ -0,0 +1,596 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <filesystem>
#include <optional>
#include <shared_mutex>
#include <variant>
#include "io/network/endpoint.hpp"
#include "storage/v3/commit_log.hpp"
#include "storage/v3/config.hpp"
#include "storage/v3/constraints.hpp"
#include "storage/v3/durability/metadata.hpp"
#include "storage/v3/durability/wal.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/edge_accessor.hpp"
#include "storage/v3/indices.hpp"
#include "storage/v3/isolation_level.hpp"
#include "storage/v3/mvcc.hpp"
#include "storage/v3/name_id_mapper.hpp"
#include "storage/v3/result.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/vertex.hpp"
#include "storage/v3/vertex_accessor.hpp"
#include "utils/file_locker.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/rw_lock.hpp"
#include "utils/scheduler.hpp"
#include "utils/skip_list.hpp"
#include "utils/synchronized.hpp"
#include "utils/uuid.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v3/replication/config.hpp"
#include "storage/v3/replication/enums.hpp"
#include "storage/v3/replication/rpc.hpp"
#include "storage/v3/replication/serialization.hpp"
namespace memgraph::storage::v3 {
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
// The paper implements a fully serializable storage, in our implementation we
// only implement snapshot isolation for transactions.
/// Iterable for iterating through all vertices of a Storage.
///
/// An instance of this will be usually be wrapped inside VerticesIterable for
/// generic, public use.
class AllVerticesIterable final {
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()}; }
};
/// 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;
double average_degree;
uint64_t memory_usage;
uint64_t disk_usage;
};
enum class ReplicationRole : uint8_t { MAIN, REPLICA };
class Storage final {
public:
/// @throw std::system_error
/// @throw std::bad_alloc
explicit Storage(Config config = Config());
Storage(const Storage &) = delete;
Storage(Storage &&) = delete;
Storage &operator=(const Storage &) = delete;
Storage &operator=(Storage &&) = delete;
~Storage();
class Accessor final {
private:
friend class Storage;
explicit Accessor(Storage *storage, IsolationLevel isolation_level);
public:
Accessor(const Accessor &) = delete;
Accessor &operator=(const Accessor &) = delete;
Accessor &operator=(Accessor &&other) = delete;
// NOTE: After the accessor is moved, all objects derived from it (accessors
// and iterators) are *invalid*. You have to get all derived objects again.
Accessor(Accessor &&other) noexcept;
~Accessor();
/// @throw std::bad_alloc
VertexAccessor CreateVertex();
std::optional<VertexAccessor> FindVertex(Gid gid, View view);
VerticesIterable Vertices(View view) {
return VerticesIterable(AllVerticesIterable(storage_->vertices_.access(), &transaction_, view,
&storage_->indices_, &storage_->constraints_,
storage_->config_.items));
}
VerticesIterable Vertices(LabelId label, View view);
VerticesIterable Vertices(LabelId label, PropertyId property, View view);
VerticesIterable Vertices(LabelId label, PropertyId property, const PropertyValue &value, View view);
VerticesIterable Vertices(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower_bound,
const std::optional<utils::Bound<PropertyValue>> &upper_bound, View view);
/// Return approximate number of all vertices in the database.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount() const { return static_cast<int64_t>(storage_->vertices_.size()); }
/// Return approximate number of vertices with the given label.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label) const {
return storage_->indices_.label_index.ApproximateVertexCount(label);
}
/// Return approximate number of vertices with the given label and property.
/// Note that this is always an over-estimate and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property);
}
/// Return approximate number of vertices with the given label and the given
/// value for the given property. Note that this is always an over-estimate
/// and never an under-estimate.
int64_t ApproximateVertexCount(LabelId label, PropertyId property, const PropertyValue &value) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, value);
}
/// Return approximate number of vertices with the given label and value for
/// the given property in the range defined by provided upper and lower
/// bounds.
int64_t ApproximateVertexCount(LabelId label, PropertyId property,
const std::optional<utils::Bound<PropertyValue>> &lower,
const std::optional<utils::Bound<PropertyValue>> &upper) const {
return storage_->indices_.label_property_index.ApproximateVertexCount(label, property, lower, upper);
}
/// @return Accessor to the deleted vertex if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<VertexAccessor>> DeleteVertex(VertexAccessor *vertex);
/// @return Accessor to the deleted vertex and deleted edges if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<std::pair<VertexAccessor, std::vector<EdgeAccessor>>>> DetachDeleteVertex(
VertexAccessor *vertex);
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type);
/// Accessor to the deleted edge if a deletion took place, std::nullopt otherwise
/// @throw std::bad_alloc
Result<std::optional<EdgeAccessor>> DeleteEdge(EdgeAccessor *edge);
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(const std::string_view &name);
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(const std::string_view &name);
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(const 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();
/// Commit returns `ConstraintViolation` if the changes made by this
/// transaction violate an existence or unique constraint. In that case the
/// transaction is automatically aborted. Otherwise, void is returned.
/// @throw std::bad_alloc
utils::BasicResult<ConstraintViolation, void> Commit(std::optional<uint64_t> desired_commit_timestamp = {});
/// @throw std::bad_alloc
void Abort();
void FinalizeTransaction();
private:
/// @throw std::bad_alloc
VertexAccessor CreateVertex(Gid gid);
/// @throw std::bad_alloc
Result<EdgeAccessor> CreateEdge(VertexAccessor *from, VertexAccessor *to, EdgeTypeId edge_type, Gid gid);
Storage *storage_;
std::shared_lock<utils::RWLock> storage_guard_;
Transaction transaction_;
std::optional<uint64_t> commit_timestamp_;
bool is_transaction_active_;
Config::Items config_;
};
Accessor Access(std::optional<IsolationLevel> override_isolation_level = {}) {
return Accessor{this, override_isolation_level.value_or(isolation_level_)};
}
const std::string &LabelToName(LabelId label) const;
const std::string &PropertyToName(PropertyId property) const;
const std::string &EdgeTypeToName(EdgeTypeId edge_type) const;
/// @throw std::bad_alloc if unable to insert a new mapping
LabelId NameToLabel(const std::string_view &name);
/// @throw std::bad_alloc if unable to insert a new mapping
PropertyId NameToProperty(const std::string_view &name);
/// @throw std::bad_alloc if unable to insert a new mapping
EdgeTypeId NameToEdgeType(const std::string_view &name);
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
/// @throw std::bad_alloc
bool CreateIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
bool DropIndex(LabelId label, std::optional<uint64_t> desired_commit_timestamp = {});
bool DropIndex(LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
IndicesInfo ListAllIndices() const;
/// Creates an existence constraint. Returns true if the constraint was
/// successfuly added, false if it already exists and a `ConstraintViolation`
/// if there is an existing vertex violating the constraint.
///
/// @throw std::bad_alloc
/// @throw std::length_error
utils::BasicResult<ConstraintViolation, bool> CreateExistenceConstraint(
LabelId label, PropertyId property, std::optional<uint64_t> desired_commit_timestamp = {});
/// Removes an existence constraint. Returns true if the constraint was
/// removed, and false if it doesn't exist.
bool DropExistenceConstraint(LabelId label, PropertyId property,
std::optional<uint64_t> desired_commit_timestamp = {});
/// Creates a unique constraint. In the case of two vertices violating the
/// constraint, it returns `ConstraintViolation`. Otherwise returns a
/// `UniqueConstraints::CreationStatus` enum with the following possibilities:
/// * `SUCCESS` if the constraint was successfully created,
/// * `ALREADY_EXISTS` if the constraint already existed,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
// limit of maximum number of properties.
///
/// @throw std::bad_alloc
utils::BasicResult<ConstraintViolation, UniqueConstraints::CreationStatus> CreateUniqueConstraint(
LabelId label, const std::set<PropertyId> &properties, std::optional<uint64_t> desired_commit_timestamp = {});
/// Removes a unique constraint. Returns `UniqueConstraints::DeletionStatus`
/// enum with the following possibilities:
/// * `SUCCESS` if constraint was successfully removed,
/// * `NOT_FOUND` if the specified constraint was not found,
/// * `EMPTY_PROPERTIES` if the property set is empty, or
/// * `PROPERTIES_SIZE_LIMIT_EXCEEDED` if the property set exceeds the
// limit of maximum number of properties.
UniqueConstraints::DeletionStatus DropUniqueConstraint(LabelId label, const std::set<PropertyId> &properties,
std::optional<uint64_t> desired_commit_timestamp = {});
ConstraintsInfo ListAllConstraints() const;
StorageInfo GetInfo() const;
bool LockPath();
bool UnlockPath();
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config = {});
bool SetMainReplicationRole();
enum class RegisterReplicaError : uint8_t { NAME_EXISTS, END_POINT_EXISTS, CONNECTION_FAILED };
/// @pre The instance should have a MAIN role
/// @pre Timeout can only be set for SYNC replication
utils::BasicResult<RegisterReplicaError, void> RegisterReplica(
std::string name, io::network::Endpoint endpoint, replication::ReplicationMode replication_mode,
const replication::ReplicationClientConfig &config = {});
/// @pre The instance should have a MAIN role
bool UnregisterReplica(std::string_view name);
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
ReplicationRole GetReplicationRole() const;
struct ReplicaInfo {
std::string name;
replication::ReplicationMode mode;
std::optional<double> timeout;
io::network::Endpoint endpoint;
replication::ReplicaState state;
};
std::vector<ReplicaInfo> ReplicasInfo();
void FreeMemory();
void SetIsolationLevel(IsolationLevel isolation_level);
enum class CreateSnapshotError : uint8_t { DisabledForReplica };
utils::BasicResult<CreateSnapshotError> CreateSnapshot();
private:
Transaction CreateTransaction(IsolationLevel isolation_level);
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
/// be part of a transaction, and no other transaction can be active at the same time.
/// This allows it to delete immediately vertices without worrying that some other
/// transaction is possibly using it. If there are active transactions when this method
/// is called with force set to true, it will fallback to the same method with the force
/// set to false.
/// If it's set to false, it will execute in parallel with other transactions, ensuring
/// that no object in use can be deleted.
/// @throw std::system_error
/// @throw std::bad_alloc
template <bool force>
void CollectGarbage();
bool InitializeWalFile();
void FinalizeWalFile();
void AppendToWal(const Transaction &transaction, uint64_t final_commit_timestamp);
void AppendToWal(durability::StorageGlobalOperation operation, LabelId label, const std::set<PropertyId> &properties,
uint64_t final_commit_timestamp);
uint64_t CommitTimestamp(std::optional<uint64_t> desired_commit_timestamp = {});
// Main 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<Vertex> vertices_;
utils::SkipList<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_;
// 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_;
Config config_;
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_;
// Durability
std::filesystem::path snapshot_directory_;
std::filesystem::path wal_directory_;
std::filesystem::path lock_file_path_;
utils::OutputFile lock_file_handle_;
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<ReplicationRole> replication_role_{ReplicationRole::MAIN};
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,17 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/temporal.hpp"
namespace memgraph::storage::v3 {
TemporalData::TemporalData(TemporalType type, int64_t microseconds) : type{type}, microseconds{microseconds} {}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,56 @@
// 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 <cstdint>
#include <iostream>
#include <string_view>
#include "utils/temporal.hpp"
namespace memgraph::storage::v3 {
enum class TemporalType : uint8_t { Date = 0, LocalTime, LocalDateTime, Duration };
constexpr std::string_view TemporalTypeTostring(const TemporalType type) {
switch (type) {
case TemporalType::Date:
return "Date";
case TemporalType::LocalTime:
return "LocalTime";
case TemporalType::LocalDateTime:
return "LocalDateTime";
case TemporalType::Duration:
return "Duration";
}
}
struct TemporalData {
explicit TemporalData(TemporalType type, int64_t microseconds);
auto operator<=>(const TemporalData &) const = default;
friend std::ostream &operator<<(std::ostream &os, const TemporalData &t) {
switch (t.type) {
case TemporalType::Date:
return os << "DATE(\"" << utils::Date(t.microseconds) << "\")";
case TemporalType::LocalTime:
return os << "LOCALTIME(\"" << utils::LocalTime(t.microseconds) << "\")";
case TemporalType::LocalDateTime:
return os << "LOCALDATETIME(\"" << utils::LocalDateTime(t.microseconds) << "\")";
case TemporalType::Duration:
return os << "DURATION(\"" << utils::Duration(t.microseconds) << "\")";
}
}
TemporalType type;
int64_t microseconds;
};
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,84 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <atomic>
#include <limits>
#include <list>
#include <memory>
#include "utils/skip_list.hpp"
#include "storage/v3/delta.hpp"
#include "storage/v3/edge.hpp"
#include "storage/v3/isolation_level.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/vertex.hpp"
#include "storage/v3/view.hpp"
namespace memgraph::storage::v3 {
const uint64_t kTimestampInitialId = 0;
const uint64_t kTransactionInitialId = 1ULL << 63U;
struct Transaction {
Transaction(uint64_t transaction_id, uint64_t start_timestamp, IsolationLevel isolation_level)
: transaction_id(transaction_id),
start_timestamp(start_timestamp),
command_id(0),
must_abort(false),
isolation_level(isolation_level) {}
Transaction(Transaction &&other) noexcept
: transaction_id(other.transaction_id),
start_timestamp(other.start_timestamp),
commit_timestamp(std::move(other.commit_timestamp)),
command_id(other.command_id),
deltas(std::move(other.deltas)),
must_abort(other.must_abort),
isolation_level(other.isolation_level) {}
Transaction(const Transaction &) = delete;
Transaction &operator=(const Transaction &) = delete;
Transaction &operator=(Transaction &&other) = delete;
~Transaction() {}
/// @throw std::bad_alloc if failed to create the `commit_timestamp`
void EnsureCommitTimestampExists() {
if (commit_timestamp != nullptr) return;
commit_timestamp = std::make_unique<std::atomic<uint64_t>>(transaction_id);
}
uint64_t transaction_id;
uint64_t start_timestamp;
// The `Transaction` object is stack allocated, but the `commit_timestamp`
// must be heap allocated because `Delta`s have a pointer to it, and that
// pointer must stay valid after the `Transaction` is moved into
// `commited_transactions_` list for GC.
std::unique_ptr<std::atomic<uint64_t>> commit_timestamp;
uint64_t command_id;
std::list<Delta> deltas;
bool must_abort;
IsolationLevel isolation_level;
};
inline bool operator==(const Transaction &first, const Transaction &second) {
return first.transaction_id == second.transaction_id;
}
inline bool operator<(const Transaction &first, const Transaction &second) {
return first.transaction_id < second.transaction_id;
}
inline bool operator==(const Transaction &first, const uint64_t &second) { return first.transaction_id == second; }
inline bool operator<(const Transaction &first, const uint64_t &second) { return first.transaction_id < second; }
} // namespace memgraph::storage::v3

55
src/storage/v3/vertex.hpp Normal file
View File

@ -0,0 +1,55 @@
// 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 <limits>
#include <tuple>
#include <vector>
#include "storage/v3/delta.hpp"
#include "storage/v3/edge_ref.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/property_store.hpp"
#include "utils/spin_lock.hpp"
namespace memgraph::storage::v3 {
struct Vertex {
Vertex(Gid gid, Delta *delta) : gid(gid), deleted(false), delta(delta) {
MG_ASSERT(delta == nullptr || delta->action == Delta::Action::DELETE_OBJECT,
"Vertex must be created with an initial DELETE_OBJECT delta!");
}
Gid gid;
std::vector<LabelId> labels;
PropertyStore properties;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
mutable utils::SpinLock lock;
bool deleted;
// uint8_t PAD;
// uint16_t PAD;
Delta *delta;
};
static_assert(alignof(Vertex) >= 8, "The Vertex should be aligned to at least 8!");
inline bool operator==(const Vertex &first, const Vertex &second) { return first.gid == second.gid; }
inline bool operator<(const Vertex &first, const Vertex &second) { return first.gid < second.gid; }
inline bool operator==(const Vertex &first, const Gid &second) { return first.gid == second; }
inline bool operator<(const Vertex &first, const Gid &second) { return first.gid < second; }
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,578 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v3/vertex_accessor.hpp"
#include <memory>
#include "storage/v3/edge_accessor.hpp"
#include "storage/v3/id_types.hpp"
#include "storage/v3/indices.hpp"
#include "storage/v3/mvcc.hpp"
#include "storage/v3/property_value.hpp"
#include "utils/logging.hpp"
#include "utils/memory_tracker.hpp"
namespace memgraph::storage::v3 {
namespace detail {
namespace {
std::pair<bool, bool> IsVisible(Vertex *vertex, Transaction *transaction, View view) {
bool exists = true;
bool deleted = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex->lock);
deleted = vertex->deleted;
delta = vertex->delta;
}
ApplyDeltasForRead(transaction, delta, view, [&](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
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;
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
}
});
return {exists, deleted};
}
} // namespace
} // namespace detail
std::optional<VertexAccessor> VertexAccessor::Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view) {
if (const auto [exists, deleted] = detail::IsVisible(vertex, transaction, view); !exists || deleted) {
return std::nullopt;
}
return VertexAccessor{vertex, transaction, indices, constraints, config};
}
bool VertexAccessor::IsVisible(View view) const {
const auto [exists, deleted] = detail::IsVisible(vertex_, transaction_, view);
return exists && (for_deleted_ || !deleted);
}
Result<bool> VertexAccessor::AddLabel(LabelId label) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
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_);
return true;
}
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();
return true;
}
Result<bool> VertexAccessor::HasLabel(LabelId label, View view) const {
bool exists = true;
bool deleted = false;
bool has_label = false;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
has_label = std::find(vertex_->labels.begin(), vertex_->labels.end(), label) != vertex_->labels.end();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &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: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
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;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return has_label;
}
Result<std::vector<LabelId>> VertexAccessor::Labels(View view) const {
bool exists = true;
bool deleted = false;
std::vector<LabelId> labels;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
labels = vertex_->labels;
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &labels](const Delta &delta) {
switch (delta.action) {
case Delta::Action::REMOVE_LABEL: {
// Remove the label because we don't see the addition.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it != labels.end(), "Invalid database state!");
std::swap(*it, *labels.rbegin());
labels.pop_back();
break;
}
case Delta::Action::ADD_LABEL: {
// Add the label because we don't see the removal.
auto it = std::find(labels.begin(), labels.end(), delta.label);
MG_ASSERT(it == labels.end(), "Invalid database state!");
labels.push_back(delta.label);
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
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;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(labels);
}
Result<PropertyValue> VertexAccessor::SetProperty(PropertyId property, const PropertyValue &value) {
utils::MemoryTracker::OutOfMemoryExceptionEnabler oom_exception;
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
auto current_value = vertex_->properties.GetProperty(property);
// We could skip setting the value if the previous one is the same to the new
// one. This would save some memory as a delta would not be created as well as
// avoid copying the value. The reason we are not doing that is because the
// current code always follows the logical pattern of "create a delta" and
// "modify in-place". Additionally, the created delta will make other
// transactions get a SERIALIZATION_ERROR.
CreateAndLinkDelta(transaction_, vertex_, Delta::SetPropertyTag(), property, current_value);
vertex_->properties.SetProperty(property, value);
UpdateOnSetProperty(indices_, property, value, vertex_, *transaction_);
return std::move(current_value);
}
Result<std::map<PropertyId, PropertyValue>> VertexAccessor::ClearProperties() {
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
if (!PrepareForWrite(transaction_, vertex_)) return Error::SERIALIZATION_ERROR;
if (vertex_->deleted) return Error::DELETED_OBJECT;
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_);
}
vertex_->properties.ClearProperties();
return std::move(properties);
}
Result<PropertyValue> VertexAccessor::GetProperty(PropertyId property, View view) const {
bool exists = true;
bool deleted = false;
PropertyValue value;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
value = vertex_->properties.GetProperty(property);
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &value, property](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
if (delta.property.key == property) {
value = delta.property.value;
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(value);
}
Result<std::map<PropertyId, PropertyValue>> VertexAccessor::Properties(View view) const {
bool exists = true;
bool deleted = false;
std::map<PropertyId, PropertyValue> properties;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
properties = vertex_->properties.Properties();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &properties](const Delta &delta) {
switch (delta.action) {
case Delta::Action::SET_PROPERTY: {
auto it = properties.find(delta.property.key);
if (it != properties.end()) {
if (delta.property.value.IsNull()) {
// remove the property
properties.erase(it);
} else {
// set the value
it->second = delta.property.value;
}
} else if (!delta.property.value.IsNull()) {
properties.emplace(delta.property.key, delta.property.value);
}
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return std::move(properties);
}
Result<std::vector<EdgeAccessor>> VertexAccessor::InEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const {
MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> in_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destination) {
in_edges = vertex_->in_edges;
} else {
for (const auto &item : vertex_->in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
if (destination && from_vertex != destination->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
in_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &in_edges, &edge_types, &destination](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
if (!edge_types.empty() &&
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
break;
// Add the edge because we don't see the removal.
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
delta.vertex_edge.edge};
auto it = std::find(in_edges.begin(), in_edges.end(), link);
MG_ASSERT(it == in_edges.end(), "Invalid database state!");
in_edges.push_back(link);
break;
}
case Delta::Action::REMOVE_IN_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
if (!edge_types.empty() &&
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
break;
// Remove the label because we don't see the addition.
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
delta.vertex_edge.edge};
auto it = std::find(in_edges.begin(), in_edges.end(), link);
MG_ASSERT(it != in_edges.end(), "Invalid database state!");
std::swap(*it, *in_edges.rbegin());
in_edges.pop_back();
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<EdgeAccessor> ret;
ret.reserve(in_edges.size());
for (const auto &item : in_edges) {
const auto &[edge_type, from_vertex, edge] = item;
ret.emplace_back(edge, edge_type, from_vertex, vertex_, transaction_, indices_, constraints_, config_);
}
return std::move(ret);
}
Result<std::vector<EdgeAccessor>> VertexAccessor::OutEdges(View view, const std::vector<EdgeTypeId> &edge_types,
const VertexAccessor *destination) const {
MG_ASSERT(!destination || destination->transaction_ == transaction_, "Invalid accessor!");
bool exists = true;
bool deleted = false;
std::vector<std::tuple<EdgeTypeId, Vertex *, EdgeRef>> out_edges;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
if (edge_types.empty() && !destination) {
out_edges = vertex_->out_edges;
} else {
for (const auto &item : vertex_->out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
if (destination && to_vertex != destination->vertex_) continue;
if (!edge_types.empty() && std::find(edge_types.begin(), edge_types.end(), edge_type) == edge_types.end())
continue;
out_edges.push_back(item);
}
}
delta = vertex_->delta;
}
ApplyDeltasForRead(
transaction_, delta, view, [&exists, &deleted, &out_edges, &edge_types, &destination](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
if (!edge_types.empty() &&
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
break;
// Add the edge because we don't see the removal.
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
delta.vertex_edge.edge};
auto it = std::find(out_edges.begin(), out_edges.end(), link);
MG_ASSERT(it == out_edges.end(), "Invalid database state!");
out_edges.push_back(link);
break;
}
case Delta::Action::REMOVE_OUT_EDGE: {
if (destination && delta.vertex_edge.vertex != destination->vertex_) break;
if (!edge_types.empty() &&
std::find(edge_types.begin(), edge_types.end(), delta.vertex_edge.edge_type) == edge_types.end())
break;
// Remove the label because we don't see the addition.
std::tuple<EdgeTypeId, Vertex *, EdgeRef> link{delta.vertex_edge.edge_type, delta.vertex_edge.vertex,
delta.vertex_edge.edge};
auto it = std::find(out_edges.begin(), out_edges.end(), link);
MG_ASSERT(it != out_edges.end(), "Invalid database state!");
std::swap(*it, *out_edges.rbegin());
out_edges.pop_back();
break;
}
case Delta::Action::DELETE_OBJECT: {
exists = false;
break;
}
case Delta::Action::RECREATE_OBJECT: {
deleted = false;
break;
}
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (deleted) return Error::DELETED_OBJECT;
std::vector<EdgeAccessor> ret;
ret.reserve(out_edges.size());
for (const auto &item : out_edges) {
const auto &[edge_type, to_vertex, edge] = item;
ret.emplace_back(edge, edge_type, vertex_, to_vertex, transaction_, indices_, constraints_, config_);
}
return std::move(ret);
}
Result<size_t> VertexAccessor::InDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->in_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_IN_EDGE:
++degree;
break;
case Delta::Action::REMOVE_IN_EDGE:
--degree;
break;
case Delta::Action::DELETE_OBJECT:
exists = false;
break;
case Delta::Action::RECREATE_OBJECT:
deleted = false;
break;
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
Result<size_t> VertexAccessor::OutDegree(View view) const {
bool exists = true;
bool deleted = false;
size_t degree = 0;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(vertex_->lock);
deleted = vertex_->deleted;
degree = vertex_->out_edges.size();
delta = vertex_->delta;
}
ApplyDeltasForRead(transaction_, delta, view, [&exists, &deleted, &degree](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_OUT_EDGE:
++degree;
break;
case Delta::Action::REMOVE_OUT_EDGE:
--degree;
break;
case Delta::Action::DELETE_OBJECT:
exists = false;
break;
case Delta::Action::RECREATE_OBJECT:
deleted = false;
break;
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
break;
}
});
if (!exists) return Error::NONEXISTENT_OBJECT;
if (!for_deleted_ && deleted) return Error::DELETED_OBJECT;
return degree;
}
} // namespace memgraph::storage::v3

View File

@ -0,0 +1,128 @@
// 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 "storage/v3/vertex.hpp"
#include "storage/v3/config.hpp"
#include "storage/v3/result.hpp"
#include "storage/v3/transaction.hpp"
#include "storage/v3/view.hpp"
namespace memgraph::storage::v3 {
class EdgeAccessor;
class Storage;
struct Indices;
struct Constraints;
class VertexAccessor final {
private:
friend class Storage;
public:
VertexAccessor(Vertex *vertex, Transaction *transaction, Indices *indices, Constraints *constraints,
Config::Items config, bool for_deleted = false)
: vertex_(vertex),
transaction_(transaction),
indices_(indices),
constraints_(constraints),
config_(config),
for_deleted_(for_deleted) {}
static std::optional<VertexAccessor> Create(Vertex *vertex, Transaction *transaction, Indices *indices,
Constraints *constraints, Config::Items config, View view);
/// @return true if the object is visible from the current transaction
bool IsVisible(View view) const;
/// Add a label and return `true` if insertion took place.
/// `false` is returned if the label already existed.
/// @throw std::bad_alloc
Result<bool> AddLabel(LabelId label);
/// Remove a label and return `true` if deletion took place.
/// `false` is returned if the vertex did not have a label already.
/// @throw std::bad_alloc
Result<bool> RemoveLabel(LabelId label);
Result<bool> HasLabel(LabelId label, View view) const;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<LabelId>> Labels(View view) const;
/// Set a property value and return the old value.
/// @throw std::bad_alloc
Result<PropertyValue> SetProperty(PropertyId property, const PropertyValue &value);
/// Remove all properties and return the values of the removed properties.
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> ClearProperties();
/// @throw std::bad_alloc
Result<PropertyValue> GetProperty(PropertyId property, View view) const;
/// @throw std::bad_alloc
Result<std::map<PropertyId, PropertyValue>> Properties(View view) const;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<EdgeAccessor>> InEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
const VertexAccessor *destination = nullptr) const;
/// @throw std::bad_alloc
/// @throw std::length_error if the resulting vector exceeds
/// std::vector::max_size().
Result<std::vector<EdgeAccessor>> OutEdges(View view, const std::vector<EdgeTypeId> &edge_types = {},
const VertexAccessor *destination = nullptr) const;
Result<size_t> InDegree(View view) const;
Result<size_t> OutDegree(View view) const;
Gid Gid() const noexcept { return vertex_->gid; }
bool operator==(const VertexAccessor &other) const noexcept {
return vertex_ == other.vertex_ && transaction_ == other.transaction_;
}
bool operator!=(const VertexAccessor &other) const noexcept { return !(*this == other); }
private:
Vertex *vertex_;
Transaction *transaction_;
Indices *indices_;
Constraints *constraints_;
Config::Items config_;
// if the accessor was created for a deleted vertex.
// Accessor behaves differently for some methods based on this
// flag.
// E.g. If this field is set to true, GetProperty will return the property of the node
// even though the node is deleted.
// All the write operations, and operators used for traversal (e.g. InEdges) will still
// return an error if it's called for a deleted vertex.
bool for_deleted_{false};
};
} // namespace memgraph::storage::v3
namespace std {
template <>
struct hash<memgraph::storage::v3::VertexAccessor> {
size_t operator()(const memgraph::storage::v3::VertexAccessor &v) const noexcept { return v.Gid().AsUint(); }
};
} // namespace std

22
src/storage/v3/view.hpp Normal file
View File

@ -0,0 +1,22 @@
// 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
namespace memgraph::storage::v3 {
/// Indicator for obtaining the state before or after a transaction & command.
enum class View {
OLD,
NEW,
};
} // namespace memgraph::storage::v3

View File

@ -22,44 +22,43 @@ function(_add_unit_test test_cpp custom_main)
set(target_name ${test_prefix}${exec_name})
set(source_files
${test_cpp}
${ARGN})
${test_cpp}
${ARGN})
if(NOT ${custom_main})
set(source_files
${source_files}
${memgraph_unit_main})
${source_files}
${memgraph_unit_main})
endif()
add_executable(${target_name} ${source_files})
# OUTPUT_NAME sets the real name of a target when it is built and can be
# used to help create two targets of the same name even though CMake
# requires unique logical target names
set_target_properties(${target_name} PROPERTIES OUTPUT_NAME ${exec_name})
target_link_libraries(${target_name} mg-memory mg-utils gtest gmock Threads::Threads dl)
# register test
if(TEST_COVERAGE)
add_test(${target_name} env LLVM_PROFILE_FILE=${exec_name}.profraw ./${exec_name})
else()
add_test(${target_name} ${exec_name})
endif()
# add to memgraph__unit target
add_dependencies(memgraph__unit ${target_name})
endfunction(_add_unit_test)
# Test utilities
add_library(storage_test_utils storage_test_utils.cpp)
target_link_libraries(storage_test_utils mg-storage-v2)
# Test integrations-kafka
add_library(kafka-mock STATIC kafka_mock.cpp)
target_link_libraries(kafka-mock mg-utils librdkafka++ librdkafka Threads::Threads gtest)
# Include directories are intentionally not set, because kafka-mock isn't meant to be used apart from unit tests
# Include directories are intentionally not set, because kafka-mock isn't meant to be used apart from unit tests
add_unit_test(integrations_kafka_consumer.cpp kafka_mock.cpp)
target_link_libraries(${test_prefix}integrations_kafka_consumer kafka-mock mg-integrations-kafka)
@ -70,7 +69,6 @@ add_unit_test(mgp_trans_c_api.cpp)
target_link_libraries(${test_prefix}mgp_trans_c_api mg-query)
# Test mg-query
add_unit_test(bfs_single_node.cpp)
target_link_libraries(${test_prefix}bfs_single_node mg-query)
@ -134,7 +132,6 @@ add_unit_test(query_function_mgp_module.cpp)
target_link_libraries(${test_prefix}query_function_mgp_module mg-query)
target_include_directories(${test_prefix}query_function_mgp_module PRIVATE ${CMAKE_SOURCE_DIR}/include)
# Test query/procedure
add_unit_test(query_procedure_mgp_type.cpp)
target_link_libraries(${test_prefix}query_procedure_mgp_type mg-query)
@ -151,8 +148,8 @@ target_include_directories(${test_prefix}query_procedure_py_module PRIVATE ${CMA
add_unit_test(query_procedures_mgp_graph.cpp)
target_link_libraries(${test_prefix}query_procedures_mgp_graph mg-query storage_test_utils)
target_include_directories(${test_prefix}query_procedures_mgp_graph PRIVATE ${CMAKE_SOURCE_DIR}/include)
# END query/procedure
# END query/procedure
add_unit_test(query_profile.cpp)
target_link_libraries(${test_prefix}query_profile mg-query)
@ -171,9 +168,7 @@ target_link_libraries(${test_prefix}stripped mg-query)
add_unit_test(typed_value.cpp)
target_link_libraries(${test_prefix}typed_value mg-query)
# Test mg-communication
add_unit_test(bolt_chunked_decoder_buffer.cpp)
target_link_libraries(${test_prefix}bolt_chunked_decoder_buffer mg-communication)
@ -195,21 +190,15 @@ target_link_libraries(${test_prefix}communication_buffer mg-communication mg-uti
add_unit_test(network_timeouts.cpp)
target_link_libraries(${test_prefix}network_timeouts mg-communication)
# Test mg-kvstore
add_unit_test(kvstore.cpp)
target_link_libraries(${test_prefix}kvstore mg-kvstore mg-utils)
# Test data structures
add_unit_test(ring_buffer.cpp)
target_link_libraries(${test_prefix}ring_buffer mg-utils)
# Test mg-io
add_unit_test(network_endpoint.cpp)
target_link_libraries(${test_prefix}network_endpoint mg-io)
@ -219,9 +208,7 @@ target_link_libraries(${test_prefix}network_utils mg-io)
add_unit_test(socket.cpp)
target_link_libraries(${test_prefix}socket mg-io)
# Test mg-utils
add_unit_test(utils_algorithm.cpp)
target_link_libraries(${test_prefix}utils_algorithm mg-utils)
@ -289,7 +276,6 @@ add_unit_test(utils_temporal utils_temporal.cpp)
target_link_libraries(${test_prefix}utils_temporal mg-utils)
# Test mg-storage-v2
add_unit_test(commit_log_v2.cpp)
target_link_libraries(${test_prefix}commit_log_v2 gflags mg-utils mg-storage-v2)
@ -333,39 +319,32 @@ add_unit_test(storage_v2_isolation_level.cpp)
target_link_libraries(${test_prefix}storage_v2_isolation_level mg-storage-v2)
# Test mg-auth
if (MG_ENTERPRISE)
add_unit_test(auth.cpp)
target_link_libraries(${test_prefix}auth mg-auth mg-license)
if(MG_ENTERPRISE)
add_unit_test(auth.cpp)
target_link_libraries(${test_prefix}auth mg-auth mg-license)
endif()
# Test mg-slk
if (MG_ENTERPRISE)
add_unit_test(slk_advanced.cpp)
target_link_libraries(${test_prefix}slk_advanced mg-storage-v2)
if(MG_ENTERPRISE)
add_unit_test(slk_advanced.cpp)
target_link_libraries(${test_prefix}slk_advanced mg-storage-v2)
endif()
if (MG_ENTERPRISE)
add_unit_test(slk_core.cpp)
target_link_libraries(${test_prefix}slk_core mg-slk gflags fmt)
if(MG_ENTERPRISE)
add_unit_test(slk_core.cpp)
target_link_libraries(${test_prefix}slk_core mg-slk gflags fmt)
add_unit_test(slk_streams.cpp)
target_link_libraries(${test_prefix}slk_streams mg-slk gflags fmt)
add_unit_test(slk_streams.cpp)
target_link_libraries(${test_prefix}slk_streams mg-slk gflags fmt)
endif()
# Test mg-rpc
if (MG_ENTERPRISE)
add_unit_test(rpc.cpp)
target_link_libraries(${test_prefix}rpc mg-rpc)
if(MG_ENTERPRISE)
add_unit_test(rpc.cpp)
target_link_libraries(${test_prefix}rpc mg-rpc)
endif()
# Test LCP
add_custom_command(
OUTPUT test_lcp
DEPENDS ${lcp_src_files} lcp test_lcp.lisp
@ -379,3 +358,7 @@ find_package(Boost REQUIRED)
add_unit_test(websocket.cpp)
target_link_libraries(${test_prefix}websocket mg-communication Boost::headers)
# Test storage-v3
add_unit_test(storage_v3.cpp)
target_link_libraries(${test_prefix}storage_v3 mg-storage-v3)

21
tests/unit/storage_v3.cpp Normal file
View File

@ -0,0 +1,21 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include "storage/v3/storage.hpp"
// NOLINTNEXTLINE(hicpp-special-member-functions)
TEST(StorageV3, DummyTest) {
memgraph::storage::v3::Storage store;
EXPECT_EQ(store.GetInfo().vertex_count, 0);
}