Merge branch 'project-pineapples' into T1083-MG-limit-and-order-expand-one_v3
This commit is contained in:
commit
d7ce7cea13
9
.github/workflows/diff.yaml
vendored
9
.github/workflows/diff.yaml
vendored
@ -219,3 +219,12 @@ jobs:
|
||||
# Run simulation tests.
|
||||
cd build
|
||||
ctest -R memgraph__simulation --output-on-failure -j$THREADS
|
||||
|
||||
- name: Run e2e tests
|
||||
run: |
|
||||
# TODO(gitbuda): Setup mgclient and pymgclient properly.
|
||||
cd tests
|
||||
./setup.sh
|
||||
source ve3/bin/activate
|
||||
cd e2e
|
||||
LD_LIBRARY_PATH=$LD_LIBRARY_PATH:../../libs/mgclient/lib python runner.py --workloads-root-directory ./distributed_queries
|
||||
|
@ -116,7 +116,7 @@ declare -A primary_urls=(
|
||||
["pymgclient"]="http://$local_cache_host/git/pymgclient.git"
|
||||
["mgconsole"]="http://$local_cache_host/git/mgconsole.git"
|
||||
["spdlog"]="http://$local_cache_host/git/spdlog"
|
||||
["nlohmann"]="http://$local_cache_host/file/nlohmann/json/4f8fba14066156b73f1189a2b8bd568bde5284c5/single_include/nlohmann/json.hpp"
|
||||
["nlohmann"]="http://$local_cache_host/file/nlohmann/json/9d69186291aca4f0137b69c1dee313b391ff564c/single_include/nlohmann/json.hpp"
|
||||
["neo4j"]="http://$local_cache_host/file/neo4j-community-3.2.3-unix.tar.gz"
|
||||
["librdkafka"]="http://$local_cache_host/git/librdkafka.git"
|
||||
["protobuf"]="http://$local_cache_host/git/protobuf.git"
|
||||
@ -141,7 +141,7 @@ declare -A secondary_urls=(
|
||||
["pymgclient"]="https://github.com/memgraph/pymgclient.git"
|
||||
["mgconsole"]="http://github.com/memgraph/mgconsole.git"
|
||||
["spdlog"]="https://github.com/gabime/spdlog"
|
||||
["nlohmann"]="https://raw.githubusercontent.com/nlohmann/json/4f8fba14066156b73f1189a2b8bd568bde5284c5/single_include/nlohmann/json.hpp"
|
||||
["nlohmann"]="https://raw.githubusercontent.com/nlohmann/json/9d69186291aca4f0137b69c1dee313b391ff564c/single_include/nlohmann/json.hpp"
|
||||
["neo4j"]="https://s3-eu-west-1.amazonaws.com/deps.memgraph.io/neo4j-community-3.2.3-unix.tar.gz"
|
||||
["librdkafka"]="https://github.com/edenhill/librdkafka.git"
|
||||
["protobuf"]="https://github.com/protocolbuffers/protobuf.git"
|
||||
|
156
src/coordinator/coordinator_worker.hpp
Normal file
156
src/coordinator/coordinator_worker.hpp
Normal file
@ -0,0 +1,156 @@
|
||||
// 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 <deque>
|
||||
#include <memory>
|
||||
#include <queue>
|
||||
#include <variant>
|
||||
|
||||
#include "coordinator/coordinator.hpp"
|
||||
#include "coordinator/coordinator_rsm.hpp"
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "io/address.hpp"
|
||||
#include "io/future.hpp"
|
||||
#include "io/messages.hpp"
|
||||
#include "io/rsm/raft.hpp"
|
||||
#include "io/time.hpp"
|
||||
#include "io/transport.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
|
||||
namespace memgraph::coordinator::coordinator_worker {
|
||||
|
||||
/// Obligations:
|
||||
/// * ShutDown
|
||||
/// * Cron
|
||||
/// * RouteMessage
|
||||
|
||||
using coordinator::Coordinator;
|
||||
using coordinator::CoordinatorRsm;
|
||||
using io::Address;
|
||||
using io::RequestId;
|
||||
using io::Time;
|
||||
using io::messages::CoordinatorMessages;
|
||||
using msgs::ReadRequests;
|
||||
using msgs::ReadResponses;
|
||||
using msgs::WriteRequests;
|
||||
using msgs::WriteResponses;
|
||||
|
||||
struct ShutDown {};
|
||||
|
||||
struct Cron {};
|
||||
|
||||
struct RouteMessage {
|
||||
CoordinatorMessages message;
|
||||
RequestId request_id;
|
||||
Address to;
|
||||
Address from;
|
||||
};
|
||||
|
||||
using Message = std::variant<RouteMessage, Cron, ShutDown>;
|
||||
|
||||
struct QueueInner {
|
||||
std::mutex mu{};
|
||||
std::condition_variable cv;
|
||||
// TODO(tyler) handle simulator communication std::shared_ptr<std::atomic<int>> blocked;
|
||||
|
||||
// TODO(tyler) investigate using a priority queue that prioritizes messages in a way that
|
||||
// improves overall QoS. For example, maybe we want to schedule raft Append messages
|
||||
// ahead of Read messages or generally writes before reads for lowering the load on the
|
||||
// overall system faster etc... When we do this, we need to make sure to avoid
|
||||
// starvation by sometimes randomizing priorities, rather than following a strict
|
||||
// prioritization.
|
||||
std::deque<Message> queue;
|
||||
};
|
||||
|
||||
/// There are two reasons to implement our own Queue instead of using
|
||||
/// one off-the-shelf:
|
||||
/// 1. we will need to know in the simulator when all threads are waiting
|
||||
/// 2. we will want to implement our own priority queue within this for QoS
|
||||
class Queue {
|
||||
std::shared_ptr<QueueInner> inner_ = std::make_shared<QueueInner>();
|
||||
|
||||
public:
|
||||
void Push(Message &&message) {
|
||||
{
|
||||
MG_ASSERT(inner_.use_count() > 0);
|
||||
std::unique_lock<std::mutex> lock(inner_->mu);
|
||||
|
||||
inner_->queue.emplace_back(std::move(message));
|
||||
} // lock dropped before notifying condition variable
|
||||
|
||||
inner_->cv.notify_all();
|
||||
}
|
||||
|
||||
Message Pop() {
|
||||
MG_ASSERT(inner_.use_count() > 0);
|
||||
std::unique_lock<std::mutex> lock(inner_->mu);
|
||||
|
||||
while (inner_->queue.empty()) {
|
||||
inner_->cv.wait(lock);
|
||||
}
|
||||
|
||||
Message message = std::move(inner_->queue.front());
|
||||
inner_->queue.pop_front();
|
||||
|
||||
return message;
|
||||
}
|
||||
};
|
||||
|
||||
/// A CoordinatorWorker owns Raft<CoordinatorRsm> instances. receives messages from the MachineManager.
|
||||
template <typename IoImpl>
|
||||
class CoordinatorWorker {
|
||||
io::Io<IoImpl> io_;
|
||||
Queue queue_;
|
||||
CoordinatorRsm<IoImpl> coordinator_;
|
||||
|
||||
bool Process(ShutDown && /*shut_down*/) { return false; }
|
||||
|
||||
bool Process(Cron && /* cron */) {
|
||||
coordinator_.Cron();
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Process(RouteMessage &&route_message) {
|
||||
coordinator_.Handle(std::move(route_message.message), route_message.request_id, route_message.from);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
public:
|
||||
CoordinatorWorker(io::Io<IoImpl> io, Queue queue, Coordinator coordinator)
|
||||
: io_(std::move(io)),
|
||||
queue_(std::move(queue)),
|
||||
coordinator_{std::move(io_.ForkLocal()), {}, std::move(coordinator)} {}
|
||||
|
||||
CoordinatorWorker(CoordinatorWorker &&) noexcept = default;
|
||||
CoordinatorWorker &operator=(CoordinatorWorker &&) noexcept = default;
|
||||
CoordinatorWorker(const CoordinatorWorker &) = delete;
|
||||
CoordinatorWorker &operator=(const CoordinatorWorker &) = delete;
|
||||
~CoordinatorWorker() = default;
|
||||
|
||||
void Run() {
|
||||
while (true) {
|
||||
Message message = queue_.Pop();
|
||||
|
||||
const bool should_continue = std::visit(
|
||||
[this](auto &&msg) { return this->Process(std::forward<decltype(msg)>(msg)); }, std::move(message));
|
||||
|
||||
if (!should_continue) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace memgraph::coordinator::coordinator_worker
|
@ -9,6 +9,7 @@
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <optional>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
@ -227,6 +228,24 @@ Hlc ShardMap::IncrementShardMapVersion() noexcept {
|
||||
return shard_map_version;
|
||||
}
|
||||
|
||||
// TODO(antaljanosbenjamin) use a single map for all name id
|
||||
// mapping and a single counter to maintain the next id
|
||||
std::unordered_map<uint64_t, std::string> ShardMap::IdToNames() {
|
||||
std::unordered_map<uint64_t, std::string> id_to_names;
|
||||
|
||||
const auto map_type_ids = [&id_to_names](const auto &name_to_id_type) {
|
||||
for (const auto &[name, id] : name_to_id_type) {
|
||||
id_to_names.emplace(id.AsUint(), name);
|
||||
}
|
||||
};
|
||||
|
||||
map_type_ids(edge_types);
|
||||
map_type_ids(labels);
|
||||
map_type_ids(properties);
|
||||
|
||||
return id_to_names;
|
||||
}
|
||||
|
||||
Hlc ShardMap::GetHlc() const noexcept { return shard_map_version; }
|
||||
|
||||
std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
|
||||
@ -258,6 +277,7 @@ std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
|
||||
if (same_machine) {
|
||||
machine_contains_shard = true;
|
||||
spdlog::info("reminding shard manager that they should begin participating in shard");
|
||||
|
||||
ret.push_back(ShardToInitialize{
|
||||
.uuid = aas.address.unique_id,
|
||||
.label_id = label_id,
|
||||
@ -265,6 +285,7 @@ std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
|
||||
.max_key = high_key,
|
||||
.schema = schemas[label_id],
|
||||
.config = Config{},
|
||||
.id_to_names = IdToNames(),
|
||||
});
|
||||
}
|
||||
}
|
||||
@ -285,6 +306,7 @@ std::vector<ShardToInitialize> ShardMap::AssignShards(Address storage_manager,
|
||||
.max_key = high_key,
|
||||
.schema = schemas[label_id],
|
||||
.config = Config{},
|
||||
.id_to_names = IdToNames(),
|
||||
});
|
||||
|
||||
AddressAndStatus aas = {
|
||||
@ -365,7 +387,6 @@ std::optional<LabelId> ShardMap::GetLabelId(const std::string &label) const {
|
||||
if (const auto it = labels.find(label); it != labels.end()) {
|
||||
return it->second;
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
@ -382,7 +403,6 @@ std::optional<PropertyId> ShardMap::GetPropertyId(const std::string &property_na
|
||||
if (const auto it = properties.find(property_name); it != properties.end()) {
|
||||
return it->second;
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
@ -399,7 +419,6 @@ std::optional<EdgeTypeId> ShardMap::GetEdgeTypeId(const std::string &edge_type)
|
||||
if (const auto it = edge_types.find(edge_type); it != edge_types.end()) {
|
||||
return it->second;
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
@ -411,6 +430,7 @@ const std::string &ShardMap::GetEdgeTypeName(const EdgeTypeId property) const {
|
||||
}
|
||||
throw utils::BasicException("EdgeTypeId not found!");
|
||||
}
|
||||
|
||||
Shards ShardMap::GetShardsForRange(const LabelName &label_name, const PrimaryKey &start_key,
|
||||
const PrimaryKey &end_key) const {
|
||||
MG_ASSERT(start_key <= end_key);
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include "io/address.hpp"
|
||||
#include "storage/v3/config.hpp"
|
||||
#include "storage/v3/id_types.hpp"
|
||||
#include "storage/v3/name_id_mapper.hpp"
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/schemas.hpp"
|
||||
#include "storage/v3/temporal.hpp"
|
||||
@ -90,6 +91,7 @@ struct ShardToInitialize {
|
||||
std::optional<PrimaryKey> max_key;
|
||||
std::vector<SchemaProperty> schema;
|
||||
Config config;
|
||||
std::unordered_map<uint64_t, std::string> id_to_names;
|
||||
};
|
||||
|
||||
PrimaryKey SchemaToMinKey(const std::vector<SchemaProperty> &schema);
|
||||
@ -120,9 +122,9 @@ struct ShardMap {
|
||||
std::map<PropertyName, PropertyId> properties;
|
||||
std::map<EdgeTypeName, EdgeTypeId> edge_types;
|
||||
uint64_t max_label_id{kNotExistingId};
|
||||
std::map<LabelName, LabelId> labels;
|
||||
std::map<LabelId, LabelSpace> label_spaces;
|
||||
std::map<LabelId, std::vector<SchemaProperty>> schemas;
|
||||
std::map<LabelName, LabelId> labels;
|
||||
|
||||
[[nodiscard]] static ShardMap Parse(std::istream &input_stream);
|
||||
friend std::ostream &operator<<(std::ostream &in, const ShardMap &shard_map);
|
||||
@ -136,6 +138,8 @@ struct ShardMap {
|
||||
Hlc IncrementShardMapVersion() noexcept;
|
||||
Hlc GetHlc() const noexcept;
|
||||
|
||||
std::unordered_map<uint64_t, std::string> IdToNames();
|
||||
|
||||
// Returns the shard UUIDs that have been assigned but not yet acknowledged for this storage manager
|
||||
std::vector<ShardToInitialize> AssignShards(Address storage_manager, std::set<boost::uuids::uuid> initialized);
|
||||
|
||||
|
@ -13,7 +13,7 @@
|
||||
#ifndef MG_AST_INCLUDE_PATH
|
||||
#ifdef MG_CLANG_TIDY_CHECK
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define MG_AST_INCLUDE_PATH "query/v2/frontend/ast/ast.hpp"
|
||||
#include "query/v2/bindings/bindings.hpp"
|
||||
#else
|
||||
#error Missing AST include path
|
||||
#endif
|
||||
@ -21,8 +21,6 @@
|
||||
|
||||
#ifndef MG_INJECTED_NAMESPACE_NAME
|
||||
#ifdef MG_CLANG_TIDY_CHECK
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define MG_INJECTED_NAMESPACE_NAME memgraph::query::v2
|
||||
#else
|
||||
#error Missing AST namespace
|
||||
#endif
|
||||
|
@ -718,7 +718,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
|
||||
TReturnType GetProperty(const TRecordAccessor &record_accessor, PropertyIx prop) {
|
||||
auto maybe_prop = record_accessor.GetProperty(prop.name);
|
||||
// Handler non existent property
|
||||
return conv_(maybe_prop);
|
||||
return conv_(maybe_prop, dba_);
|
||||
}
|
||||
|
||||
template <class TRecordAccessor, class TTag = Tag,
|
||||
@ -726,7 +726,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
|
||||
TReturnType GetProperty(const TRecordAccessor &record_accessor, const std::string_view name) {
|
||||
auto maybe_prop = record_accessor.GetProperty(std::string(name));
|
||||
// Handler non existent property
|
||||
return conv_(maybe_prop);
|
||||
return conv_(maybe_prop, dba_);
|
||||
}
|
||||
|
||||
template <class TRecordAccessor, class TTag = Tag,
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "query/v2/accessors.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
#include "storage/v3/edge_accessor.hpp"
|
||||
#include "storage/v3/id_types.hpp"
|
||||
#include "storage/v3/result.hpp"
|
||||
@ -70,51 +71,52 @@ query::v2::TypedValue ToTypedValue(const Value &value) {
|
||||
}
|
||||
}
|
||||
|
||||
storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const query::v2::accessors::VertexAccessor &vertex,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View /*view*/) {
|
||||
storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(
|
||||
const query::v2::accessors::VertexAccessor &vertex, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View /*view*/) {
|
||||
auto id = communication::bolt::Id::FromUint(0);
|
||||
|
||||
auto labels = vertex.Labels();
|
||||
std::vector<std::string> new_labels;
|
||||
new_labels.reserve(labels.size());
|
||||
for (const auto &label : labels) {
|
||||
new_labels.push_back(shard_map.GetLabelName(label.id));
|
||||
new_labels.push_back(shard_request_manager->LabelToName(label.id));
|
||||
}
|
||||
|
||||
auto properties = vertex.Properties();
|
||||
std::map<std::string, Value> new_properties;
|
||||
for (const auto &[prop, property_value] : properties) {
|
||||
new_properties[shard_map.GetPropertyName(prop)] = ToBoltValue(property_value);
|
||||
new_properties[shard_request_manager->PropertyToName(prop)] = ToBoltValue(property_value);
|
||||
}
|
||||
return communication::bolt::Vertex{id, new_labels, new_properties};
|
||||
}
|
||||
|
||||
storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const query::v2::accessors::EdgeAccessor &edge,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View /*view*/) {
|
||||
storage::v3::Result<communication::bolt::Edge> ToBoltEdge(
|
||||
const query::v2::accessors::EdgeAccessor &edge, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View /*view*/) {
|
||||
// TODO(jbajic) Fix bolt communication
|
||||
auto id = communication::bolt::Id::FromUint(0);
|
||||
auto from = communication::bolt::Id::FromUint(0);
|
||||
auto to = communication::bolt::Id::FromUint(0);
|
||||
const auto &type = shard_map.GetEdgeTypeName(edge.EdgeType());
|
||||
const auto &type = shard_request_manager->EdgeTypeToName(edge.EdgeType());
|
||||
|
||||
auto properties = edge.Properties();
|
||||
std::map<std::string, Value> new_properties;
|
||||
for (const auto &[prop, property_value] : properties) {
|
||||
new_properties[shard_map.GetPropertyName(prop)] = ToBoltValue(property_value);
|
||||
new_properties[shard_request_manager->PropertyToName(prop)] = ToBoltValue(property_value);
|
||||
}
|
||||
return communication::bolt::Edge{id, from, to, type, new_properties};
|
||||
}
|
||||
|
||||
storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::accessors::Path & /*edge*/,
|
||||
const coordinator::ShardMap & /*shard_map*/,
|
||||
storage::v3::View /*view*/) {
|
||||
storage::v3::Result<communication::bolt::Path> ToBoltPath(
|
||||
const query::v2::accessors::Path & /*edge*/, const msgs::ShardRequestManagerInterface * /*shard_request_manager*/,
|
||||
storage::v3::View /*view*/) {
|
||||
// TODO(jbajic) Fix bolt communication
|
||||
return {storage::v3::Error::DELETED_OBJECT};
|
||||
}
|
||||
|
||||
storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const coordinator::ShardMap &shard_map,
|
||||
storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value,
|
||||
const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view) {
|
||||
switch (value.type()) {
|
||||
case query::v2::TypedValue::Type::Null:
|
||||
@ -131,7 +133,7 @@ storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const
|
||||
std::vector<Value> values;
|
||||
values.reserve(value.ValueList().size());
|
||||
for (const auto &v : value.ValueList()) {
|
||||
auto maybe_value = ToBoltValue(v, shard_map, view);
|
||||
auto maybe_value = ToBoltValue(v, shard_request_manager, view);
|
||||
if (maybe_value.HasError()) return maybe_value.GetError();
|
||||
values.emplace_back(std::move(*maybe_value));
|
||||
}
|
||||
@ -140,24 +142,24 @@ storage::v3::Result<Value> ToBoltValue(const query::v2::TypedValue &value, const
|
||||
case query::v2::TypedValue::Type::Map: {
|
||||
std::map<std::string, Value> map;
|
||||
for (const auto &kv : value.ValueMap()) {
|
||||
auto maybe_value = ToBoltValue(kv.second, shard_map, view);
|
||||
auto maybe_value = ToBoltValue(kv.second, shard_request_manager, view);
|
||||
if (maybe_value.HasError()) return maybe_value.GetError();
|
||||
map.emplace(kv.first, std::move(*maybe_value));
|
||||
}
|
||||
return Value(std::move(map));
|
||||
}
|
||||
case query::v2::TypedValue::Type::Vertex: {
|
||||
auto maybe_vertex = ToBoltVertex(value.ValueVertex(), shard_map, view);
|
||||
auto maybe_vertex = ToBoltVertex(value.ValueVertex(), shard_request_manager, view);
|
||||
if (maybe_vertex.HasError()) return maybe_vertex.GetError();
|
||||
return Value(std::move(*maybe_vertex));
|
||||
}
|
||||
case query::v2::TypedValue::Type::Edge: {
|
||||
auto maybe_edge = ToBoltEdge(value.ValueEdge(), shard_map, view);
|
||||
auto maybe_edge = ToBoltEdge(value.ValueEdge(), shard_request_manager, view);
|
||||
if (maybe_edge.HasError()) return maybe_edge.GetError();
|
||||
return Value(std::move(*maybe_edge));
|
||||
}
|
||||
case query::v2::TypedValue::Type::Path: {
|
||||
auto maybe_path = ToBoltPath(value.ValuePath(), shard_map, view);
|
||||
auto maybe_path = ToBoltPath(value.ValuePath(), shard_request_manager, view);
|
||||
if (maybe_path.HasError()) return maybe_path.GetError();
|
||||
return Value(std::move(*maybe_path));
|
||||
}
|
||||
@ -209,12 +211,6 @@ Value ToBoltValue(msgs::Value value) {
|
||||
}
|
||||
}
|
||||
|
||||
storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::accessors::Path & /*path*/,
|
||||
const storage::v3::Shard & /*db*/,
|
||||
storage::v3::View /*view*/) {
|
||||
return communication::bolt::Path();
|
||||
}
|
||||
|
||||
storage::v3::PropertyValue ToPropertyValue(const Value &value) {
|
||||
switch (value.type()) {
|
||||
case Value::Type::Null:
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include "communication/bolt/v1/value.hpp"
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "query/v2/bindings/typed_value.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/result.hpp"
|
||||
#include "storage/v3/shard.hpp"
|
||||
@ -30,40 +31,40 @@ namespace memgraph::glue::v2 {
|
||||
|
||||
/// @param storage::v3::VertexAccessor for converting to
|
||||
/// communication::bolt::Vertex.
|
||||
/// @param coordinator::ShardMap shard_map getting label and property names.
|
||||
/// @param msgs::ShardRequestManagerInterface *shard_request_manager getting label and property names.
|
||||
/// @param storage::v3::View for deciding which vertex attributes are visible.
|
||||
///
|
||||
/// @throw std::bad_alloc
|
||||
storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(const storage::v3::VertexAccessor &vertex,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View view);
|
||||
storage::v3::Result<communication::bolt::Vertex> ToBoltVertex(
|
||||
const storage::v3::VertexAccessor &vertex, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view);
|
||||
|
||||
/// @param storage::v3::EdgeAccessor for converting to communication::bolt::Edge.
|
||||
/// @param coordinator::ShardMap shard_map getting edge type and property names.
|
||||
/// @param msgs::ShardRequestManagerInterface *shard_request_manager getting edge type and property names.
|
||||
/// @param storage::v3::View for deciding which edge attributes are visible.
|
||||
///
|
||||
/// @throw std::bad_alloc
|
||||
storage::v3::Result<communication::bolt::Edge> ToBoltEdge(const storage::v3::EdgeAccessor &edge,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View view);
|
||||
storage::v3::Result<communication::bolt::Edge> ToBoltEdge(
|
||||
const storage::v3::EdgeAccessor &edge, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view);
|
||||
|
||||
/// @param query::v2::Path for converting to communication::bolt::Path.
|
||||
/// @param coordinator::ShardMap shard_map ToBoltVertex and ToBoltEdge.
|
||||
/// @param msgs::ShardRequestManagerInterface *shard_request_manager ToBoltVertex and ToBoltEdge.
|
||||
/// @param storage::v3::View for ToBoltVertex and ToBoltEdge.
|
||||
///
|
||||
/// @throw std::bad_alloc
|
||||
storage::v3::Result<communication::bolt::Path> ToBoltPath(const query::v2::accessors::Path &path,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View view);
|
||||
storage::v3::Result<communication::bolt::Path> ToBoltPath(
|
||||
const query::v2::accessors::Path &path, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view);
|
||||
|
||||
/// @param query::v2::TypedValue for converting to communication::bolt::Value.
|
||||
/// @param coordinator::ShardMap shard_map ToBoltVertex and ToBoltEdge.
|
||||
/// @param msgs::ShardRequestManagerInterface *shard_request_manager ToBoltVertex and ToBoltEdge.
|
||||
/// @param storage::v3::View for ToBoltVertex and ToBoltEdge.
|
||||
///
|
||||
/// @throw std::bad_alloc
|
||||
storage::v3::Result<communication::bolt::Value> ToBoltValue(const query::v2::TypedValue &value,
|
||||
const coordinator::ShardMap &shard_map,
|
||||
storage::v3::View view);
|
||||
storage::v3::Result<communication::bolt::Value> ToBoltValue(
|
||||
const query::v2::TypedValue &value, const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view);
|
||||
|
||||
query::v2::TypedValue ToTypedValue(const communication::bolt::Value &value);
|
||||
|
||||
@ -73,7 +74,8 @@ storage::v3::PropertyValue ToPropertyValue(const communication::bolt::Value &val
|
||||
|
||||
communication::bolt::Value ToBoltValue(msgs::Value value);
|
||||
|
||||
communication::bolt::Value ToBoltValue(msgs::Value value, const coordinator::ShardMap &shard_map,
|
||||
communication::bolt::Value ToBoltValue(msgs::Value value,
|
||||
const msgs::ShardRequestManagerInterface *shard_request_manager,
|
||||
storage::v3::View view);
|
||||
|
||||
} // namespace memgraph::glue::v2
|
||||
|
@ -31,14 +31,9 @@ class LocalTransport {
|
||||
: local_transport_handle_(std::move(local_transport_handle)) {}
|
||||
|
||||
template <Message RequestT, Message ResponseT>
|
||||
ResponseFuture<ResponseT> Request(Address to_address, Address from_address, RequestId request_id, RequestT request,
|
||||
Duration timeout) {
|
||||
auto [future, promise] = memgraph::io::FuturePromisePair<ResponseResult<ResponseT>>();
|
||||
|
||||
local_transport_handle_->SubmitRequest(to_address, from_address, request_id, std::move(request), timeout,
|
||||
std::move(promise));
|
||||
|
||||
return std::move(future);
|
||||
ResponseFuture<ResponseT> Request(Address to_address, Address from_address, RequestT request, Duration timeout) {
|
||||
return local_transport_handle_->template SubmitRequest<RequestT, ResponseT>(to_address, from_address,
|
||||
std::move(request), timeout);
|
||||
}
|
||||
|
||||
template <Message... Ms>
|
||||
@ -60,5 +55,7 @@ class LocalTransport {
|
||||
std::random_device rng;
|
||||
return distrib(rng);
|
||||
}
|
||||
|
||||
LatencyHistogramSummaries ResponseLatencies() { return local_transport_handle_->ResponseLatencies(); }
|
||||
};
|
||||
}; // namespace memgraph::io::local_transport
|
||||
|
@ -30,6 +30,7 @@ class LocalTransportHandle {
|
||||
mutable std::condition_variable cv_;
|
||||
bool should_shut_down_ = false;
|
||||
MessageHistogramCollector histograms_;
|
||||
RequestId request_id_counter_ = 0;
|
||||
|
||||
// the responses to requests that are being waited on
|
||||
std::map<PromiseKey, DeadlineAndOpaquePromise> promises_;
|
||||
@ -56,7 +57,7 @@ class LocalTransportHandle {
|
||||
return should_shut_down_;
|
||||
}
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ResponseLatencies() {
|
||||
LatencyHistogramSummaries ResponseLatencies() {
|
||||
std::unique_lock<std::mutex> lock(mu_);
|
||||
return histograms_.ResponseLatencies();
|
||||
}
|
||||
@ -113,8 +114,7 @@ class LocalTransportHandle {
|
||||
.message = std::move(message_any),
|
||||
.type_info = type_info};
|
||||
|
||||
PromiseKey promise_key{
|
||||
.requester_address = to_address, .request_id = opaque_message.request_id, .replier_address = from_address};
|
||||
PromiseKey promise_key{.requester_address = to_address, .request_id = opaque_message.request_id};
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mu_);
|
||||
@ -139,8 +139,10 @@ class LocalTransportHandle {
|
||||
}
|
||||
|
||||
template <Message RequestT, Message ResponseT>
|
||||
void SubmitRequest(Address to_address, Address from_address, RequestId request_id, RequestT &&request,
|
||||
Duration timeout, ResponsePromise<ResponseT> promise) {
|
||||
ResponseFuture<ResponseT> SubmitRequest(Address to_address, Address from_address, RequestT &&request,
|
||||
Duration timeout) {
|
||||
auto [future, promise] = memgraph::io::FuturePromisePair<ResponseResult<ResponseT>>();
|
||||
|
||||
const bool port_matches = to_address.last_known_port == from_address.last_known_port;
|
||||
const bool ip_matches = to_address.last_known_ip == from_address.last_known_ip;
|
||||
|
||||
@ -149,17 +151,22 @@ class LocalTransportHandle {
|
||||
const auto now = Now();
|
||||
const Time deadline = now + timeout;
|
||||
|
||||
RequestId request_id = 0;
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mu_);
|
||||
|
||||
PromiseKey promise_key{
|
||||
.requester_address = from_address, .request_id = request_id, .replier_address = to_address};
|
||||
request_id = ++request_id_counter_;
|
||||
PromiseKey promise_key{.requester_address = from_address, .request_id = request_id};
|
||||
OpaquePromise opaque_promise(std::move(promise).ToUnique());
|
||||
DeadlineAndOpaquePromise dop{.requested_at = now, .deadline = deadline, .promise = std::move(opaque_promise)};
|
||||
|
||||
MG_ASSERT(!promises_.contains(promise_key));
|
||||
promises_.emplace(std::move(promise_key), std::move(dop));
|
||||
} // lock dropped
|
||||
|
||||
Send(to_address, from_address, request_id, std::forward<RequestT>(request));
|
||||
|
||||
return std::move(future);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -11,6 +11,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <boost/core/demangle.hpp>
|
||||
|
||||
#include "io/transport.hpp"
|
||||
#include "utils/type_info_ref.hpp"
|
||||
|
||||
@ -19,9 +21,6 @@ namespace memgraph::io {
|
||||
struct PromiseKey {
|
||||
Address requester_address;
|
||||
uint64_t request_id;
|
||||
// TODO(tyler) possibly remove replier_address from promise key
|
||||
// once we want to support DSR.
|
||||
Address replier_address;
|
||||
|
||||
public:
|
||||
friend bool operator<(const PromiseKey &lhs, const PromiseKey &rhs) {
|
||||
@ -29,11 +28,7 @@ struct PromiseKey {
|
||||
return lhs.requester_address < rhs.requester_address;
|
||||
}
|
||||
|
||||
if (lhs.request_id != rhs.request_id) {
|
||||
return lhs.request_id < rhs.request_id;
|
||||
}
|
||||
|
||||
return lhs.replier_address < rhs.replier_address;
|
||||
return lhs.request_id < rhs.request_id;
|
||||
}
|
||||
};
|
||||
|
||||
@ -90,6 +85,10 @@ struct OpaqueMessage {
|
||||
};
|
||||
}
|
||||
|
||||
std::string demangled_name = "\"" + boost::core::demangle(message.type().name()) + "\"";
|
||||
spdlog::error("failed to cast message of type {} to expected request type (probably in Receive argument types)",
|
||||
demangled_name);
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
};
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include "io/time.hpp"
|
||||
#include "utils/histogram.hpp"
|
||||
#include "utils/logging.hpp"
|
||||
#include "utils/print_helpers.hpp"
|
||||
#include "utils/type_info_ref.hpp"
|
||||
|
||||
namespace memgraph::io {
|
||||
@ -57,6 +58,35 @@ struct LatencyHistogramSummary {
|
||||
}
|
||||
};
|
||||
|
||||
struct LatencyHistogramSummaries {
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> latencies;
|
||||
|
||||
std::string SummaryTable() {
|
||||
std::string output;
|
||||
|
||||
const auto row = [&output](const auto &c1, const auto &c2, const auto &c3, const auto &c4, const auto &c5,
|
||||
const auto &c6, const auto &c7) {
|
||||
output +=
|
||||
fmt::format("{: >50} | {: >8} | {: >8} | {: >8} | {: >8} | {: >8} | {: >8}\n", c1, c2, c3, c4, c5, c6, c7);
|
||||
};
|
||||
row("name", "count", "min (μs)", "med (μs)", "p99 (μs)", "max (μs)", "sum (ms)");
|
||||
|
||||
for (const auto &[name, histo] : latencies) {
|
||||
row(name, histo.count, histo.p0.count(), histo.p50.count(), histo.p99.count(), histo.p100.count(),
|
||||
histo.sum.count() / 1000);
|
||||
}
|
||||
|
||||
output += "\n";
|
||||
return output;
|
||||
}
|
||||
|
||||
friend std::ostream &operator<<(std::ostream &in, const LatencyHistogramSummaries &histo) {
|
||||
using memgraph::utils::print_helpers::operator<<;
|
||||
in << histo.latencies;
|
||||
return in;
|
||||
}
|
||||
};
|
||||
|
||||
class MessageHistogramCollector {
|
||||
std::unordered_map<utils::TypeInfoRef, utils::Histogram, utils::TypeInfoHasher, utils::TypeInfoEqualTo> histograms_;
|
||||
|
||||
@ -66,7 +96,7 @@ class MessageHistogramCollector {
|
||||
histo.Measure(duration.count());
|
||||
}
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ResponseLatencies() {
|
||||
LatencyHistogramSummaries ResponseLatencies() {
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ret{};
|
||||
|
||||
for (const auto &[type_id, histo] : histograms_) {
|
||||
@ -90,7 +120,7 @@ class MessageHistogramCollector {
|
||||
ret.emplace(demangled_name, latency_histogram_summary);
|
||||
}
|
||||
|
||||
return ret;
|
||||
return LatencyHistogramSummaries{.latencies = ret};
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -22,6 +22,8 @@
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
#include <boost/core/demangle.hpp>
|
||||
|
||||
#include "io/message_conversion.hpp"
|
||||
#include "io/simulator/simulator.hpp"
|
||||
#include "io/transport.hpp"
|
||||
@ -109,6 +111,16 @@ utils::TypeInfoRef TypeInfoFor(const WriteResponse<WriteReturn> & /* write_respo
|
||||
return typeid(WriteReturn);
|
||||
}
|
||||
|
||||
template <class WriteOperation>
|
||||
utils::TypeInfoRef TypeInfoFor(const WriteRequest<WriteOperation> & /* write_request */) {
|
||||
return typeid(WriteOperation);
|
||||
}
|
||||
|
||||
template <class... WriteOperations>
|
||||
utils::TypeInfoRef TypeInfoFor(const WriteRequest<std::variant<WriteOperations...>> &write_request) {
|
||||
return TypeInfoForVariant(write_request.operation);
|
||||
}
|
||||
|
||||
/// AppendRequest is a raft-level message that the Leader
|
||||
/// periodically broadcasts to all Follower peers. This
|
||||
/// serves three main roles:
|
||||
@ -569,7 +581,7 @@ class Raft {
|
||||
const Time now = io_.Now();
|
||||
const Duration broadcast_timeout = RandomTimeout(kMinimumBroadcastTimeout, kMaximumBroadcastTimeout);
|
||||
|
||||
if (now - leader.last_broadcast > broadcast_timeout) {
|
||||
if (now > leader.last_broadcast + broadcast_timeout) {
|
||||
BroadcastAppendEntries(leader.followers);
|
||||
leader.last_broadcast = now;
|
||||
}
|
||||
@ -918,7 +930,9 @@ class Raft {
|
||||
// only leaders actually handle replication requests from clients
|
||||
std::optional<Role> Handle(Leader &leader, WriteRequest<WriteOperation> &&req, RequestId request_id,
|
||||
Address from_address) {
|
||||
Log("handling WriteRequest");
|
||||
auto type_info = TypeInfoFor(req);
|
||||
std::string demangled_name = boost::core::demangle(type_info.get().name());
|
||||
Log("handling WriteRequest<" + demangled_name + ">");
|
||||
|
||||
// we are the leader. add item to log and send Append to peers
|
||||
MG_ASSERT(state_.term >= LastLogTerm());
|
||||
|
@ -31,7 +31,7 @@ bool SimulatorHandle::ShouldShutDown() const {
|
||||
return should_shut_down_;
|
||||
}
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> SimulatorHandle::ResponseLatencies() {
|
||||
LatencyHistogramSummaries SimulatorHandle::ResponseLatencies() {
|
||||
std::unique_lock<std::mutex> lock(mu_);
|
||||
return histograms_.ResponseLatencies();
|
||||
}
|
||||
@ -108,9 +108,7 @@ bool SimulatorHandle::MaybeTickSimulator() {
|
||||
stats_.dropped_messages++;
|
||||
}
|
||||
|
||||
PromiseKey promise_key{.requester_address = to_address,
|
||||
.request_id = opaque_message.request_id,
|
||||
.replier_address = opaque_message.from_address};
|
||||
PromiseKey promise_key{.requester_address = to_address, .request_id = opaque_message.request_id};
|
||||
|
||||
if (promises_.contains(promise_key)) {
|
||||
// complete waiting promise if it's there
|
||||
|
@ -56,14 +56,14 @@ class SimulatorHandle {
|
||||
std::uniform_int_distribution<int> drop_distrib_{0, 99};
|
||||
SimulatorConfig config_;
|
||||
MessageHistogramCollector histograms_;
|
||||
RequestId request_id_counter_{0};
|
||||
|
||||
void TimeoutPromisesPastDeadline() {
|
||||
const Time now = cluster_wide_time_microseconds_;
|
||||
for (auto it = promises_.begin(); it != promises_.end();) {
|
||||
auto &[promise_key, dop] = *it;
|
||||
if (dop.deadline < now && config_.perform_timeouts) {
|
||||
spdlog::info("timing out request from requester {} to replier {}.", promise_key.requester_address.ToString(),
|
||||
promise_key.replier_address.ToString());
|
||||
spdlog::info("timing out request from requester {}.", promise_key.requester_address.ToString());
|
||||
std::move(dop).promise.TimeOut();
|
||||
it = promises_.erase(it);
|
||||
|
||||
@ -78,7 +78,7 @@ class SimulatorHandle {
|
||||
explicit SimulatorHandle(SimulatorConfig config)
|
||||
: cluster_wide_time_microseconds_(config.start_time), rng_(config.rng_seed), config_(config) {}
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ResponseLatencies();
|
||||
LatencyHistogramSummaries ResponseLatencies();
|
||||
|
||||
~SimulatorHandle() {
|
||||
for (auto it = promises_.begin(); it != promises_.end();) {
|
||||
@ -101,12 +101,17 @@ class SimulatorHandle {
|
||||
bool ShouldShutDown() const;
|
||||
|
||||
template <Message Request, Message Response>
|
||||
void SubmitRequest(Address to_address, Address from_address, RequestId request_id, Request &&request,
|
||||
Duration timeout, ResponsePromise<Response> &&promise) {
|
||||
ResponseFuture<Response> SubmitRequest(Address to_address, Address from_address, Request &&request, Duration timeout,
|
||||
std::function<bool()> &&maybe_tick_simulator) {
|
||||
auto type_info = TypeInfoFor(request);
|
||||
|
||||
auto [future, promise] = memgraph::io::FuturePromisePairWithNotifier<ResponseResult<Response>>(
|
||||
std::forward<std::function<bool()>>(maybe_tick_simulator));
|
||||
|
||||
std::unique_lock<std::mutex> lock(mu_);
|
||||
|
||||
RequestId request_id = ++request_id_counter_;
|
||||
|
||||
const Time deadline = cluster_wide_time_microseconds_ + timeout;
|
||||
|
||||
std::any message(request);
|
||||
@ -117,19 +122,24 @@ class SimulatorHandle {
|
||||
.type_info = type_info};
|
||||
in_flight_.emplace_back(std::make_pair(to_address, std::move(om)));
|
||||
|
||||
PromiseKey promise_key{.requester_address = from_address, .request_id = request_id, .replier_address = to_address};
|
||||
PromiseKey promise_key{.requester_address = from_address, .request_id = request_id};
|
||||
OpaquePromise opaque_promise(std::move(promise).ToUnique());
|
||||
DeadlineAndOpaquePromise dop{
|
||||
.requested_at = cluster_wide_time_microseconds_,
|
||||
.deadline = deadline,
|
||||
.promise = std::move(opaque_promise),
|
||||
};
|
||||
|
||||
MG_ASSERT(!promises_.contains(promise_key));
|
||||
|
||||
promises_.emplace(std::move(promise_key), std::move(dop));
|
||||
|
||||
stats_.total_messages++;
|
||||
stats_.total_requests++;
|
||||
|
||||
cv_.notify_all();
|
||||
|
||||
return std::move(future);
|
||||
}
|
||||
|
||||
template <Message... Ms>
|
||||
|
@ -33,16 +33,11 @@ class SimulatorTransport {
|
||||
: simulator_handle_(simulator_handle), address_(address), rng_(std::mt19937{seed}) {}
|
||||
|
||||
template <Message RequestT, Message ResponseT>
|
||||
ResponseFuture<ResponseT> Request(Address to_address, Address from_address, uint64_t request_id, RequestT request,
|
||||
Duration timeout) {
|
||||
ResponseFuture<ResponseT> Request(Address to_address, Address from_address, RequestT request, Duration timeout) {
|
||||
std::function<bool()> maybe_tick_simulator = [this] { return simulator_handle_->MaybeTickSimulator(); };
|
||||
auto [future, promise] =
|
||||
memgraph::io::FuturePromisePairWithNotifier<ResponseResult<ResponseT>>(maybe_tick_simulator);
|
||||
|
||||
simulator_handle_->SubmitRequest(to_address, from_address, request_id, std::move(request), timeout,
|
||||
std::move(promise));
|
||||
|
||||
return std::move(future);
|
||||
return simulator_handle_->template SubmitRequest<RequestT, ResponseT>(to_address, from_address, std::move(request),
|
||||
timeout, std::move(maybe_tick_simulator));
|
||||
}
|
||||
|
||||
template <Message... Ms>
|
||||
@ -64,8 +59,6 @@ class SimulatorTransport {
|
||||
return distrib(rng_);
|
||||
}
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ResponseLatencies() {
|
||||
return simulator_handle_->ResponseLatencies();
|
||||
}
|
||||
LatencyHistogramSummaries ResponseLatencies() { return simulator_handle_->ResponseLatencies(); }
|
||||
};
|
||||
}; // namespace memgraph::io::simulator
|
||||
|
@ -68,7 +68,6 @@ template <typename I>
|
||||
class Io {
|
||||
I implementation_;
|
||||
Address address_;
|
||||
RequestId request_id_counter_ = 0;
|
||||
Duration default_timeout_ = std::chrono::microseconds{100000};
|
||||
|
||||
public:
|
||||
@ -84,20 +83,17 @@ class Io {
|
||||
/// Issue a request with an explicit timeout in microseconds provided. This tends to be used by clients.
|
||||
template <Message RequestT, Message ResponseT>
|
||||
ResponseFuture<ResponseT> RequestWithTimeout(Address address, RequestT request, Duration timeout) {
|
||||
const RequestId request_id = ++request_id_counter_;
|
||||
const Address from_address = address_;
|
||||
return implementation_.template Request<RequestT, ResponseT>(address, from_address, request_id, request, timeout);
|
||||
return implementation_.template Request<RequestT, ResponseT>(address, from_address, request, timeout);
|
||||
}
|
||||
|
||||
/// Issue a request that times out after the default timeout. This tends
|
||||
/// to be used by clients.
|
||||
template <Message RequestT, Message ResponseT>
|
||||
ResponseFuture<ResponseT> Request(Address to_address, RequestT request) {
|
||||
const RequestId request_id = ++request_id_counter_;
|
||||
const Duration timeout = default_timeout_;
|
||||
const Address from_address = address_;
|
||||
return implementation_.template Request<RequestT, ResponseT>(to_address, from_address, request_id,
|
||||
std::move(request), timeout);
|
||||
return implementation_.template Request<RequestT, ResponseT>(to_address, from_address, std::move(request), timeout);
|
||||
}
|
||||
|
||||
/// Wait for an explicit number of microseconds for a request of one of the
|
||||
@ -143,8 +139,6 @@ class Io {
|
||||
|
||||
Io<I> ForkLocal() { return Io(implementation_, address_.ForkUniqueAddress()); }
|
||||
|
||||
std::unordered_map<std::string, LatencyHistogramSummary> ResponseLatencies() {
|
||||
return implementation_.ResponseLatencies();
|
||||
}
|
||||
LatencyHistogramSummaries ResponseLatencies() { return implementation_.ResponseLatencies(); }
|
||||
};
|
||||
}; // namespace memgraph::io
|
||||
|
@ -11,7 +11,11 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <thread>
|
||||
|
||||
#include <boost/asio/ip/tcp.hpp>
|
||||
|
||||
#include "io/address.hpp"
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/schemas.hpp"
|
||||
@ -37,6 +41,7 @@ struct MachineConfig {
|
||||
bool is_query_engine;
|
||||
boost::asio::ip::address listen_ip;
|
||||
uint16_t listen_port;
|
||||
size_t shard_worker_threads = std::max(static_cast<unsigned int>(1), std::thread::hardware_concurrency());
|
||||
};
|
||||
|
||||
} // namespace memgraph::machine_manager
|
||||
|
@ -11,40 +11,43 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <coordinator/coordinator_rsm.hpp>
|
||||
#include <io/message_conversion.hpp>
|
||||
#include <io/messages.hpp>
|
||||
#include <io/rsm/rsm_client.hpp>
|
||||
#include <io/time.hpp>
|
||||
#include <machine_manager/machine_config.hpp>
|
||||
#include <storage/v3/shard_manager.hpp>
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "coordinator/coordinator_rsm.hpp"
|
||||
#include "coordinator/coordinator_worker.hpp"
|
||||
#include "io/message_conversion.hpp"
|
||||
#include "io/messages.hpp"
|
||||
#include "io/rsm/rsm_client.hpp"
|
||||
#include "io/time.hpp"
|
||||
#include "machine_manager/machine_config.hpp"
|
||||
#include "storage/v3/shard_manager.hpp"
|
||||
|
||||
namespace memgraph::machine_manager {
|
||||
|
||||
using memgraph::coordinator::Coordinator;
|
||||
using memgraph::coordinator::CoordinatorReadRequests;
|
||||
using memgraph::coordinator::CoordinatorReadResponses;
|
||||
using memgraph::coordinator::CoordinatorRsm;
|
||||
using memgraph::coordinator::CoordinatorWriteRequests;
|
||||
using memgraph::coordinator::CoordinatorWriteResponses;
|
||||
using memgraph::io::ConvertVariant;
|
||||
using memgraph::io::Duration;
|
||||
using memgraph::io::RequestId;
|
||||
using memgraph::io::Time;
|
||||
using memgraph::io::messages::CoordinatorMessages;
|
||||
using memgraph::io::messages::ShardManagerMessages;
|
||||
using memgraph::io::messages::ShardMessages;
|
||||
using memgraph::io::messages::StorageReadRequest;
|
||||
using memgraph::io::messages::StorageWriteRequest;
|
||||
using memgraph::io::rsm::AppendRequest;
|
||||
using memgraph::io::rsm::AppendResponse;
|
||||
using memgraph::io::rsm::ReadRequest;
|
||||
using memgraph::io::rsm::VoteRequest;
|
||||
using memgraph::io::rsm::VoteResponse;
|
||||
using memgraph::io::rsm::WriteRequest;
|
||||
using memgraph::io::rsm::WriteResponse;
|
||||
using memgraph::storage::v3::ShardManager;
|
||||
using coordinator::Coordinator;
|
||||
using coordinator::CoordinatorReadRequests;
|
||||
using coordinator::CoordinatorReadResponses;
|
||||
using coordinator::CoordinatorRsm;
|
||||
using coordinator::CoordinatorWriteRequests;
|
||||
using coordinator::CoordinatorWriteResponses;
|
||||
using coordinator::coordinator_worker::CoordinatorWorker;
|
||||
using CoordinatorRouteMessage = coordinator::coordinator_worker::RouteMessage;
|
||||
using CoordinatorQueue = coordinator::coordinator_worker::Queue;
|
||||
using io::ConvertVariant;
|
||||
using io::Duration;
|
||||
using io::RequestId;
|
||||
using io::Time;
|
||||
using io::messages::CoordinatorMessages;
|
||||
using io::messages::ShardManagerMessages;
|
||||
using io::messages::ShardMessages;
|
||||
using io::messages::StorageReadRequest;
|
||||
using io::messages::StorageWriteRequest;
|
||||
using io::rsm::AppendRequest;
|
||||
using io::rsm::AppendResponse;
|
||||
using io::rsm::ReadRequest;
|
||||
using io::rsm::VoteRequest;
|
||||
using io::rsm::VoteResponse;
|
||||
using io::rsm::WriteRequest;
|
||||
using io::rsm::WriteResponse;
|
||||
using storage::v3::ShardManager;
|
||||
|
||||
/// The MachineManager is responsible for:
|
||||
/// * starting the entire system and ensuring that high-level
|
||||
@ -63,20 +66,39 @@ template <typename IoImpl>
|
||||
class MachineManager {
|
||||
io::Io<IoImpl> io_;
|
||||
MachineConfig config_;
|
||||
CoordinatorRsm<IoImpl> coordinator_;
|
||||
Address coordinator_address_;
|
||||
CoordinatorQueue coordinator_queue_;
|
||||
std::jthread coordinator_handle_;
|
||||
ShardManager<IoImpl> shard_manager_;
|
||||
Time next_cron_ = Time::min();
|
||||
|
||||
public:
|
||||
// TODO initialize ShardManager with "real" coordinator addresses instead of io.GetAddress
|
||||
// which is only true for single-machine config.
|
||||
MachineManager(io::Io<IoImpl> io, MachineConfig config, Coordinator coordinator, coordinator::ShardMap &shard_map)
|
||||
MachineManager(io::Io<IoImpl> io, MachineConfig config, Coordinator coordinator)
|
||||
: io_(io),
|
||||
config_(config),
|
||||
coordinator_{std::move(io.ForkLocal()), {}, std::move(coordinator)},
|
||||
shard_manager_{io.ForkLocal(), coordinator_.GetAddress(), shard_map} {}
|
||||
coordinator_address_(io.GetAddress().ForkUniqueAddress()),
|
||||
shard_manager_{io.ForkLocal(), config.shard_worker_threads, coordinator_address_} {
|
||||
auto coordinator_io = io.ForkLocal();
|
||||
coordinator_io.SetAddress(coordinator_address_);
|
||||
CoordinatorWorker coordinator_worker{coordinator_io, coordinator_queue_, coordinator};
|
||||
coordinator_handle_ = std::jthread([coordinator = std::move(coordinator_worker)]() mutable { coordinator.Run(); });
|
||||
}
|
||||
|
||||
Address CoordinatorAddress() { return coordinator_.GetAddress(); }
|
||||
MachineManager(MachineManager &&) noexcept = default;
|
||||
MachineManager &operator=(MachineManager &&) noexcept = default;
|
||||
MachineManager(const MachineManager &) = delete;
|
||||
MachineManager &operator=(const MachineManager &) = delete;
|
||||
|
||||
~MachineManager() {
|
||||
if (coordinator_handle_.joinable()) {
|
||||
coordinator_queue_.Push(coordinator::coordinator_worker::ShutDown{});
|
||||
coordinator_handle_.join();
|
||||
}
|
||||
}
|
||||
|
||||
Address CoordinatorAddress() { return coordinator_address_; }
|
||||
|
||||
void Run() {
|
||||
while (!io_.ShouldShutDown()) {
|
||||
@ -86,7 +108,7 @@ class MachineManager {
|
||||
next_cron_ = Cron();
|
||||
}
|
||||
|
||||
Duration receive_timeout = next_cron_ - now;
|
||||
Duration receive_timeout = std::max(next_cron_, now) - now;
|
||||
|
||||
// Note: this parameter pack must be kept in-sync with the ReceiveWithTimeout parameter pack below
|
||||
using AllMessages =
|
||||
@ -114,7 +136,7 @@ class MachineManager {
|
||||
spdlog::info("MM got message to {}", request_envelope.to_address.ToString());
|
||||
|
||||
// If message is for the coordinator, cast it to subset and pass it to the coordinator
|
||||
bool to_coordinator = coordinator_.GetAddress() == request_envelope.to_address;
|
||||
bool to_coordinator = coordinator_address_ == request_envelope.to_address;
|
||||
if (to_coordinator) {
|
||||
std::optional<CoordinatorMessages> conversion_attempt =
|
||||
ConvertVariant<AllMessages, ReadRequest<CoordinatorReadRequests>, AppendRequest<CoordinatorWriteRequests>,
|
||||
@ -127,8 +149,13 @@ class MachineManager {
|
||||
|
||||
CoordinatorMessages &&cm = std::move(conversion_attempt.value());
|
||||
|
||||
coordinator_.Handle(std::forward<CoordinatorMessages>(cm), request_envelope.request_id,
|
||||
request_envelope.from_address);
|
||||
CoordinatorRouteMessage route_message{
|
||||
.message = std::move(cm),
|
||||
.request_id = request_envelope.request_id,
|
||||
.to = request_envelope.to_address,
|
||||
.from = request_envelope.from_address,
|
||||
};
|
||||
coordinator_queue_.Push(std::move(route_message));
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -169,6 +196,7 @@ class MachineManager {
|
||||
private:
|
||||
Time Cron() {
|
||||
spdlog::info("running MachineManager::Cron, address {}", io_.GetAddress().ToString());
|
||||
coordinator_queue_.Push(coordinator::coordinator_worker::Cron{});
|
||||
return shard_manager_.Cron();
|
||||
}
|
||||
};
|
||||
|
@ -407,9 +407,8 @@ DEFINE_string(organization_name, "", "Organization name.");
|
||||
struct SessionData {
|
||||
// Explicit constructor here to ensure that pointers to all objects are
|
||||
// supplied.
|
||||
SessionData(memgraph::coordinator::ShardMap &shard_map, memgraph::query::v2::InterpreterContext *interpreter_context)
|
||||
: shard_map(&shard_map), interpreter_context(interpreter_context) {}
|
||||
memgraph::coordinator::ShardMap *shard_map;
|
||||
explicit SessionData(memgraph::query::v2::InterpreterContext *interpreter_context)
|
||||
: interpreter_context(interpreter_context) {}
|
||||
memgraph::query::v2::InterpreterContext *interpreter_context;
|
||||
};
|
||||
|
||||
@ -424,7 +423,6 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
|
||||
memgraph::communication::v2::OutputStream *output_stream)
|
||||
: memgraph::communication::bolt::Session<memgraph::communication::v2::InputStream,
|
||||
memgraph::communication::v2::OutputStream>(input_stream, output_stream),
|
||||
shard_map_(data.shard_map),
|
||||
interpreter_(data.interpreter_context),
|
||||
endpoint_(endpoint) {}
|
||||
|
||||
@ -455,7 +453,7 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
|
||||
|
||||
std::map<std::string, memgraph::communication::bolt::Value> Pull(TEncoder *encoder, std::optional<int> n,
|
||||
std::optional<int> qid) override {
|
||||
TypedValueResultStream stream(encoder, *shard_map_);
|
||||
TypedValueResultStream stream(encoder, interpreter_.GetShardRequestManager());
|
||||
return PullResults(stream, n, qid);
|
||||
}
|
||||
|
||||
@ -482,7 +480,8 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
|
||||
const auto &summary = interpreter_.Pull(&stream, n, qid);
|
||||
std::map<std::string, memgraph::communication::bolt::Value> decoded_summary;
|
||||
for (const auto &kv : summary) {
|
||||
auto maybe_value = memgraph::glue::v2::ToBoltValue(kv.second, *shard_map_, memgraph::storage::v3::View::NEW);
|
||||
auto maybe_value = memgraph::glue::v2::ToBoltValue(kv.second, interpreter_.GetShardRequestManager(),
|
||||
memgraph::storage::v3::View::NEW);
|
||||
if (maybe_value.HasError()) {
|
||||
switch (maybe_value.GetError()) {
|
||||
case memgraph::storage::v3::Error::DELETED_OBJECT:
|
||||
@ -507,14 +506,14 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
|
||||
/// before forwarding the calls to original TEncoder.
|
||||
class TypedValueResultStream {
|
||||
public:
|
||||
TypedValueResultStream(TEncoder *encoder, const memgraph::coordinator::ShardMap &shard_map)
|
||||
: encoder_(encoder), shard_map_(&shard_map) {}
|
||||
TypedValueResultStream(TEncoder *encoder, const memgraph::msgs::ShardRequestManagerInterface *shard_request_manager)
|
||||
: encoder_(encoder), shard_request_manager_(shard_request_manager) {}
|
||||
|
||||
void Result(const std::vector<memgraph::query::v2::TypedValue> &values) {
|
||||
std::vector<memgraph::communication::bolt::Value> decoded_values;
|
||||
decoded_values.reserve(values.size());
|
||||
for (const auto &v : values) {
|
||||
auto maybe_value = memgraph::glue::v2::ToBoltValue(v, *shard_map_, memgraph::storage::v3::View::NEW);
|
||||
auto maybe_value = memgraph::glue::v2::ToBoltValue(v, shard_request_manager_, memgraph::storage::v3::View::NEW);
|
||||
if (maybe_value.HasError()) {
|
||||
switch (maybe_value.GetError()) {
|
||||
case memgraph::storage::v3::Error::DELETED_OBJECT:
|
||||
@ -534,12 +533,8 @@ class BoltSession final : public memgraph::communication::bolt::Session<memgraph
|
||||
|
||||
private:
|
||||
TEncoder *encoder_;
|
||||
// NOTE: Needed only for ToBoltValue conversions
|
||||
const memgraph::coordinator::ShardMap *shard_map_;
|
||||
const memgraph::msgs::ShardRequestManagerInterface *shard_request_manager_{nullptr};
|
||||
};
|
||||
|
||||
// NOTE: Needed only for ToBoltValue conversions
|
||||
const memgraph::coordinator::ShardMap *shard_map_;
|
||||
memgraph::query::v2::Interpreter interpreter_;
|
||||
memgraph::communication::v2::ServerEndpoint endpoint_;
|
||||
};
|
||||
@ -663,8 +658,7 @@ int main(int argc, char **argv) {
|
||||
|
||||
memgraph::coordinator::Coordinator coordinator{sm};
|
||||
|
||||
memgraph::machine_manager::MachineManager<memgraph::io::local_transport::LocalTransport> mm{io, config, coordinator,
|
||||
sm};
|
||||
memgraph::machine_manager::MachineManager<memgraph::io::local_transport::LocalTransport> mm{io, config, coordinator};
|
||||
std::jthread mm_thread([&mm] { mm.Run(); });
|
||||
|
||||
memgraph::query::v2::InterpreterContext interpreter_context{
|
||||
@ -680,7 +674,7 @@ int main(int argc, char **argv) {
|
||||
std::move(io),
|
||||
mm.CoordinatorAddress()};
|
||||
|
||||
SessionData session_data{sm, &interpreter_context};
|
||||
SessionData session_data{&interpreter_context};
|
||||
|
||||
interpreter_context.auth = nullptr;
|
||||
interpreter_context.auth_checker = nullptr;
|
||||
|
@ -11,38 +11,59 @@
|
||||
|
||||
#include "query/v2/accessors.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
#include "storage/v3/id_types.hpp"
|
||||
|
||||
namespace memgraph::query::v2::accessors {
|
||||
EdgeAccessor::EdgeAccessor(Edge edge) : edge(std::move(edge)) {}
|
||||
EdgeAccessor::EdgeAccessor(Edge edge, const msgs::ShardRequestManagerInterface *manager)
|
||||
: edge(std::move(edge)), manager_(manager) {}
|
||||
|
||||
EdgeTypeId EdgeAccessor::EdgeType() const { return edge.type.id; }
|
||||
|
||||
const std::vector<std::pair<PropertyId, Value>> &EdgeAccessor::Properties() const {
|
||||
return edge.properties;
|
||||
// std::map<std::string, TypedValue> res;
|
||||
// for (const auto &[name, value] : *properties) {
|
||||
// res[name] = ValueToTypedValue(value);
|
||||
// }
|
||||
// return res;
|
||||
}
|
||||
const std::vector<std::pair<PropertyId, Value>> &EdgeAccessor::Properties() const { return edge.properties; }
|
||||
|
||||
// NOLINTNEXTLINE(readability-convert-member-functions-to-static)
|
||||
Value EdgeAccessor::GetProperty(const std::string & /*prop_name*/) const {
|
||||
// TODO(kostasrim) fix this
|
||||
return {};
|
||||
Value EdgeAccessor::GetProperty(const std::string &prop_name) const {
|
||||
auto prop_id = manager_->NameToProperty(prop_name);
|
||||
auto it = std::find_if(edge.properties.begin(), edge.properties.end(), [&](auto &pr) { return prop_id == pr.first; });
|
||||
if (it == edge.properties.end()) {
|
||||
return {};
|
||||
}
|
||||
return it->second;
|
||||
}
|
||||
|
||||
const Edge &EdgeAccessor::GetEdge() const { return edge; }
|
||||
|
||||
bool EdgeAccessor::IsCycle() const { return edge.src == edge.dst; };
|
||||
|
||||
VertexAccessor EdgeAccessor::To() const { return VertexAccessor(Vertex{edge.dst}, {}); }
|
||||
VertexAccessor EdgeAccessor::To() const {
|
||||
return VertexAccessor(Vertex{edge.dst}, std::vector<std::pair<PropertyId, msgs::Value>>{}, manager_);
|
||||
}
|
||||
|
||||
VertexAccessor EdgeAccessor::From() const { return VertexAccessor(Vertex{edge.src}, {}); }
|
||||
VertexAccessor EdgeAccessor::From() const {
|
||||
return VertexAccessor(Vertex{edge.src}, std::vector<std::pair<PropertyId, msgs::Value>>{}, manager_);
|
||||
}
|
||||
|
||||
VertexAccessor::VertexAccessor(Vertex v, std::vector<std::pair<PropertyId, Value>> props)
|
||||
: vertex(std::move(v)), properties(std::move(props)) {}
|
||||
VertexAccessor::VertexAccessor(Vertex v, std::vector<std::pair<PropertyId, Value>> props,
|
||||
const msgs::ShardRequestManagerInterface *manager)
|
||||
: vertex(std::move(v)), properties(std::move(props)), manager_(manager) {}
|
||||
|
||||
VertexAccessor::VertexAccessor(Vertex v, std::map<PropertyId, Value> &&props,
|
||||
const msgs::ShardRequestManagerInterface *manager)
|
||||
: vertex(std::move(v)), manager_(manager) {
|
||||
properties.reserve(props.size());
|
||||
for (auto &[id, value] : props) {
|
||||
properties.emplace_back(std::make_pair(id, std::move(value)));
|
||||
}
|
||||
}
|
||||
|
||||
VertexAccessor::VertexAccessor(Vertex v, const std::map<PropertyId, Value> &props,
|
||||
const msgs::ShardRequestManagerInterface *manager)
|
||||
: vertex(std::move(v)), manager_(manager) {
|
||||
properties.reserve(props.size());
|
||||
for (const auto &[id, value] : props) {
|
||||
properties.emplace_back(std::make_pair(id, value));
|
||||
}
|
||||
}
|
||||
|
||||
Label VertexAccessor::PrimaryLabel() const { return vertex.id.first; }
|
||||
|
||||
@ -58,15 +79,16 @@ bool VertexAccessor::HasLabel(Label &label) const {
|
||||
const std::vector<std::pair<PropertyId, Value>> &VertexAccessor::Properties() const { return properties; }
|
||||
|
||||
Value VertexAccessor::GetProperty(PropertyId prop_id) const {
|
||||
return std::find_if(properties.begin(), properties.end(), [&](auto &pr) { return prop_id == pr.first; })->second;
|
||||
// return ValueToTypedValue(properties[prop_name]);
|
||||
auto it = std::find_if(properties.begin(), properties.end(), [&](auto &pr) { return prop_id == pr.first; });
|
||||
if (it == properties.end()) {
|
||||
return {};
|
||||
}
|
||||
return it->second;
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(readability-convert-member-functions-to-static)
|
||||
Value VertexAccessor::GetProperty(const std::string & /*prop_name*/) const {
|
||||
// TODO(kostasrim) Add string mapping
|
||||
return {};
|
||||
// return ValueToTypedValue(properties[prop_name]);
|
||||
Value VertexAccessor::GetProperty(const std::string &prop_name) const {
|
||||
return GetProperty(manager_->NameToProperty(prop_name));
|
||||
}
|
||||
|
||||
msgs::Vertex VertexAccessor::GetVertex() const { return vertex; }
|
||||
|
@ -11,6 +11,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <map>
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
@ -23,6 +24,10 @@
|
||||
#include "utils/memory.hpp"
|
||||
#include "utils/memory_tracker.hpp"
|
||||
|
||||
namespace memgraph::msgs {
|
||||
class ShardRequestManagerInterface;
|
||||
} // namespace memgraph::msgs
|
||||
|
||||
namespace memgraph::query::v2::accessors {
|
||||
|
||||
using Value = memgraph::msgs::Value;
|
||||
@ -36,7 +41,7 @@ class VertexAccessor;
|
||||
|
||||
class EdgeAccessor final {
|
||||
public:
|
||||
explicit EdgeAccessor(Edge edge);
|
||||
explicit EdgeAccessor(Edge edge, const msgs::ShardRequestManagerInterface *manager);
|
||||
|
||||
[[nodiscard]] EdgeTypeId EdgeType() const;
|
||||
|
||||
@ -64,6 +69,7 @@ class EdgeAccessor final {
|
||||
|
||||
private:
|
||||
Edge edge;
|
||||
const msgs::ShardRequestManagerInterface *manager_;
|
||||
};
|
||||
|
||||
class VertexAccessor final {
|
||||
@ -71,7 +77,11 @@ class VertexAccessor final {
|
||||
using PropertyId = msgs::PropertyId;
|
||||
using Label = msgs::Label;
|
||||
using VertexId = msgs::VertexId;
|
||||
VertexAccessor(Vertex v, std::vector<std::pair<PropertyId, Value>> props);
|
||||
VertexAccessor(Vertex v, std::vector<std::pair<PropertyId, Value>> props,
|
||||
const msgs::ShardRequestManagerInterface *manager);
|
||||
|
||||
VertexAccessor(Vertex v, std::map<PropertyId, Value> &&props, const msgs::ShardRequestManagerInterface *manager);
|
||||
VertexAccessor(Vertex v, const std::map<PropertyId, Value> &props, const msgs::ShardRequestManagerInterface *manager);
|
||||
|
||||
[[nodiscard]] Label PrimaryLabel() const;
|
||||
|
||||
@ -140,6 +150,7 @@ class VertexAccessor final {
|
||||
private:
|
||||
Vertex vertex;
|
||||
std::vector<std::pair<PropertyId, Value>> properties;
|
||||
const msgs::ShardRequestManagerInterface *manager_;
|
||||
};
|
||||
|
||||
// inline VertexAccessor EdgeAccessor::To() const { return VertexAccessor(impl_.ToVertex()); }
|
||||
|
@ -23,6 +23,10 @@
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "storage/v3/view.hpp"
|
||||
|
||||
namespace memgraph::msgs {
|
||||
class ShardRequestManagerInterface;
|
||||
} // namespace memgraph::msgs
|
||||
|
||||
namespace memgraph::query::v2 {
|
||||
|
||||
inline const auto lam = [](const auto &val) { return ValueToTypedValue(val); };
|
||||
@ -32,13 +36,14 @@ class Callable {
|
||||
auto operator()(const memgraph::storage::v3::PropertyValue &val) const {
|
||||
return memgraph::storage::v3::PropertyToTypedValue<TypedValue>(val);
|
||||
};
|
||||
auto operator()(const msgs::Value &val) const { return ValueToTypedValue(val); };
|
||||
auto operator()(const msgs::Value &val, memgraph::msgs::ShardRequestManagerInterface *manager) const {
|
||||
return ValueToTypedValue(val, manager);
|
||||
};
|
||||
};
|
||||
|
||||
} // namespace detail
|
||||
using ExpressionEvaluator =
|
||||
memgraph::expr::ExpressionEvaluator<TypedValue, memgraph::query::v2::EvaluationContext, DbAccessor,
|
||||
storage::v3::View, storage::v3::LabelId, msgs::Value, detail::Callable,
|
||||
memgraph::storage::v3::Error, memgraph::expr::QueryEngineTag>;
|
||||
using ExpressionEvaluator = memgraph::expr::ExpressionEvaluator<
|
||||
TypedValue, memgraph::query::v2::EvaluationContext, memgraph::msgs::ShardRequestManagerInterface, storage::v3::View,
|
||||
storage::v3::LabelId, msgs::Value, detail::Callable, memgraph::storage::v3::Error, memgraph::expr::QueryEngineTag>;
|
||||
|
||||
} // namespace memgraph::query::v2
|
||||
|
@ -98,7 +98,7 @@ struct ExecutionContext {
|
||||
ExecutionStats execution_stats;
|
||||
utils::AsyncTimer timer;
|
||||
msgs::ShardRequestManagerInterface *shard_request_manager{nullptr};
|
||||
IdAllocator edge_ids_alloc;
|
||||
IdAllocator *edge_ids_alloc;
|
||||
};
|
||||
|
||||
static_assert(std::is_move_assignable_v<ExecutionContext>, "ExecutionContext must be move assignable!");
|
||||
|
@ -13,10 +13,11 @@
|
||||
#include "bindings/typed_value.hpp"
|
||||
#include "query/v2/accessors.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
|
||||
namespace memgraph::query::v2 {
|
||||
|
||||
inline TypedValue ValueToTypedValue(const msgs::Value &value) {
|
||||
inline TypedValue ValueToTypedValue(const msgs::Value &value, msgs::ShardRequestManagerInterface *manager) {
|
||||
using Value = msgs::Value;
|
||||
switch (value.type) {
|
||||
case Value::Type::Null:
|
||||
@ -34,7 +35,7 @@ inline TypedValue ValueToTypedValue(const msgs::Value &value) {
|
||||
std::vector<TypedValue> dst;
|
||||
dst.reserve(lst.size());
|
||||
for (const auto &elem : lst) {
|
||||
dst.push_back(ValueToTypedValue(elem));
|
||||
dst.push_back(ValueToTypedValue(elem, manager));
|
||||
}
|
||||
return TypedValue(std::move(dst));
|
||||
}
|
||||
@ -42,14 +43,15 @@ inline TypedValue ValueToTypedValue(const msgs::Value &value) {
|
||||
const auto &value_map = value.map_v;
|
||||
std::map<std::string, TypedValue> dst;
|
||||
for (const auto &[key, val] : value_map) {
|
||||
dst[key] = ValueToTypedValue(val);
|
||||
dst[key] = ValueToTypedValue(val, manager);
|
||||
}
|
||||
return TypedValue(std::move(dst));
|
||||
}
|
||||
case Value::Type::Vertex:
|
||||
return TypedValue(accessors::VertexAccessor(value.vertex_v, {}));
|
||||
return TypedValue(accessors::VertexAccessor(
|
||||
value.vertex_v, std::vector<std::pair<storage::v3::PropertyId, msgs::Value>>{}, manager));
|
||||
case Value::Type::Edge:
|
||||
return TypedValue(accessors::EdgeAccessor(value.edge_v));
|
||||
return TypedValue(accessors::EdgeAccessor(value.edge_v, manager));
|
||||
}
|
||||
throw std::runtime_error("Incorrect type in conversion");
|
||||
}
|
||||
|
@ -22,8 +22,8 @@
|
||||
|
||||
#include "query/v2/bindings/typed_value.hpp"
|
||||
#include "query/v2/conversions.hpp"
|
||||
#include "query/v2/db_accessor.hpp"
|
||||
#include "query/v2/exceptions.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
#include "storage/v3/conversions.hpp"
|
||||
#include "utils/string.hpp"
|
||||
#include "utils/temporal.hpp"
|
||||
|
@ -16,12 +16,15 @@
|
||||
#include <unordered_map>
|
||||
|
||||
#include "query/v2/bindings/typed_value.hpp"
|
||||
#include "query/v2/db_accessor.hpp"
|
||||
#include "storage/v3/view.hpp"
|
||||
#include "utils/memory.hpp"
|
||||
|
||||
namespace memgraph::query::v2 {
|
||||
namespace memgraph::msgs {
|
||||
class ShardRequestManagerInterface;
|
||||
} // namespace memgraph::msgs
|
||||
|
||||
class DbAccessor;
|
||||
namespace memgraph::query::v2 {
|
||||
|
||||
namespace {
|
||||
const char kStartsWith[] = "STARTSWITH";
|
||||
@ -31,7 +34,9 @@ const char kId[] = "ID";
|
||||
} // namespace
|
||||
|
||||
struct FunctionContext {
|
||||
DbAccessor *db_accessor;
|
||||
// TODO(kostasrim) consider optional here. ShardRequestManager does not exist on the storage.
|
||||
// DbAccessor *db_accessor;
|
||||
msgs::ShardRequestManagerInterface *manager;
|
||||
utils::MemoryResource *memory;
|
||||
int64_t timestamp;
|
||||
std::unordered_map<std::string, int64_t> *counters;
|
||||
|
@ -144,7 +144,7 @@ class ReplQueryHandler final : public query::v2::ReplicationQueryHandler {
|
||||
/// @throw QueryRuntimeException if an error ocurred.
|
||||
|
||||
Callback HandleAuthQuery(AuthQuery *auth_query, AuthQueryHandler *auth, const Parameters ¶meters,
|
||||
DbAccessor *db_accessor) {
|
||||
msgs::ShardRequestManagerInterface *manager) {
|
||||
// Empty frame for evaluation of password expression. This is OK since
|
||||
// password should be either null or string literal and it's evaluation
|
||||
// should not depend on frame.
|
||||
@ -155,7 +155,7 @@ Callback HandleAuthQuery(AuthQuery *auth_query, AuthQueryHandler *auth, const Pa
|
||||
// the argument to Callback.
|
||||
evaluation_context.timestamp = QueryTimestamp();
|
||||
evaluation_context.parameters = parameters;
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, db_accessor, storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, manager, storage::v3::View::OLD);
|
||||
|
||||
std::string username = auth_query->user_;
|
||||
std::string rolename = auth_query->role_;
|
||||
@ -313,7 +313,7 @@ Callback HandleAuthQuery(AuthQuery *auth_query, AuthQueryHandler *auth, const Pa
|
||||
}
|
||||
|
||||
Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters ¶meters,
|
||||
InterpreterContext *interpreter_context, DbAccessor *db_accessor,
|
||||
InterpreterContext *interpreter_context, msgs::ShardRequestManagerInterface *manager,
|
||||
std::vector<Notification> *notifications) {
|
||||
expr::Frame<TypedValue> frame(0);
|
||||
SymbolTable symbol_table;
|
||||
@ -322,7 +322,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
|
||||
// the argument to Callback.
|
||||
evaluation_context.timestamp = QueryTimestamp();
|
||||
evaluation_context.parameters = parameters;
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, db_accessor, storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, manager, storage::v3::View::OLD);
|
||||
|
||||
Callback callback;
|
||||
switch (repl_query->action_) {
|
||||
@ -448,7 +448,8 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
|
||||
}
|
||||
}
|
||||
|
||||
Callback HandleSettingQuery(SettingQuery *setting_query, const Parameters ¶meters, DbAccessor *db_accessor) {
|
||||
Callback HandleSettingQuery(SettingQuery *setting_query, const Parameters ¶meters,
|
||||
msgs::ShardRequestManagerInterface *manager) {
|
||||
expr::Frame<TypedValue> frame(0);
|
||||
SymbolTable symbol_table;
|
||||
EvaluationContext evaluation_context;
|
||||
@ -458,7 +459,7 @@ Callback HandleSettingQuery(SettingQuery *setting_query, const Parameters ¶m
|
||||
std::chrono::duration_cast<std::chrono::milliseconds>(std::chrono::system_clock::now().time_since_epoch())
|
||||
.count();
|
||||
evaluation_context.parameters = parameters;
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, db_accessor, storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, manager, storage::v3::View::OLD);
|
||||
|
||||
Callback callback;
|
||||
switch (setting_query->action_) {
|
||||
@ -696,7 +697,7 @@ PullPlan::PullPlan(const std::shared_ptr<CachedPlan> plan, const Parameters &par
|
||||
ctx_.is_shutting_down = &interpreter_context->is_shutting_down;
|
||||
ctx_.is_profile_query = is_profile_query;
|
||||
ctx_.shard_request_manager = shard_request_manager;
|
||||
ctx_.edge_ids_alloc = interpreter_context->edge_ids_alloc;
|
||||
ctx_.edge_ids_alloc = &interpreter_context->edge_ids_alloc;
|
||||
}
|
||||
|
||||
std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
|
||||
@ -886,7 +887,8 @@ PreparedQuery PrepareCypherQuery(ParsedQuery parsed_query, std::map<std::string,
|
||||
EvaluationContext evaluation_context;
|
||||
evaluation_context.timestamp = QueryTimestamp();
|
||||
evaluation_context.parameters = parsed_query.parameters;
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, dba, storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, shard_request_manager,
|
||||
storage::v3::View::OLD);
|
||||
const auto memory_limit =
|
||||
expr::EvaluateMemoryLimit(&evaluator, cypher_query->memory_limit_, cypher_query->memory_scale_);
|
||||
if (memory_limit) {
|
||||
@ -901,7 +903,6 @@ PreparedQuery PrepareCypherQuery(ParsedQuery parsed_query, std::map<std::string,
|
||||
"convert the parsed row values to the appropriate type. This can be done using the built-in "
|
||||
"conversion functions such as ToInteger, ToFloat, ToBoolean etc.");
|
||||
}
|
||||
shard_request_manager->StartTransaction();
|
||||
auto plan = CypherQueryToPlan(
|
||||
parsed_query.stripped_query.hash(), std::move(parsed_query.ast_storage), cypher_query, parsed_query.parameters,
|
||||
parsed_query.is_cacheable ? &interpreter_context->plan_cache : nullptr, shard_request_manager);
|
||||
@ -957,10 +958,10 @@ PreparedQuery PrepareExplainQuery(ParsedQuery parsed_query, std::map<std::string
|
||||
auto *cypher_query = utils::Downcast<CypherQuery>(parsed_inner_query.query);
|
||||
MG_ASSERT(cypher_query, "Cypher grammar should not allow other queries in EXPLAIN");
|
||||
|
||||
auto cypher_query_plan =
|
||||
CypherQueryToPlan(parsed_inner_query.stripped_query.hash(), std::move(parsed_inner_query.ast_storage),
|
||||
cypher_query, parsed_inner_query.parameters,
|
||||
parsed_inner_query.is_cacheable ? &interpreter_context->plan_cache : nullptr, nullptr);
|
||||
auto cypher_query_plan = CypherQueryToPlan(
|
||||
parsed_inner_query.stripped_query.hash(), std::move(parsed_inner_query.ast_storage), cypher_query,
|
||||
parsed_inner_query.parameters, parsed_inner_query.is_cacheable ? &interpreter_context->plan_cache : nullptr,
|
||||
shard_request_manager);
|
||||
|
||||
std::stringstream printed_plan;
|
||||
plan::PrettyPrint(*shard_request_manager, &cypher_query_plan->plan(), &printed_plan);
|
||||
@ -1030,7 +1031,8 @@ PreparedQuery PrepareProfileQuery(ParsedQuery parsed_query, bool in_explicit_tra
|
||||
EvaluationContext evaluation_context;
|
||||
evaluation_context.timestamp = QueryTimestamp();
|
||||
evaluation_context.parameters = parsed_inner_query.parameters;
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, dba, storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context, shard_request_manager,
|
||||
storage::v3::View::OLD);
|
||||
const auto memory_limit =
|
||||
expr::EvaluateMemoryLimit(&evaluator, cypher_query->memory_limit_, cypher_query->memory_scale_);
|
||||
|
||||
@ -1041,7 +1043,7 @@ PreparedQuery PrepareProfileQuery(ParsedQuery parsed_query, bool in_explicit_tra
|
||||
auto rw_type_checker = plan::ReadWriteTypeChecker();
|
||||
rw_type_checker.InferRWType(const_cast<plan::LogicalOperator &>(cypher_query_plan->plan()));
|
||||
|
||||
return PreparedQuery{{"OPERATOR", "ACTUAL HITS", "RELATIVE TIME", "ABSOLUTE TIME"},
|
||||
return PreparedQuery{{"OPERATOR", "ACTUAL HITS", "RELATIVE TIME", "ABSOLUTE TIME", "CUSTOM DATA"},
|
||||
std::move(parsed_query.required_privileges),
|
||||
[plan = std::move(cypher_query_plan), parameters = std::move(parsed_inner_query.parameters),
|
||||
summary, dba, interpreter_context, execution_memory, memory_limit, shard_request_manager,
|
||||
@ -1179,14 +1181,15 @@ PreparedQuery PrepareIndexQuery(ParsedQuery parsed_query, bool in_explicit_trans
|
||||
|
||||
PreparedQuery PrepareAuthQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
|
||||
std::map<std::string, TypedValue> *summary, InterpreterContext *interpreter_context,
|
||||
DbAccessor *dba, utils::MemoryResource *execution_memory) {
|
||||
DbAccessor *dba, utils::MemoryResource *execution_memory,
|
||||
msgs::ShardRequestManagerInterface *manager) {
|
||||
if (in_explicit_transaction) {
|
||||
throw UserModificationInMulticommandTxException();
|
||||
}
|
||||
|
||||
auto *auth_query = utils::Downcast<AuthQuery>(parsed_query.query);
|
||||
|
||||
auto callback = HandleAuthQuery(auth_query, interpreter_context->auth, parsed_query.parameters, dba);
|
||||
auto callback = HandleAuthQuery(auth_query, interpreter_context->auth, parsed_query.parameters, manager);
|
||||
|
||||
SymbolTable symbol_table;
|
||||
std::vector<Symbol> output_symbols;
|
||||
@ -1215,14 +1218,14 @@ PreparedQuery PrepareAuthQuery(ParsedQuery parsed_query, bool in_explicit_transa
|
||||
|
||||
PreparedQuery PrepareReplicationQuery(ParsedQuery parsed_query, const bool in_explicit_transaction,
|
||||
std::vector<Notification> *notifications, InterpreterContext *interpreter_context,
|
||||
DbAccessor *dba) {
|
||||
msgs::ShardRequestManagerInterface *manager) {
|
||||
if (in_explicit_transaction) {
|
||||
throw ReplicationModificationInMulticommandTxException();
|
||||
}
|
||||
|
||||
auto *replication_query = utils::Downcast<ReplicationQuery>(parsed_query.query);
|
||||
auto callback =
|
||||
HandleReplicationQuery(replication_query, parsed_query.parameters, interpreter_context, dba, notifications);
|
||||
HandleReplicationQuery(replication_query, parsed_query.parameters, interpreter_context, manager, notifications);
|
||||
|
||||
return PreparedQuery{callback.header, std::move(parsed_query.required_privileges),
|
||||
[callback_fn = std::move(callback.fn), pull_plan = std::shared_ptr<PullPlanVector>{nullptr}](
|
||||
@ -1310,14 +1313,15 @@ PreparedQuery PrepareCreateSnapshotQuery(ParsedQuery parsed_query, bool in_expli
|
||||
throw SemanticException("CreateSnapshot query is not supported!");
|
||||
}
|
||||
|
||||
PreparedQuery PrepareSettingQuery(ParsedQuery parsed_query, const bool in_explicit_transaction, DbAccessor *dba) {
|
||||
PreparedQuery PrepareSettingQuery(ParsedQuery parsed_query, const bool in_explicit_transaction,
|
||||
msgs::ShardRequestManagerInterface *manager) {
|
||||
if (in_explicit_transaction) {
|
||||
throw SettingConfigInMulticommandTxException{};
|
||||
}
|
||||
|
||||
auto *setting_query = utils::Downcast<SettingQuery>(parsed_query.query);
|
||||
MG_ASSERT(setting_query);
|
||||
auto callback = HandleSettingQuery(setting_query, parsed_query.parameters, dba);
|
||||
auto callback = HandleSettingQuery(setting_query, parsed_query.parameters, manager);
|
||||
|
||||
return PreparedQuery{std::move(callback.header), std::move(parsed_query.required_privileges),
|
||||
[callback_fn = std::move(callback.fn), pull_plan = std::shared_ptr<PullPlanVector>{nullptr}](
|
||||
@ -1511,6 +1515,11 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
|
||||
ParsedQuery parsed_query =
|
||||
ParseQuery(query_string, params, &interpreter_context_->ast_cache, interpreter_context_->config.query);
|
||||
query_execution->summary["parsing_time"] = parsing_timer.Elapsed().count();
|
||||
if (!in_explicit_transaction_ &&
|
||||
(utils::Downcast<CypherQuery>(parsed_query.query) || utils::Downcast<ExplainQuery>(parsed_query.query) ||
|
||||
utils::Downcast<ProfileQuery>(parsed_query.query))) {
|
||||
shard_request_manager_->StartTransaction();
|
||||
}
|
||||
|
||||
utils::Timer planning_timer;
|
||||
PreparedQuery prepared_query;
|
||||
@ -1533,9 +1542,9 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
|
||||
prepared_query = PrepareIndexQuery(std::move(parsed_query), in_explicit_transaction_,
|
||||
&query_execution->notifications, interpreter_context_);
|
||||
} else if (utils::Downcast<AuthQuery>(parsed_query.query)) {
|
||||
prepared_query = PrepareAuthQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->summary,
|
||||
interpreter_context_, &*execution_db_accessor_,
|
||||
&query_execution->execution_memory_with_exception);
|
||||
prepared_query = PrepareAuthQuery(
|
||||
std::move(parsed_query), in_explicit_transaction_, &query_execution->summary, interpreter_context_,
|
||||
&*execution_db_accessor_, &query_execution->execution_memory_with_exception, shard_request_manager_.get());
|
||||
} else if (utils::Downcast<InfoQuery>(parsed_query.query)) {
|
||||
prepared_query = PrepareInfoQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->summary,
|
||||
interpreter_context_, interpreter_context_->db,
|
||||
@ -1546,7 +1555,7 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
|
||||
} else if (utils::Downcast<ReplicationQuery>(parsed_query.query)) {
|
||||
prepared_query =
|
||||
PrepareReplicationQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->notifications,
|
||||
interpreter_context_, &*execution_db_accessor_);
|
||||
interpreter_context_, shard_request_manager_.get());
|
||||
} else if (utils::Downcast<LockPathQuery>(parsed_query.query)) {
|
||||
prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_,
|
||||
&*execution_db_accessor_);
|
||||
@ -1563,7 +1572,8 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
|
||||
prepared_query =
|
||||
PrepareCreateSnapshotQuery(std::move(parsed_query), in_explicit_transaction_, interpreter_context_);
|
||||
} else if (utils::Downcast<SettingQuery>(parsed_query.query)) {
|
||||
prepared_query = PrepareSettingQuery(std::move(parsed_query), in_explicit_transaction_, &*execution_db_accessor_);
|
||||
prepared_query =
|
||||
PrepareSettingQuery(std::move(parsed_query), in_explicit_transaction_, shard_request_manager_.get());
|
||||
} else if (utils::Downcast<VersionQuery>(parsed_query.query)) {
|
||||
prepared_query = PrepareVersionQuery(std::move(parsed_query), in_explicit_transaction_);
|
||||
} else if (utils::Downcast<SchemaQuery>(parsed_query.query)) {
|
||||
|
@ -296,6 +296,8 @@ class Interpreter final {
|
||||
*/
|
||||
void Abort();
|
||||
|
||||
const msgs::ShardRequestManagerInterface *GetShardRequestManager() const { return shard_request_manager_.get(); }
|
||||
|
||||
private:
|
||||
struct QueryExecution {
|
||||
std::optional<PreparedQuery> prepared_query;
|
||||
|
@ -12,8 +12,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <unordered_map>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
#include "storage/v3/property_value.hpp"
|
||||
#include "utils/logging.hpp"
|
||||
@ -32,7 +32,7 @@ struct Parameters {
|
||||
* @param position Token position in query of value.
|
||||
* @param value
|
||||
*/
|
||||
void Add(int position, const storage::v3::PropertyValue &value) { storage_.emplace_back(position, value); }
|
||||
void Add(int position, const storage::v3::PropertyValue &value) { storage_.emplace(position, value); }
|
||||
|
||||
/**
|
||||
* Returns the value found for the given token position.
|
||||
@ -41,23 +41,11 @@ struct Parameters {
|
||||
* @return Value for the given token position.
|
||||
*/
|
||||
const storage::v3::PropertyValue &AtTokenPosition(int position) const {
|
||||
auto found = std::find_if(storage_.begin(), storage_.end(), [&](const auto &a) { return a.first == position; });
|
||||
auto found = storage_.find(position);
|
||||
MG_ASSERT(found != storage_.end(), "Token position must be present in container");
|
||||
return found->second;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the position-th stripped value. Asserts that this
|
||||
* container has at least (position + 1) elements.
|
||||
*
|
||||
* @param position Which stripped param is sought.
|
||||
* @return Token position and value for sought param.
|
||||
*/
|
||||
const std::pair<int, storage::v3::PropertyValue> &At(int position) const {
|
||||
MG_ASSERT(position < static_cast<int>(storage_.size()), "Invalid position");
|
||||
return storage_[position];
|
||||
}
|
||||
|
||||
/** Returns the number of arguments in this container */
|
||||
auto size() const { return storage_.size(); }
|
||||
|
||||
@ -65,7 +53,7 @@ struct Parameters {
|
||||
auto end() const { return storage_.end(); }
|
||||
|
||||
private:
|
||||
std::vector<std::pair<int, storage::v3::PropertyValue>> storage_;
|
||||
std::unordered_map<int, storage::v3::PropertyValue> storage_;
|
||||
};
|
||||
|
||||
} // namespace memgraph::query::v2
|
||||
|
@ -155,7 +155,16 @@ uint64_t ComputeProfilingKey(const T *obj) {
|
||||
|
||||
} // namespace
|
||||
|
||||
#define SCOPED_PROFILE_OP(name) ScopedProfile profile{ComputeProfilingKey(this), name, &context};
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define SCOPED_PROFILE_OP(name) \
|
||||
ScopedProfile profile { ComputeProfilingKey(this), name, &context }
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define SCOPED_CUSTOM_PROFILE(name) \
|
||||
ScopedCustomProfile custom_profile { name, context }
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define SCOPED_REQUEST_WAIT_PROFILE SCOPED_CUSTOM_PROFILE("request_wait")
|
||||
|
||||
class DistributedCreateNodeCursor : public Cursor {
|
||||
public:
|
||||
@ -168,7 +177,11 @@ class DistributedCreateNodeCursor : public Cursor {
|
||||
SCOPED_PROFILE_OP("CreateNode");
|
||||
if (input_cursor_->Pull(frame, context)) {
|
||||
auto &shard_manager = context.shard_request_manager;
|
||||
shard_manager->Request(state_, NodeCreationInfoToRequest(context, frame));
|
||||
{
|
||||
SCOPED_REQUEST_WAIT_PROFILE;
|
||||
shard_manager->Request(state_, NodeCreationInfoToRequest(context, frame));
|
||||
}
|
||||
PlaceNodeOnTheFrame(frame, context);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -179,8 +192,19 @@ class DistributedCreateNodeCursor : public Cursor {
|
||||
|
||||
void Reset() override { state_ = {}; }
|
||||
|
||||
std::vector<msgs::NewVertex> NodeCreationInfoToRequest(ExecutionContext &context, Frame &frame) const {
|
||||
void PlaceNodeOnTheFrame(Frame &frame, ExecutionContext &context) {
|
||||
// TODO(kostasrim) Make this work with batching
|
||||
const auto primary_label = msgs::Label{.id = nodes_info_[0]->labels[0]};
|
||||
msgs::Vertex v{.id = std::make_pair(primary_label, primary_keys_[0])};
|
||||
frame[nodes_info_.front()->symbol] = TypedValue(
|
||||
query::v2::accessors::VertexAccessor(std::move(v), src_vertex_props_[0], context.shard_request_manager));
|
||||
}
|
||||
|
||||
std::vector<msgs::NewVertex> NodeCreationInfoToRequest(ExecutionContext &context, Frame &frame) {
|
||||
std::vector<msgs::NewVertex> requests;
|
||||
// TODO(kostasrim) this assertion should be removed once we support multiple vertex creation
|
||||
MG_ASSERT(nodes_info_.size() == 1);
|
||||
msgs::PrimaryKey pk;
|
||||
for (const auto &node_info : nodes_info_) {
|
||||
msgs::NewVertex rqst;
|
||||
MG_ASSERT(!node_info->labels.empty(), "Cannot determine primary label");
|
||||
@ -188,17 +212,14 @@ class DistributedCreateNodeCursor : public Cursor {
|
||||
// TODO(jbajic) Fix properties not send,
|
||||
// suggestion: ignore distinction between properties and primary keys
|
||||
// since schema validation is done on storage side
|
||||
std::map<msgs::PropertyId, msgs::Value> properties;
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, nullptr,
|
||||
storage::v3::View::NEW);
|
||||
if (const auto *node_info_properties = std::get_if<PropertiesMapList>(&node_info->properties)) {
|
||||
for (const auto &[key, value_expression] : *node_info_properties) {
|
||||
TypedValue val = value_expression->Accept(evaluator);
|
||||
|
||||
if (context.shard_request_manager->IsPrimaryKey(primary_label, key)) {
|
||||
rqst.primary_key.push_back(TypedValueToValue(val));
|
||||
} else {
|
||||
properties[key] = TypedValueToValue(val);
|
||||
pk.push_back(TypedValueToValue(val));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
@ -207,9 +228,8 @@ class DistributedCreateNodeCursor : public Cursor {
|
||||
auto key_str = std::string(key);
|
||||
auto property_id = context.shard_request_manager->NameToProperty(key_str);
|
||||
if (context.shard_request_manager->IsPrimaryKey(primary_label, property_id)) {
|
||||
rqst.primary_key.push_back(storage::v3::TypedValueToValue(value));
|
||||
} else {
|
||||
properties[property_id] = TypedValueToValue(value);
|
||||
rqst.primary_key.push_back(TypedValueToValue(value));
|
||||
pk.push_back(TypedValueToValue(value));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -219,14 +239,18 @@ class DistributedCreateNodeCursor : public Cursor {
|
||||
}
|
||||
// TODO(kostasrim) Copy non primary labels as well
|
||||
rqst.label_ids.push_back(msgs::Label{.id = primary_label});
|
||||
src_vertex_props_.push_back(rqst.properties);
|
||||
requests.push_back(std::move(rqst));
|
||||
}
|
||||
primary_keys_.push_back(std::move(pk));
|
||||
return requests;
|
||||
}
|
||||
|
||||
private:
|
||||
const UniqueCursorPtr input_cursor_;
|
||||
std::vector<const NodeCreationInfo *> nodes_info_;
|
||||
std::vector<std::vector<std::pair<storage::v3::PropertyId, msgs::Value>>> src_vertex_props_;
|
||||
std::vector<msgs::PrimaryKey> primary_keys_;
|
||||
msgs::ExecutionState<msgs::CreateVerticesRequest> state_;
|
||||
};
|
||||
|
||||
@ -361,38 +385,43 @@ class DistributedScanAllAndFilterCursor : public Cursor {
|
||||
|
||||
using VertexAccessor = accessors::VertexAccessor;
|
||||
|
||||
bool MakeRequest(msgs::ShardRequestManagerInterface &shard_manager) {
|
||||
current_batch = shard_manager.Request(request_state_);
|
||||
bool MakeRequest(msgs::ShardRequestManagerInterface &shard_manager, ExecutionContext &context) {
|
||||
{
|
||||
SCOPED_REQUEST_WAIT_PROFILE;
|
||||
current_batch = shard_manager.Request(request_state_);
|
||||
}
|
||||
current_vertex_it = current_batch.begin();
|
||||
return !current_batch.empty();
|
||||
}
|
||||
|
||||
bool Pull(Frame &frame, ExecutionContext &context) override {
|
||||
SCOPED_PROFILE_OP(op_name_);
|
||||
|
||||
auto &shard_manager = *context.shard_request_manager;
|
||||
if (MustAbort(context)) {
|
||||
throw HintedAbortError();
|
||||
}
|
||||
using State = msgs::ExecutionState<msgs::ScanVerticesRequest>;
|
||||
|
||||
if (request_state_.state == State::INITIALIZING) {
|
||||
if (!input_cursor_->Pull(frame, context)) {
|
||||
return false;
|
||||
while (true) {
|
||||
if (MustAbort(context)) {
|
||||
throw HintedAbortError();
|
||||
}
|
||||
}
|
||||
using State = msgs::ExecutionState<msgs::ScanVerticesRequest>;
|
||||
|
||||
request_state_.label = label_.has_value() ? std::make_optional(shard_manager.LabelToName(*label_)) : std::nullopt;
|
||||
if (request_state_.state == State::INITIALIZING) {
|
||||
if (!input_cursor_->Pull(frame, context)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
if (current_vertex_it == current_batch.end()) {
|
||||
if (request_state_.state == State::COMPLETED || !MakeRequest(shard_manager)) {
|
||||
request_state_.label = label_.has_value() ? std::make_optional(shard_manager.LabelToName(*label_)) : std::nullopt;
|
||||
|
||||
if (current_vertex_it == current_batch.end() &&
|
||||
(request_state_.state == State::COMPLETED || !MakeRequest(shard_manager, context))) {
|
||||
ResetExecutionState();
|
||||
return Pull(frame, context);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
frame[output_symbol_] = TypedValue(std::move(*current_vertex_it));
|
||||
++current_vertex_it;
|
||||
return true;
|
||||
frame[output_symbol_] = TypedValue(std::move(*current_vertex_it));
|
||||
++current_vertex_it;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
void Shutdown() override { input_cursor_->Shutdown(); }
|
||||
@ -687,7 +716,7 @@ bool Filter::FilterCursor::Pull(Frame &frame, ExecutionContext &context) {
|
||||
|
||||
// Like all filters, newly set values should not affect filtering of old
|
||||
// nodes and edges.
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.shard_request_manager,
|
||||
storage::v3::View::OLD);
|
||||
while (input_cursor_->Pull(frame, context)) {
|
||||
if (EvaluateFilter(evaluator, self_.expression_)) return true;
|
||||
@ -728,8 +757,8 @@ bool Produce::ProduceCursor::Pull(Frame &frame, ExecutionContext &context) {
|
||||
|
||||
if (input_cursor_->Pull(frame, context)) {
|
||||
// Produce should always yield the latest results.
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::NEW);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::NEW);
|
||||
for (auto named_expr : self_.named_expressions_) named_expr->Accept(evaluator);
|
||||
|
||||
return true;
|
||||
@ -1149,8 +1178,8 @@ class AggregateCursor : public Cursor {
|
||||
* aggregation results, and not on the number of inputs.
|
||||
*/
|
||||
void ProcessAll(Frame *frame, ExecutionContext *context) {
|
||||
ExpressionEvaluator evaluator(frame, context->symbol_table, context->evaluation_context, context->db_accessor,
|
||||
storage::v3::View::NEW);
|
||||
ExpressionEvaluator evaluator(frame, context->symbol_table, context->evaluation_context,
|
||||
context->shard_request_manager, storage::v3::View::NEW);
|
||||
while (input_cursor_->Pull(*frame, *context)) {
|
||||
ProcessOne(*frame, &evaluator);
|
||||
}
|
||||
@ -1370,8 +1399,8 @@ bool Skip::SkipCursor::Pull(Frame &frame, ExecutionContext &context) {
|
||||
// First successful pull from the input, evaluate the skip expression.
|
||||
// The skip expression doesn't contain identifiers so graph view
|
||||
// parameter is not important.
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::OLD);
|
||||
TypedValue to_skip = self_.expression_->Accept(evaluator);
|
||||
if (to_skip.type() != TypedValue::Type::Int)
|
||||
throw QueryRuntimeException("Number of elements to skip must be an integer.");
|
||||
@ -1425,8 +1454,8 @@ bool Limit::LimitCursor::Pull(Frame &frame, ExecutionContext &context) {
|
||||
if (limit_ == -1) {
|
||||
// Limit expression doesn't contain identifiers so graph view is not
|
||||
// important.
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::OLD);
|
||||
TypedValue limit = self_.expression_->Accept(evaluator);
|
||||
if (limit.type() != TypedValue::Type::Int)
|
||||
throw QueryRuntimeException("Limit on number of returned elements must be an integer.");
|
||||
@ -1481,8 +1510,8 @@ class OrderByCursor : public Cursor {
|
||||
SCOPED_PROFILE_OP("OrderBy");
|
||||
|
||||
if (!did_pull_all_) {
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::OLD);
|
||||
auto *mem = cache_.get_allocator().GetMemoryResource();
|
||||
while (input_cursor_->Pull(frame, context)) {
|
||||
// collect the order_by elements
|
||||
@ -1739,8 +1768,8 @@ class UnwindCursor : public Cursor {
|
||||
if (!input_cursor_->Pull(frame, context)) return false;
|
||||
|
||||
// successful pull from input, initialize value and iterator
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::OLD);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::OLD);
|
||||
TypedValue input_value = self_.input_expression_->Accept(evaluator);
|
||||
if (input_value.type() != TypedValue::Type::List)
|
||||
throw QueryRuntimeException("Argument of UNWIND must be a list, but '{}' was provided.", input_value.type());
|
||||
@ -2217,7 +2246,7 @@ class LoadCsvCursor : public Cursor {
|
||||
// self_->delimiter_, and self_->quote_ earlier (say, in the interpreter.cpp)
|
||||
// without massacring the code even worse than I did here
|
||||
if (UNLIKELY(!reader_)) {
|
||||
reader_ = MakeReader(&context.evaluation_context);
|
||||
reader_ = MakeReader(context);
|
||||
}
|
||||
|
||||
bool input_pulled = input_cursor_->Pull(frame, context);
|
||||
@ -2246,11 +2275,12 @@ class LoadCsvCursor : public Cursor {
|
||||
void Shutdown() override { input_cursor_->Shutdown(); }
|
||||
|
||||
private:
|
||||
csv::Reader MakeReader(EvaluationContext *eval_context) {
|
||||
csv::Reader MakeReader(ExecutionContext &context) {
|
||||
auto &eval_context = context.evaluation_context;
|
||||
Frame frame(0);
|
||||
SymbolTable symbol_table;
|
||||
DbAccessor *dba = nullptr;
|
||||
auto evaluator = ExpressionEvaluator(&frame, symbol_table, *eval_context, dba, storage::v3::View::OLD);
|
||||
auto evaluator =
|
||||
ExpressionEvaluator(&frame, symbol_table, eval_context, context.shard_request_manager, storage::v3::View::OLD);
|
||||
|
||||
auto maybe_file = ToOptionalString(&evaluator, self_->file_);
|
||||
auto maybe_delim = ToOptionalString(&evaluator, self_->delimiter_);
|
||||
@ -2287,8 +2317,8 @@ class ForeachCursor : public Cursor {
|
||||
return false;
|
||||
}
|
||||
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.db_accessor,
|
||||
storage::v3::View::NEW);
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context,
|
||||
context.shard_request_manager, storage::v3::View::NEW);
|
||||
TypedValue expr_result = expression->Accept(evaluator);
|
||||
|
||||
if (expr_result.IsNull()) {
|
||||
@ -2366,7 +2396,10 @@ class DistributedCreateExpandCursor : public Cursor {
|
||||
}
|
||||
auto &shard_manager = context.shard_request_manager;
|
||||
ResetExecutionState();
|
||||
shard_manager->Request(state_, ExpandCreationInfoToRequest(context, frame));
|
||||
{
|
||||
SCOPED_REQUEST_WAIT_PROFILE;
|
||||
shard_manager->Request(state_, ExpandCreationInfoToRequest(context, frame));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -2389,7 +2422,7 @@ class DistributedCreateExpandCursor : public Cursor {
|
||||
std::vector<msgs::NewExpand> ExpandCreationInfoToRequest(ExecutionContext &context, Frame &frame) const {
|
||||
std::vector<msgs::NewExpand> edge_requests;
|
||||
for (const auto &edge_info : std::vector{self_.edge_info_}) {
|
||||
msgs::NewExpand request{.id = {context.edge_ids_alloc.AllocateId()}};
|
||||
msgs::NewExpand request{.id = {context.edge_ids_alloc->AllocateId()}};
|
||||
ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, nullptr,
|
||||
storage::v3::View::NEW);
|
||||
request.type = {edge_info.edge_type};
|
||||
@ -2425,13 +2458,13 @@ class DistributedCreateExpandCursor : public Cursor {
|
||||
std::invoke([&]() {
|
||||
switch (edge_info.direction) {
|
||||
case EdgeAtom::Direction::IN: {
|
||||
set_vertex(v1, request.src_vertex);
|
||||
set_vertex(v2, request.dest_vertex);
|
||||
set_vertex(v2, request.src_vertex);
|
||||
set_vertex(v1, request.dest_vertex);
|
||||
break;
|
||||
}
|
||||
case EdgeAtom::Direction::OUT: {
|
||||
set_vertex(v1, request.dest_vertex);
|
||||
set_vertex(v2, request.src_vertex);
|
||||
set_vertex(v1, request.src_vertex);
|
||||
set_vertex(v2, request.dest_vertex);
|
||||
break;
|
||||
}
|
||||
case EdgeAtom::Direction::BOTH:
|
||||
@ -2458,15 +2491,52 @@ class DistributedExpandCursor : public Cursor {
|
||||
: self_(self),
|
||||
input_cursor_(self.input_->MakeCursor(mem)),
|
||||
current_in_edge_it_(current_in_edges_.begin()),
|
||||
current_out_edge_it_(current_out_edges_.begin()) {
|
||||
if (self_.common_.existing_node) {
|
||||
throw QueryRuntimeException("Cannot use existing node with DistributedExpandOne cursor!");
|
||||
}
|
||||
}
|
||||
current_out_edge_it_(current_out_edges_.begin()) {}
|
||||
|
||||
using VertexAccessor = accessors::VertexAccessor;
|
||||
using EdgeAccessor = accessors::EdgeAccessor;
|
||||
|
||||
static constexpr auto DirectionToMsgsDirection(const auto direction) {
|
||||
switch (direction) {
|
||||
case EdgeAtom::Direction::IN:
|
||||
return msgs::EdgeDirection::IN;
|
||||
case EdgeAtom::Direction::OUT:
|
||||
return msgs::EdgeDirection::OUT;
|
||||
case EdgeAtom::Direction::BOTH:
|
||||
return msgs::EdgeDirection::BOTH;
|
||||
}
|
||||
};
|
||||
|
||||
void PullDstVertex(Frame &frame, ExecutionContext &context, const EdgeAtom::Direction direction) {
|
||||
if (self_.common_.existing_node) {
|
||||
return;
|
||||
}
|
||||
MG_ASSERT(direction != EdgeAtom::Direction::BOTH);
|
||||
const auto &edge = frame[self_.common_.edge_symbol].ValueEdge();
|
||||
static constexpr auto get_dst_vertex = [](const EdgeAccessor &edge,
|
||||
const EdgeAtom::Direction direction) -> msgs::VertexId {
|
||||
switch (direction) {
|
||||
case EdgeAtom::Direction::IN:
|
||||
return edge.From().Id();
|
||||
case EdgeAtom::Direction::OUT:
|
||||
return edge.To().Id();
|
||||
case EdgeAtom::Direction::BOTH:
|
||||
throw std::runtime_error("EdgeDirection Both not implemented");
|
||||
}
|
||||
};
|
||||
msgs::ExpandOneRequest request;
|
||||
// to not fetch any properties of the edges
|
||||
request.edge_properties.emplace();
|
||||
request.src_vertices.push_back(get_dst_vertex(edge, direction));
|
||||
request.direction = (direction == EdgeAtom::Direction::IN) ? msgs::EdgeDirection::OUT : msgs::EdgeDirection::IN;
|
||||
msgs::ExecutionState<msgs::ExpandOneRequest> request_state;
|
||||
auto result_rows = context.shard_request_manager->Request(request_state, std::move(request));
|
||||
MG_ASSERT(result_rows.size() == 1);
|
||||
auto &result_row = result_rows.front();
|
||||
frame[self_.common_.node_symbol] = accessors::VertexAccessor(
|
||||
msgs::Vertex{result_row.src_vertex}, result_row.src_vertex_properties, context.shard_request_manager);
|
||||
}
|
||||
|
||||
bool InitEdges(Frame &frame, ExecutionContext &context) {
|
||||
// Input Vertex could be null if it is created by a failed optional match. In
|
||||
// those cases we skip that input pull and continue with the next.
|
||||
@ -2480,44 +2550,45 @@ class DistributedExpandCursor : public Cursor {
|
||||
|
||||
ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
|
||||
auto &vertex = vertex_value.ValueVertex();
|
||||
static constexpr auto direction_to_msgs_direction = [](const EdgeAtom::Direction direction) {
|
||||
switch (direction) {
|
||||
case EdgeAtom::Direction::IN:
|
||||
return msgs::EdgeDirection::IN;
|
||||
case EdgeAtom::Direction::OUT:
|
||||
return msgs::EdgeDirection::OUT;
|
||||
case EdgeAtom::Direction::BOTH:
|
||||
return msgs::EdgeDirection::BOTH;
|
||||
}
|
||||
};
|
||||
|
||||
msgs::ExpandOneRequest request;
|
||||
request.direction = direction_to_msgs_direction(self_.common_.direction);
|
||||
request.direction = DirectionToMsgsDirection(self_.common_.direction);
|
||||
// to not fetch any properties of the edges
|
||||
request.edge_properties.emplace();
|
||||
request.src_vertices.push_back(vertex.Id());
|
||||
msgs::ExecutionState<msgs::ExpandOneRequest> request_state;
|
||||
auto result_rows = context.shard_request_manager->Request(request_state, std::move(request));
|
||||
auto result_rows = std::invoke([&context, &request_state, &request]() mutable {
|
||||
SCOPED_REQUEST_WAIT_PROFILE;
|
||||
return context.shard_request_manager->Request(request_state, std::move(request));
|
||||
});
|
||||
MG_ASSERT(result_rows.size() == 1);
|
||||
auto &result_row = result_rows.front();
|
||||
|
||||
const auto convert_edges = [&vertex](
|
||||
if (self_.common_.existing_node) {
|
||||
const auto &node = frame[self_.common_.node_symbol].ValueVertex().Id();
|
||||
auto &in = result_row.in_edges_with_specific_properties;
|
||||
std::erase_if(in, [&node](auto &edge) { return edge.other_end != node; });
|
||||
auto &out = result_row.out_edges_with_specific_properties;
|
||||
std::erase_if(out, [&node](auto &edge) { return edge.other_end != node; });
|
||||
}
|
||||
|
||||
const auto convert_edges = [&vertex, &context](
|
||||
std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties> &&edge_messages,
|
||||
const EdgeAtom::Direction direction) {
|
||||
std::vector<EdgeAccessor> edge_accessors;
|
||||
edge_accessors.reserve(edge_messages.size());
|
||||
|
||||
switch (direction) {
|
||||
case EdgeAtom::Direction::IN: {
|
||||
for (auto &edge : edge_messages) {
|
||||
edge_accessors.emplace_back(
|
||||
msgs::Edge{std::move(edge.other_end), vertex.Id(), {}, {edge.gid}, edge.type});
|
||||
edge_accessors.emplace_back(msgs::Edge{std::move(edge.other_end), vertex.Id(), {}, {edge.gid}, edge.type},
|
||||
context.shard_request_manager);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case EdgeAtom::Direction::OUT: {
|
||||
for (auto &edge : edge_messages) {
|
||||
edge_accessors.emplace_back(
|
||||
msgs::Edge{vertex.Id(), std::move(edge.other_end), {}, {edge.gid}, edge.type});
|
||||
edge_accessors.emplace_back(msgs::Edge{vertex.Id(), std::move(edge.other_end), {}, {edge.gid}, edge.type},
|
||||
context.shard_request_manager);
|
||||
}
|
||||
break;
|
||||
}
|
||||
@ -2527,12 +2598,13 @@ class DistributedExpandCursor : public Cursor {
|
||||
}
|
||||
return edge_accessors;
|
||||
};
|
||||
|
||||
current_in_edges_ =
|
||||
convert_edges(std::move(result_row.in_edges_with_specific_properties), EdgeAtom::Direction::IN);
|
||||
current_in_edge_it_ = current_in_edges_.begin();
|
||||
current_in_edges_ =
|
||||
convert_edges(std::move(result_row.in_edges_with_specific_properties), EdgeAtom::Direction::OUT);
|
||||
current_in_edge_it_ = current_in_edges_.begin();
|
||||
current_out_edges_ =
|
||||
convert_edges(std::move(result_row.out_edges_with_specific_properties), EdgeAtom::Direction::OUT);
|
||||
current_out_edge_it_ = current_out_edges_.begin();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@ -2540,19 +2612,6 @@ class DistributedExpandCursor : public Cursor {
|
||||
bool Pull(Frame &frame, ExecutionContext &context) override {
|
||||
SCOPED_PROFILE_OP("DistributedExpand");
|
||||
// A helper function for expanding a node from an edge.
|
||||
auto pull_node = [this, &frame](const EdgeAccessor &new_edge, EdgeAtom::Direction direction) {
|
||||
if (self_.common_.existing_node) return;
|
||||
switch (direction) {
|
||||
case EdgeAtom::Direction::IN:
|
||||
frame[self_.common_.node_symbol] = new_edge.From();
|
||||
break;
|
||||
case EdgeAtom::Direction::OUT:
|
||||
frame[self_.common_.node_symbol] = new_edge.To();
|
||||
break;
|
||||
case EdgeAtom::Direction::BOTH:
|
||||
LOG_FATAL("Must indicate exact expansion direction here");
|
||||
}
|
||||
};
|
||||
|
||||
while (true) {
|
||||
if (MustAbort(context)) throw HintedAbortError();
|
||||
@ -2561,7 +2620,7 @@ class DistributedExpandCursor : public Cursor {
|
||||
auto &edge = *current_in_edge_it_;
|
||||
++current_in_edge_it_;
|
||||
frame[self_.common_.edge_symbol] = edge;
|
||||
pull_node(edge, EdgeAtom::Direction::IN);
|
||||
PullDstVertex(frame, context, EdgeAtom::Direction::IN);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -2573,7 +2632,7 @@ class DistributedExpandCursor : public Cursor {
|
||||
continue;
|
||||
};
|
||||
frame[self_.common_.edge_symbol] = edge;
|
||||
pull_node(edge, EdgeAtom::Direction::OUT);
|
||||
PullDstVertex(frame, context, EdgeAtom::Direction::OUT);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -24,18 +24,18 @@ namespace memgraph::query::v2::plan {
|
||||
|
||||
namespace {
|
||||
|
||||
unsigned long long IndividualCycles(const ProfilingStats &cumulative_stats) {
|
||||
uint64_t IndividualCycles(const ProfilingStats &cumulative_stats) {
|
||||
return cumulative_stats.num_cycles - std::accumulate(cumulative_stats.children.begin(),
|
||||
cumulative_stats.children.end(), 0ULL,
|
||||
[](auto acc, auto &stats) { return acc + stats.num_cycles; });
|
||||
}
|
||||
|
||||
double RelativeTime(unsigned long long num_cycles, unsigned long long total_cycles) {
|
||||
return static_cast<double>(num_cycles) / total_cycles;
|
||||
double RelativeTime(const uint64_t num_cycles, const uint64_t total_cycles) {
|
||||
return static_cast<double>(num_cycles) / static_cast<double>(total_cycles);
|
||||
}
|
||||
|
||||
double AbsoluteTime(unsigned long long num_cycles, unsigned long long total_cycles,
|
||||
std::chrono::duration<double> total_time) {
|
||||
double AbsoluteTime(const uint64_t num_cycles, const uint64_t total_cycles,
|
||||
const std::chrono::duration<double> total_time) {
|
||||
return (RelativeTime(num_cycles, total_cycles) * static_cast<std::chrono::duration<double, std::milli>>(total_time))
|
||||
.count();
|
||||
}
|
||||
@ -50,26 +50,29 @@ namespace {
|
||||
|
||||
class ProfilingStatsToTableHelper {
|
||||
public:
|
||||
ProfilingStatsToTableHelper(unsigned long long total_cycles, std::chrono::duration<double> total_time)
|
||||
ProfilingStatsToTableHelper(uint64_t total_cycles, std::chrono::duration<double> total_time)
|
||||
: total_cycles_(total_cycles), total_time_(total_time) {}
|
||||
|
||||
void Output(const ProfilingStats &cumulative_stats) {
|
||||
auto cycles = IndividualCycles(cumulative_stats);
|
||||
auto custom_data_copy = cumulative_stats.custom_data;
|
||||
ConvertCyclesToTime(custom_data_copy);
|
||||
|
||||
rows_.emplace_back(std::vector<TypedValue>{
|
||||
TypedValue(FormatOperator(cumulative_stats.name)), TypedValue(cumulative_stats.actual_hits),
|
||||
TypedValue(FormatRelativeTime(cycles)), TypedValue(FormatAbsoluteTime(cycles))});
|
||||
rows_.emplace_back(
|
||||
std::vector<TypedValue>{TypedValue(FormatOperator(cumulative_stats.name)),
|
||||
TypedValue(cumulative_stats.actual_hits), TypedValue(FormatRelativeTime(cycles)),
|
||||
TypedValue(FormatAbsoluteTime(cycles)), TypedValue(custom_data_copy.dump())});
|
||||
|
||||
for (size_t i = 1; i < cumulative_stats.children.size(); ++i) {
|
||||
Branch(cumulative_stats.children[i]);
|
||||
}
|
||||
|
||||
if (cumulative_stats.children.size() >= 1) {
|
||||
if (!cumulative_stats.children.empty()) {
|
||||
Output(cumulative_stats.children[0]);
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<std::vector<TypedValue>> rows() { return rows_; }
|
||||
std::vector<std::vector<TypedValue>> rows() const { return rows_; }
|
||||
|
||||
private:
|
||||
void Branch(const ProfilingStats &cumulative_stats) {
|
||||
@ -80,7 +83,28 @@ class ProfilingStatsToTableHelper {
|
||||
--depth_;
|
||||
}
|
||||
|
||||
std::string Format(const char *str) {
|
||||
double AbsoluteTime(const uint64_t cycles) const { return plan::AbsoluteTime(cycles, total_cycles_, total_time_); }
|
||||
|
||||
double RelativeTime(const uint64_t cycles) const { return plan::RelativeTime(cycles, total_cycles_); }
|
||||
|
||||
void ConvertCyclesToTime(nlohmann::json &custom_data) const {
|
||||
const auto convert_cycles_in_json = [this](nlohmann::json &json) {
|
||||
if (!json.is_object()) {
|
||||
return;
|
||||
}
|
||||
if (auto it = json.find(ProfilingStats::kNumCycles); it != json.end()) {
|
||||
auto num_cycles = it.value().get<uint64_t>();
|
||||
json[ProfilingStats::kAbsoluteTime] = AbsoluteTime(num_cycles);
|
||||
json[ProfilingStats::kRelativeTime] = RelativeTime(num_cycles);
|
||||
}
|
||||
};
|
||||
|
||||
for (auto &json : custom_data) {
|
||||
convert_cycles_in_json(json);
|
||||
}
|
||||
}
|
||||
|
||||
std::string Format(const char *str) const {
|
||||
std::ostringstream ss;
|
||||
for (int64_t i = 0; i < depth_; ++i) {
|
||||
ss << "| ";
|
||||
@ -89,21 +113,21 @@ class ProfilingStatsToTableHelper {
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
std::string Format(const std::string &str) { return Format(str.c_str()); }
|
||||
std::string Format(const std::string &str) const { return Format(str.c_str()); }
|
||||
|
||||
std::string FormatOperator(const char *str) { return Format(std::string("* ") + str); }
|
||||
std::string FormatOperator(const char *str) const { return Format(std::string("* ") + str); }
|
||||
|
||||
std::string FormatRelativeTime(unsigned long long num_cycles) {
|
||||
return fmt::format("{: 10.6f} %", RelativeTime(num_cycles, total_cycles_) * 100);
|
||||
std::string FormatRelativeTime(uint64_t num_cycles) const {
|
||||
return fmt::format("{: 10.6f} %", RelativeTime(num_cycles) * 100);
|
||||
}
|
||||
|
||||
std::string FormatAbsoluteTime(unsigned long long num_cycles) {
|
||||
return fmt::format("{: 10.6f} ms", AbsoluteTime(num_cycles, total_cycles_, total_time_));
|
||||
std::string FormatAbsoluteTime(uint64_t num_cycles) const {
|
||||
return fmt::format("{: 10.6f} ms", AbsoluteTime(num_cycles));
|
||||
}
|
||||
|
||||
int64_t depth_{0};
|
||||
std::vector<std::vector<TypedValue>> rows_;
|
||||
unsigned long long total_cycles_;
|
||||
uint64_t total_cycles_;
|
||||
std::chrono::duration<double> total_time_;
|
||||
};
|
||||
|
||||
@ -126,7 +150,7 @@ class ProfilingStatsToJsonHelper {
|
||||
using json = nlohmann::json;
|
||||
|
||||
public:
|
||||
ProfilingStatsToJsonHelper(unsigned long long total_cycles, std::chrono::duration<double> total_time)
|
||||
ProfilingStatsToJsonHelper(uint64_t total_cycles, std::chrono::duration<double> total_time)
|
||||
: total_cycles_(total_cycles), total_time_(total_time) {}
|
||||
|
||||
void Output(const ProfilingStats &cumulative_stats) { return Output(cumulative_stats, &json_); }
|
||||
@ -151,7 +175,7 @@ class ProfilingStatsToJsonHelper {
|
||||
}
|
||||
|
||||
json json_;
|
||||
unsigned long long total_cycles_;
|
||||
uint64_t total_cycles_;
|
||||
std::chrono::duration<double> total_time_;
|
||||
};
|
||||
|
||||
|
@ -26,10 +26,16 @@ namespace plan {
|
||||
* Stores profiling statistics for a single logical operator.
|
||||
*/
|
||||
struct ProfilingStats {
|
||||
static constexpr std::string_view kNumCycles{"num_cycles"};
|
||||
static constexpr std::string_view kRelativeTime{"relative_time"};
|
||||
static constexpr std::string_view kAbsoluteTime{"absolute_time"};
|
||||
static constexpr std::string_view kActualHits{"actual_hits"};
|
||||
|
||||
int64_t actual_hits{0};
|
||||
unsigned long long num_cycles{0};
|
||||
uint64_t num_cycles{0};
|
||||
uint64_t key{0};
|
||||
const char *name{nullptr};
|
||||
nlohmann::json custom_data;
|
||||
// TODO: This should use the allocator for query execution
|
||||
std::vector<ProfilingStats> children;
|
||||
};
|
||||
|
@ -13,6 +13,8 @@
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
#include <json/json.hpp>
|
||||
|
||||
#include "query/v2/context.hpp"
|
||||
#include "query/v2/plan/profile.hpp"
|
||||
#include "utils/likely.hpp"
|
||||
@ -20,6 +22,43 @@
|
||||
|
||||
namespace memgraph::query::v2::plan {
|
||||
|
||||
class ScopedCustomProfile {
|
||||
public:
|
||||
explicit ScopedCustomProfile(const std::string_view custom_data_name, ExecutionContext &context)
|
||||
: custom_data_name_(custom_data_name), start_time_{utils::ReadTSC()}, context_{&context} {}
|
||||
|
||||
ScopedCustomProfile(const ScopedCustomProfile &) = delete;
|
||||
ScopedCustomProfile(ScopedCustomProfile &&) = delete;
|
||||
ScopedCustomProfile &operator=(const ScopedCustomProfile &) = delete;
|
||||
ScopedCustomProfile &operator=(ScopedCustomProfile &&) = delete;
|
||||
|
||||
// If an exception is thrown in any of these functions that signals a problem that is much bigger than we could handle
|
||||
// it here, thus we don't attempt to handle it.
|
||||
// NOLINTNEXTLINE(bugprone-exception-escape)
|
||||
~ScopedCustomProfile() {
|
||||
if (nullptr != context_->stats_root) [[unlikely]] {
|
||||
auto &custom_data = context_->stats_root->custom_data[custom_data_name_];
|
||||
if (!custom_data.is_object()) {
|
||||
custom_data = nlohmann::json::object();
|
||||
}
|
||||
const auto elapsed = utils::ReadTSC() - start_time_;
|
||||
IncreaseCustomData(custom_data, ProfilingStats::kNumCycles, elapsed);
|
||||
IncreaseCustomData(custom_data, ProfilingStats::kActualHits, 1);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
static void IncreaseCustomData(nlohmann::json &custom_data, const std::string_view key, const uint64_t increment) {
|
||||
auto &json_data = custom_data[key];
|
||||
const auto numerical_data = json_data.is_null() ? 0 : json_data.get<uint64_t>();
|
||||
json_data = numerical_data + increment;
|
||||
}
|
||||
|
||||
std::string_view custom_data_name_;
|
||||
uint64_t start_time_;
|
||||
ExecutionContext *context_;
|
||||
};
|
||||
|
||||
/**
|
||||
* A RAII class used for profiling logical operators. Instances of this class
|
||||
* update the profiling data stored within the `ExecutionContext` object and build
|
||||
@ -29,7 +68,7 @@ namespace memgraph::query::v2::plan {
|
||||
class ScopedProfile {
|
||||
public:
|
||||
ScopedProfile(uint64_t key, const char *name, query::v2::ExecutionContext *context) noexcept : context_(context) {
|
||||
if (UNLIKELY(context_->is_profile_query)) {
|
||||
if (IsProfiling()) [[unlikely]] {
|
||||
root_ = context_->stats_root;
|
||||
|
||||
// Are we the root logical operator?
|
||||
@ -60,8 +99,13 @@ class ScopedProfile {
|
||||
}
|
||||
}
|
||||
|
||||
ScopedProfile(const ScopedProfile &) = delete;
|
||||
ScopedProfile(ScopedProfile &&) = delete;
|
||||
ScopedProfile &operator=(const ScopedProfile &) = delete;
|
||||
ScopedProfile &operator=(ScopedProfile &&) = delete;
|
||||
|
||||
~ScopedProfile() noexcept {
|
||||
if (UNLIKELY(context_->is_profile_query)) {
|
||||
if (IsProfiling()) [[unlikely]] {
|
||||
stats_->num_cycles += utils::ReadTSC() - start_time_;
|
||||
|
||||
// Restore the old root ("pop")
|
||||
@ -70,10 +114,12 @@ class ScopedProfile {
|
||||
}
|
||||
|
||||
private:
|
||||
[[nodiscard]] bool IsProfiling() const { return context_->is_profile_query; }
|
||||
|
||||
query::v2::ExecutionContext *context_;
|
||||
ProfilingStats *root_{nullptr};
|
||||
ProfilingStats *stats_{nullptr};
|
||||
unsigned long long start_time_{0};
|
||||
uint64_t start_time_{0};
|
||||
};
|
||||
|
||||
} // namespace memgraph::query::v2::plan
|
||||
|
@ -387,7 +387,6 @@ struct GetPropertiesResponse {
|
||||
enum class EdgeDirection : uint8_t { OUT = 1, IN = 2, BOTH = 3 };
|
||||
|
||||
struct ExpandOneRequest {
|
||||
// TODO(antaljanosbenjamin): Filtering based on the id of the other end of the edge?
|
||||
Hlc transaction_id;
|
||||
std::vector<VertexId> src_vertices;
|
||||
// return types that type is in this list
|
||||
|
@ -171,6 +171,7 @@ class ShardRequestManager : public ShardRequestManagerInterface {
|
||||
|
||||
if (hlc_response.fresher_shard_map) {
|
||||
shards_map_ = hlc_response.fresher_shard_map.value();
|
||||
SetUpNameIdMappers();
|
||||
}
|
||||
}
|
||||
|
||||
@ -186,6 +187,7 @@ class ShardRequestManager : public ShardRequestManagerInterface {
|
||||
|
||||
if (hlc_response.fresher_shard_map) {
|
||||
shards_map_ = hlc_response.fresher_shard_map.value();
|
||||
SetUpNameIdMappers();
|
||||
}
|
||||
auto commit_timestamp = hlc_response.new_hlc;
|
||||
|
||||
@ -223,14 +225,14 @@ class ShardRequestManager : public ShardRequestManagerInterface {
|
||||
return shards_map_.GetLabelId(name).value();
|
||||
}
|
||||
|
||||
const std::string &PropertyToName(memgraph::storage::v3::PropertyId prop) const override {
|
||||
return shards_map_.GetPropertyName(prop);
|
||||
const std::string &PropertyToName(memgraph::storage::v3::PropertyId id) const override {
|
||||
return properties_.IdToName(id.AsUint());
|
||||
}
|
||||
const std::string &LabelToName(memgraph::storage::v3::LabelId label) const override {
|
||||
return shards_map_.GetLabelName(label);
|
||||
const std::string &LabelToName(memgraph::storage::v3::LabelId id) const override {
|
||||
return labels_.IdToName(id.AsUint());
|
||||
}
|
||||
const std::string &EdgeTypeToName(memgraph::storage::v3::EdgeTypeId type) const override {
|
||||
return shards_map_.GetEdgeTypeName(type);
|
||||
const std::string &EdgeTypeToName(memgraph::storage::v3::EdgeTypeId id) const override {
|
||||
return edge_types_.IdToName(id.AsUint());
|
||||
}
|
||||
|
||||
bool IsPrimaryKey(LabelId primary_label, PropertyId property) const override {
|
||||
@ -358,7 +360,7 @@ class ShardRequestManager : public ShardRequestManagerInterface {
|
||||
std::vector<VertexAccessor> accessors;
|
||||
for (auto &response : responses) {
|
||||
for (auto &result_row : response.results) {
|
||||
accessors.emplace_back(VertexAccessor(std::move(result_row.vertex), std::move(result_row.props)));
|
||||
accessors.emplace_back(VertexAccessor(std::move(result_row.vertex), std::move(result_row.props), this));
|
||||
}
|
||||
}
|
||||
return accessors;
|
||||
@ -697,7 +699,28 @@ class ShardRequestManager : public ShardRequestManagerInterface {
|
||||
}
|
||||
}
|
||||
|
||||
void SetUpNameIdMappers() {
|
||||
std::unordered_map<uint64_t, std::string> id_to_name;
|
||||
for (const auto &[name, id] : shards_map_.labels) {
|
||||
id_to_name.emplace(id.AsUint(), name);
|
||||
}
|
||||
labels_.StoreMapping(std::move(id_to_name));
|
||||
id_to_name.clear();
|
||||
for (const auto &[name, id] : shards_map_.properties) {
|
||||
id_to_name.emplace(id.AsUint(), name);
|
||||
}
|
||||
properties_.StoreMapping(std::move(id_to_name));
|
||||
id_to_name.clear();
|
||||
for (const auto &[name, id] : shards_map_.edge_types) {
|
||||
id_to_name.emplace(id.AsUint(), name);
|
||||
}
|
||||
edge_types_.StoreMapping(std::move(id_to_name));
|
||||
}
|
||||
|
||||
ShardMap shards_map_;
|
||||
storage::v3::NameIdMapper properties_;
|
||||
storage::v3::NameIdMapper edge_types_;
|
||||
storage::v3::NameIdMapper labels_;
|
||||
CoordinatorClient coord_cli_;
|
||||
RsmStorageClientManager<StorageClient> storage_cli_manager_;
|
||||
memgraph::io::Io<TTransport> io_;
|
||||
|
@ -42,7 +42,7 @@ struct Parameters {
|
||||
* @param position Token position in query of value.
|
||||
* @param value
|
||||
*/
|
||||
void Add(int position, const storage::v3::PropertyValue &value) { storage_.emplace_back(position, value); }
|
||||
void Add(int position, const storage::v3::PropertyValue &value) { storage_.emplace(position, value); }
|
||||
|
||||
/**
|
||||
* Returns the value found for the given token position.
|
||||
@ -51,23 +51,11 @@ struct Parameters {
|
||||
* @return Value for the given token position.
|
||||
*/
|
||||
const storage::v3::PropertyValue &AtTokenPosition(int position) const {
|
||||
auto found = std::find_if(storage_.begin(), storage_.end(), [&](const auto &a) { return a.first == position; });
|
||||
auto found = storage_.find(position);
|
||||
MG_ASSERT(found != storage_.end(), "Token position must be present in container");
|
||||
return found->second;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the position-th stripped value. Asserts that this
|
||||
* container has at least (position + 1) elements.
|
||||
*
|
||||
* @param position Which stripped param is sought.
|
||||
* @return Token position and value for sought param.
|
||||
*/
|
||||
const std::pair<int, storage::v3::PropertyValue> &At(int position) const {
|
||||
MG_ASSERT(position < static_cast<int>(storage_.size()), "Invalid position");
|
||||
return storage_[position];
|
||||
}
|
||||
|
||||
/** Returns the number of arguments in this container */
|
||||
auto size() const { return storage_.size(); }
|
||||
|
||||
@ -75,7 +63,7 @@ struct Parameters {
|
||||
auto end() const { return storage_.end(); }
|
||||
|
||||
private:
|
||||
std::vector<std::pair<int, storage::v3::PropertyValue>> storage_;
|
||||
std::unordered_map<int, storage::v3::PropertyValue> storage_;
|
||||
};
|
||||
|
||||
struct EvaluationContext {
|
||||
|
@ -53,6 +53,10 @@ class NameIdMapper final {
|
||||
return it->second;
|
||||
}
|
||||
|
||||
const auto &GetIdToNameMap() const { return id_to_name_; }
|
||||
|
||||
const auto &GetNameToIdMap() const { return name_to_id_; }
|
||||
|
||||
private:
|
||||
// Necessary for comparison with string_view nad string
|
||||
// https://www.open-std.org/jtc1/sc22/wg21/docs/papers/2018/p0919r1.html
|
||||
|
@ -321,7 +321,8 @@ bool VerticesIterable::Iterator::operator==(const Iterator &other) const {
|
||||
}
|
||||
|
||||
Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key,
|
||||
const std::optional<PrimaryKey> max_primary_key, std::vector<SchemaProperty> schema, Config config)
|
||||
const std::optional<PrimaryKey> max_primary_key, std::vector<SchemaProperty> schema, Config config,
|
||||
std::unordered_map<uint64_t, std::string> id_to_name)
|
||||
: primary_label_{primary_label},
|
||||
min_primary_key_{min_primary_key},
|
||||
max_primary_key_{max_primary_key},
|
||||
@ -334,6 +335,7 @@ Shard::Shard(const LabelId primary_label, const PrimaryKey min_primary_key,
|
||||
epoch_id_{utils::GenerateUUID()},
|
||||
global_locker_{file_retainer_.AddLocker()} {
|
||||
CreateSchema(primary_label_, schema);
|
||||
StoreMapping(std::move(id_to_name));
|
||||
}
|
||||
|
||||
Shard::~Shard() {}
|
||||
|
@ -189,7 +189,8 @@ class Shard final {
|
||||
/// @throw std::system_error
|
||||
/// @throw std::bad_alloc
|
||||
explicit Shard(LabelId primary_label, PrimaryKey min_primary_key, std::optional<PrimaryKey> max_primary_key,
|
||||
std::vector<SchemaProperty> schema, Config config = Config());
|
||||
std::vector<SchemaProperty> schema, Config config = Config(),
|
||||
std::unordered_map<uint64_t, std::string> id_to_name = {});
|
||||
|
||||
Shard(const Shard &) = delete;
|
||||
Shard(Shard &&) noexcept = delete;
|
||||
|
@ -13,47 +13,50 @@
|
||||
|
||||
#include <queue>
|
||||
#include <set>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <boost/functional/hash.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
|
||||
#include <coordinator/coordinator.hpp>
|
||||
#include <io/address.hpp>
|
||||
#include <io/message_conversion.hpp>
|
||||
#include <io/messages.hpp>
|
||||
#include <io/rsm/raft.hpp>
|
||||
#include <io/time.hpp>
|
||||
#include <io/transport.hpp>
|
||||
#include <query/v2/requests.hpp>
|
||||
#include <storage/v3/shard.hpp>
|
||||
#include <storage/v3/shard_rsm.hpp>
|
||||
#include "coordinator/coordinator.hpp"
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "io/address.hpp"
|
||||
#include "io/message_conversion.hpp"
|
||||
#include "io/messages.hpp"
|
||||
#include "io/rsm/raft.hpp"
|
||||
#include "io/time.hpp"
|
||||
#include "io/transport.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "storage/v3/config.hpp"
|
||||
#include "storage/v3/shard.hpp"
|
||||
#include "storage/v3/shard_rsm.hpp"
|
||||
#include "storage/v3/shard_worker.hpp"
|
||||
|
||||
namespace memgraph::storage::v3 {
|
||||
|
||||
using boost::uuids::uuid;
|
||||
|
||||
using memgraph::coordinator::CoordinatorWriteRequests;
|
||||
using memgraph::coordinator::CoordinatorWriteResponses;
|
||||
using memgraph::coordinator::HeartbeatRequest;
|
||||
using memgraph::coordinator::HeartbeatResponse;
|
||||
using memgraph::io::Address;
|
||||
using memgraph::io::Duration;
|
||||
using memgraph::io::Message;
|
||||
using memgraph::io::RequestId;
|
||||
using memgraph::io::ResponseFuture;
|
||||
using memgraph::io::Time;
|
||||
using memgraph::io::messages::CoordinatorMessages;
|
||||
using memgraph::io::messages::ShardManagerMessages;
|
||||
using memgraph::io::messages::ShardMessages;
|
||||
using memgraph::io::rsm::Raft;
|
||||
using memgraph::io::rsm::WriteRequest;
|
||||
using memgraph::io::rsm::WriteResponse;
|
||||
using memgraph::msgs::ReadRequests;
|
||||
using memgraph::msgs::ReadResponses;
|
||||
using memgraph::msgs::WriteRequests;
|
||||
using memgraph::msgs::WriteResponses;
|
||||
using memgraph::storage::v3::ShardRsm;
|
||||
using coordinator::CoordinatorWriteRequests;
|
||||
using coordinator::CoordinatorWriteResponses;
|
||||
using coordinator::HeartbeatRequest;
|
||||
using coordinator::HeartbeatResponse;
|
||||
using io::Address;
|
||||
using io::Duration;
|
||||
using io::Message;
|
||||
using io::RequestId;
|
||||
using io::ResponseFuture;
|
||||
using io::Time;
|
||||
using io::messages::CoordinatorMessages;
|
||||
using io::messages::ShardManagerMessages;
|
||||
using io::messages::ShardMessages;
|
||||
using io::rsm::Raft;
|
||||
using io::rsm::WriteRequest;
|
||||
using io::rsm::WriteResponse;
|
||||
using msgs::ReadRequests;
|
||||
using msgs::ReadResponses;
|
||||
using msgs::WriteRequests;
|
||||
using msgs::WriteResponses;
|
||||
using storage::v3::ShardRsm;
|
||||
|
||||
using ShardManagerOrRsmMessage = std::variant<ShardMessages, ShardManagerMessages>;
|
||||
using TimeUuidPair = std::pair<Time, uuid>;
|
||||
@ -77,8 +80,71 @@ static_assert(kMinimumCronInterval < kMaximumCronInterval,
|
||||
template <typename IoImpl>
|
||||
class ShardManager {
|
||||
public:
|
||||
ShardManager(io::Io<IoImpl> io, Address coordinator_leader, coordinator::ShardMap shard_map)
|
||||
: io_(io), coordinator_leader_(coordinator_leader), shard_map_{std::move(shard_map)} {}
|
||||
ShardManager(io::Io<IoImpl> io, size_t shard_worker_threads, Address coordinator_leader)
|
||||
: io_(io), coordinator_leader_(coordinator_leader) {
|
||||
MG_ASSERT(shard_worker_threads >= 1);
|
||||
|
||||
for (int i = 0; i < shard_worker_threads; i++) {
|
||||
shard_worker::Queue queue;
|
||||
shard_worker::ShardWorker worker{io, queue};
|
||||
auto worker_handle = std::jthread([worker = std::move(worker)]() mutable { worker.Run(); });
|
||||
|
||||
workers_.emplace_back(queue);
|
||||
worker_handles_.emplace_back(std::move(worker_handle));
|
||||
worker_rsm_counts_.emplace_back(0);
|
||||
}
|
||||
}
|
||||
|
||||
ShardManager(ShardManager &&) noexcept = default;
|
||||
ShardManager &operator=(ShardManager &&) noexcept = default;
|
||||
ShardManager(const ShardManager &) = delete;
|
||||
ShardManager &operator=(const ShardManager &) = delete;
|
||||
|
||||
~ShardManager() {
|
||||
for (auto worker : workers_) {
|
||||
worker.Push(shard_worker::ShutDown{});
|
||||
}
|
||||
|
||||
workers_.clear();
|
||||
|
||||
// The jthread handes for our shard worker threads will be
|
||||
// blocked on implicitly when worker_handles_ is destroyed.
|
||||
}
|
||||
|
||||
size_t UuidToWorkerIndex(const uuid &to) {
|
||||
if (rsm_worker_mapping_.contains(to)) {
|
||||
return rsm_worker_mapping_.at(to);
|
||||
}
|
||||
|
||||
// We will now create a mapping for this (probably new) shard
|
||||
// by choosing the worker with the lowest number of existing
|
||||
// mappings.
|
||||
|
||||
size_t min_index = 0;
|
||||
size_t min_count = worker_rsm_counts_.at(min_index);
|
||||
|
||||
for (int i = 0; i < worker_rsm_counts_.size(); i++) {
|
||||
size_t worker_count = worker_rsm_counts_.at(i);
|
||||
if (worker_count <= min_count) {
|
||||
min_count = worker_count;
|
||||
min_index = i;
|
||||
}
|
||||
}
|
||||
|
||||
worker_rsm_counts_[min_index]++;
|
||||
rsm_worker_mapping_.emplace(to, min_index);
|
||||
|
||||
return min_index;
|
||||
}
|
||||
|
||||
void SendToWorkerByIndex(size_t worker_index, shard_worker::Message &&message) {
|
||||
workers_[worker_index].Push(std::forward<shard_worker::Message>(message));
|
||||
}
|
||||
|
||||
void SendToWorkerByUuid(const uuid &to, shard_worker::Message &&message) {
|
||||
size_t worker_index = UuidToWorkerIndex(to);
|
||||
SendToWorkerByIndex(worker_index, std::forward<shard_worker::Message>(message));
|
||||
}
|
||||
|
||||
/// Periodic protocol maintenance. Returns the time that Cron should be called again
|
||||
/// in the future.
|
||||
@ -86,33 +152,23 @@ class ShardManager {
|
||||
spdlog::info("running ShardManager::Cron, address {}", io_.GetAddress().ToString());
|
||||
Time now = io_.Now();
|
||||
|
||||
if (now >= next_cron_) {
|
||||
if (now >= next_reconciliation_) {
|
||||
Reconciliation();
|
||||
|
||||
std::uniform_int_distribution time_distrib(kMinimumCronInterval.count(), kMaximumCronInterval.count());
|
||||
|
||||
const auto rand = io_.Rand(time_distrib);
|
||||
|
||||
next_cron_ = now + Duration{rand};
|
||||
next_reconciliation_ = now + Duration{rand};
|
||||
}
|
||||
|
||||
if (!cron_schedule_.empty()) {
|
||||
const auto &[time, uuid] = cron_schedule_.top();
|
||||
|
||||
if (time <= now) {
|
||||
auto &rsm = rsm_map_.at(uuid);
|
||||
Time next_for_uuid = rsm.Cron();
|
||||
|
||||
cron_schedule_.pop();
|
||||
cron_schedule_.push(std::make_pair(next_for_uuid, uuid));
|
||||
|
||||
const auto &[next_time, _uuid] = cron_schedule_.top();
|
||||
|
||||
return std::min(next_cron_, next_time);
|
||||
}
|
||||
for (auto &worker : workers_) {
|
||||
worker.Push(shard_worker::Cron{});
|
||||
}
|
||||
|
||||
return next_cron_;
|
||||
Time next_worker_cron = now + std::chrono::milliseconds(500);
|
||||
|
||||
return std::min(next_worker_cron, next_reconciliation_);
|
||||
}
|
||||
|
||||
/// Returns the Address for our underlying Io implementation
|
||||
@ -126,18 +182,22 @@ class ShardManager {
|
||||
MG_ASSERT(address.last_known_port == to.last_known_port);
|
||||
MG_ASSERT(address.last_known_ip == to.last_known_ip);
|
||||
|
||||
auto &rsm = rsm_map_.at(to.unique_id);
|
||||
|
||||
rsm.Handle(std::forward<ShardMessages>(sm), request_id, from);
|
||||
SendToWorkerByUuid(to.unique_id, shard_worker::RouteMessage{
|
||||
.message = std::move(sm),
|
||||
.request_id = request_id,
|
||||
.to = to,
|
||||
.from = from,
|
||||
});
|
||||
}
|
||||
|
||||
private:
|
||||
io::Io<IoImpl> io_;
|
||||
std::map<uuid, ShardRaft<IoImpl>> rsm_map_;
|
||||
std::priority_queue<std::pair<Time, uuid>, std::vector<std::pair<Time, uuid>>, std::greater<>> cron_schedule_;
|
||||
Time next_cron_ = Time::min();
|
||||
std::vector<shard_worker::Queue> workers_;
|
||||
std::vector<std::jthread> worker_handles_;
|
||||
std::vector<size_t> worker_rsm_counts_;
|
||||
std::unordered_map<uuid, size_t, boost::hash<boost::uuids::uuid>> rsm_worker_mapping_;
|
||||
Time next_reconciliation_ = Time::min();
|
||||
Address coordinator_leader_;
|
||||
coordinator::ShardMap shard_map_;
|
||||
std::optional<ResponseFuture<WriteResponse<CoordinatorWriteResponses>>> heartbeat_res_;
|
||||
|
||||
// TODO(tyler) over time remove items from initialized_but_not_confirmed_rsm_
|
||||
@ -190,50 +250,22 @@ class ShardManager {
|
||||
}
|
||||
|
||||
void EnsureShardsInitialized(HeartbeatResponse hr) {
|
||||
for (const auto &shard_to_initialize : hr.shards_to_initialize) {
|
||||
InitializeRsm(shard_to_initialize);
|
||||
initialized_but_not_confirmed_rsm_.emplace(shard_to_initialize.uuid);
|
||||
}
|
||||
}
|
||||
for (const auto &to_init : hr.shards_to_initialize) {
|
||||
initialized_but_not_confirmed_rsm_.emplace(to_init.uuid);
|
||||
|
||||
/// Returns true if the RSM was able to be initialized, and false if it was already initialized
|
||||
void InitializeRsm(coordinator::ShardToInitialize to_init) {
|
||||
if (rsm_map_.contains(to_init.uuid)) {
|
||||
// it's not a bug for the coordinator to send us UUIDs that we have
|
||||
// already created, because there may have been lag that caused
|
||||
// the coordinator not to hear back from us.
|
||||
return;
|
||||
}
|
||||
|
||||
auto rsm_io = io_.ForkLocal();
|
||||
auto io_addr = rsm_io.GetAddress();
|
||||
io_addr.unique_id = to_init.uuid;
|
||||
rsm_io.SetAddress(io_addr);
|
||||
|
||||
// TODO(tyler) get geers from Coordinator in HeartbeatResponse
|
||||
std::vector<Address> rsm_peers = {};
|
||||
|
||||
std::unique_ptr<Shard> shard =
|
||||
std::make_unique<Shard>(to_init.label_id, to_init.min_key, to_init.max_key, to_init.schema, to_init.config);
|
||||
// TODO(jbajic) Should be sync with coordinator and not passed
|
||||
std::unordered_map<uint64_t, std::string> id_to_name;
|
||||
const auto map_type_ids = [&id_to_name](const auto &name_to_id_type) {
|
||||
for (const auto &[name, id] : name_to_id_type) {
|
||||
id_to_name.insert({id.AsUint(), name});
|
||||
if (rsm_worker_mapping_.contains(to_init.uuid)) {
|
||||
// it's not a bug for the coordinator to send us UUIDs that we have
|
||||
// already created, because there may have been lag that caused
|
||||
// the coordinator not to hear back from us.
|
||||
return;
|
||||
}
|
||||
};
|
||||
map_type_ids(shard_map_.edge_types);
|
||||
map_type_ids(shard_map_.labels);
|
||||
map_type_ids(shard_map_.properties);
|
||||
shard->StoreMapping(std::move(id_to_name));
|
||||
|
||||
ShardRsm rsm_state{std::move(shard)};
|
||||
size_t worker_index = UuidToWorkerIndex(to_init.uuid);
|
||||
|
||||
ShardRaft<IoImpl> rsm{std::move(rsm_io), rsm_peers, std::move(rsm_state)};
|
||||
SendToWorkerByIndex(worker_index, to_init);
|
||||
|
||||
spdlog::info("SM created a new shard with UUID {}", to_init.uuid);
|
||||
|
||||
rsm_map_.emplace(to_init.uuid, std::move(rsm));
|
||||
rsm_worker_mapping_.emplace(to_init.uuid, worker_index);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
|
||||
#include "parser/opencypher/parser.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "storage/v2/view.hpp"
|
||||
#include "storage/v3/bindings/ast/ast.hpp"
|
||||
#include "storage/v3/bindings/cypher_main_visitor.hpp"
|
||||
#include "storage/v3/bindings/db_accessor.hpp"
|
||||
@ -113,6 +114,24 @@ std::optional<std::map<PropertyId, Value>> CollectSpecificPropertiesFromAccessor
|
||||
return ret;
|
||||
}
|
||||
|
||||
std::optional<std::map<PropertyId, Value>> PrimaryKeysFromAccessor(const VertexAccessor &acc, View view,
|
||||
const Schemas::Schema *schema) {
|
||||
std::map<PropertyId, Value> ret;
|
||||
auto props = acc.Properties(view);
|
||||
auto maybe_pk = acc.PrimaryKey(view);
|
||||
if (maybe_pk.HasError()) {
|
||||
spdlog::debug("Encountered an error while trying to get vertex primary key.");
|
||||
return std::nullopt;
|
||||
}
|
||||
auto &pk = maybe_pk.GetValue();
|
||||
MG_ASSERT(schema->second.size() == pk.size(), "PrimaryKey size does not match schema!");
|
||||
for (size_t i{0}; i < schema->second.size(); ++i) {
|
||||
ret.emplace(schema->second[i].property_id, FromPropertyValueToValue(std::move(pk[i])));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
std::optional<std::map<PropertyId, Value>> CollectAllPropertiesFromAccessor(const VertexAccessor &acc, View view,
|
||||
const Schemas::Schema *schema) {
|
||||
std::map<PropertyId, Value> ret;
|
||||
@ -129,17 +148,9 @@ std::optional<std::map<PropertyId, Value>> CollectAllPropertiesFromAccessor(cons
|
||||
});
|
||||
properties.clear();
|
||||
|
||||
// TODO(antaljanosbenjamin): Once the VertexAccessor::Properties returns also the primary keys, we can get rid of this
|
||||
// code.
|
||||
auto maybe_pk = acc.PrimaryKey(view);
|
||||
if (maybe_pk.HasError()) {
|
||||
spdlog::debug("Encountered an error while trying to get vertex primary key.");
|
||||
}
|
||||
|
||||
auto &pk = maybe_pk.GetValue();
|
||||
MG_ASSERT(schema->second.size() == pk.size(), "PrimaryKey size does not match schema!");
|
||||
for (size_t i{0}; i < schema->second.size(); ++i) {
|
||||
ret.emplace(schema->second[i].property_id, FromPropertyValueToValue(std::move(pk[i])));
|
||||
auto pks = PrimaryKeysFromAccessor(acc, view, schema);
|
||||
if (pks) {
|
||||
ret.merge(*pks);
|
||||
}
|
||||
|
||||
return ret;
|
||||
@ -190,7 +201,9 @@ std::optional<msgs::Vertex> FillUpSourceVertex(const std::optional<VertexAccesso
|
||||
}
|
||||
|
||||
std::optional<std::map<PropertyId, Value>> FillUpSourceVertexProperties(const std::optional<VertexAccessor> &v_acc,
|
||||
const msgs::ExpandOneRequest &req) {
|
||||
const msgs::ExpandOneRequest &req,
|
||||
storage::v3::View view,
|
||||
const Schemas::Schema *schema) {
|
||||
std::map<PropertyId, Value> src_vertex_properties;
|
||||
|
||||
if (!req.src_vertex_properties) {
|
||||
@ -204,6 +217,10 @@ std::optional<std::map<PropertyId, Value>> FillUpSourceVertexProperties(const st
|
||||
for (auto &[key, val] : props.GetValue()) {
|
||||
src_vertex_properties.insert(std::make_pair(key, FromPropertyValueToValue(std::move(val))));
|
||||
}
|
||||
auto pks = PrimaryKeysFromAccessor(*v_acc, view, schema);
|
||||
if (pks) {
|
||||
src_vertex_properties.merge(*pks);
|
||||
}
|
||||
|
||||
} else if (req.src_vertex_properties.value().empty()) {
|
||||
// NOOP
|
||||
@ -264,7 +281,6 @@ std::optional<std::array<std::vector<EdgeAccessor>, 2>> FillUpConnectingEdges(
|
||||
return std::nullopt;
|
||||
}
|
||||
in_edges = maybe_filter_based_on_edge_uniquness(std::move(in_edges_result.GetValue()), msgs::EdgeDirection::IN);
|
||||
|
||||
auto out_edges_result = v_acc->OutEdges(View::NEW, edge_types);
|
||||
if (out_edges_result.HasError()) {
|
||||
spdlog::debug("Encountered an error while trying to get out-going EdgeAccessors. Transaction id: {}",
|
||||
@ -303,7 +319,8 @@ bool FillEdges(const std::vector<EdgeAccessor> &edges, msgs::ExpandOneResultRow
|
||||
|
||||
std::optional<msgs::ExpandOneResultRow> GetExpandOneResult(
|
||||
Shard::Accessor &acc, msgs::VertexId src_vertex, const msgs::ExpandOneRequest &req,
|
||||
const EdgeUniqunessFunction &maybe_filter_based_on_edge_uniquness, const EdgeFiller &edge_filler) {
|
||||
const EdgeUniqunessFunction &maybe_filter_based_on_edge_uniquness, const EdgeFiller &edge_filler,
|
||||
const Schemas::Schema *schema) {
|
||||
/// Fill up source vertex
|
||||
const auto primary_key = ConvertPropertyVector(std::move(src_vertex.second));
|
||||
auto v_acc = acc.FindVertex(primary_key, View::NEW);
|
||||
@ -312,9 +329,9 @@ std::optional<msgs::ExpandOneResultRow> GetExpandOneResult(
|
||||
if (!source_vertex) {
|
||||
return std::nullopt;
|
||||
}
|
||||
std::optional<std::map<PropertyId, Value>> src_vertex_properties;
|
||||
src_vertex_properties = FillUpSourceVertexProperties(v_acc, req, storage::v3::View::NEW, schema);
|
||||
|
||||
/// Fill up source vertex properties
|
||||
auto src_vertex_properties = FillUpSourceVertexProperties(v_acc, req);
|
||||
if (!src_vertex_properties) {
|
||||
return std::nullopt;
|
||||
}
|
||||
|
224
src/storage/v3/shard_worker.hpp
Normal file
224
src/storage/v3/shard_worker.hpp
Normal file
@ -0,0 +1,224 @@
|
||||
// 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 <deque>
|
||||
#include <memory>
|
||||
#include <queue>
|
||||
#include <variant>
|
||||
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
|
||||
#include "coordinator/coordinator.hpp"
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "io/address.hpp"
|
||||
#include "io/future.hpp"
|
||||
#include "io/messages.hpp"
|
||||
#include "io/rsm/raft.hpp"
|
||||
#include "io/time.hpp"
|
||||
#include "io/transport.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "storage/v3/shard_rsm.hpp"
|
||||
|
||||
namespace memgraph::storage::v3::shard_worker {
|
||||
|
||||
/// Obligations:
|
||||
/// * ShutDown
|
||||
/// * Cron
|
||||
/// * RouteMessage
|
||||
/// * ShardToInitialize
|
||||
|
||||
using boost::uuids::uuid;
|
||||
|
||||
using coordinator::ShardToInitialize;
|
||||
using io::Address;
|
||||
using io::RequestId;
|
||||
using io::Time;
|
||||
using io::messages::ShardMessages;
|
||||
using io::rsm::Raft;
|
||||
using msgs::ReadRequests;
|
||||
using msgs::ReadResponses;
|
||||
using msgs::WriteRequests;
|
||||
using msgs::WriteResponses;
|
||||
using storage::v3::ShardRsm;
|
||||
|
||||
template <typename IoImpl>
|
||||
using ShardRaft = Raft<IoImpl, ShardRsm, WriteRequests, WriteResponses, ReadRequests, ReadResponses>;
|
||||
|
||||
struct ShutDown {};
|
||||
|
||||
struct Cron {};
|
||||
|
||||
struct RouteMessage {
|
||||
ShardMessages message;
|
||||
RequestId request_id;
|
||||
Address to;
|
||||
Address from;
|
||||
};
|
||||
|
||||
using Message = std::variant<ShutDown, Cron, ShardToInitialize, RouteMessage>;
|
||||
|
||||
struct QueueInner {
|
||||
std::mutex mu{};
|
||||
std::condition_variable cv;
|
||||
// TODO(tyler) handle simulator communication std::shared_ptr<std::atomic<int>> blocked;
|
||||
|
||||
// TODO(tyler) investigate using a priority queue that prioritizes messages in a way that
|
||||
// improves overall QoS. For example, maybe we want to schedule raft Append messages
|
||||
// ahead of Read messages or generally writes before reads for lowering the load on the
|
||||
// overall system faster etc... When we do this, we need to make sure to avoid
|
||||
// starvation by sometimes randomizing priorities, rather than following a strict
|
||||
// prioritization.
|
||||
std::deque<Message> queue;
|
||||
};
|
||||
|
||||
/// There are two reasons to implement our own Queue instead of using
|
||||
/// one off-the-shelf:
|
||||
/// 1. we will need to know in the simulator when all threads are waiting
|
||||
/// 2. we will want to implement our own priority queue within this for QoS
|
||||
class Queue {
|
||||
std::shared_ptr<QueueInner> inner_ = std::make_shared<QueueInner>();
|
||||
|
||||
public:
|
||||
void Push(Message &&message) {
|
||||
{
|
||||
MG_ASSERT(inner_.use_count() > 0);
|
||||
std::unique_lock<std::mutex> lock(inner_->mu);
|
||||
|
||||
inner_->queue.emplace_back(std::forward<Message>(message));
|
||||
} // lock dropped before notifying condition variable
|
||||
|
||||
inner_->cv.notify_all();
|
||||
}
|
||||
|
||||
Message Pop() {
|
||||
MG_ASSERT(inner_.use_count() > 0);
|
||||
std::unique_lock<std::mutex> lock(inner_->mu);
|
||||
|
||||
while (inner_->queue.empty()) {
|
||||
inner_->cv.wait(lock);
|
||||
}
|
||||
|
||||
Message message = std::move(inner_->queue.front());
|
||||
inner_->queue.pop_front();
|
||||
|
||||
return message;
|
||||
}
|
||||
};
|
||||
|
||||
/// A ShardWorker owns Raft<ShardRsm> instances. receives messages from the ShardManager.
|
||||
template <class IoImpl>
|
||||
class ShardWorker {
|
||||
io::Io<IoImpl> io_;
|
||||
Queue queue_;
|
||||
std::priority_queue<std::pair<Time, uuid>, std::vector<std::pair<Time, uuid>>, std::greater<>> cron_schedule_;
|
||||
Time next_cron_ = Time::min();
|
||||
std::map<uuid, ShardRaft<IoImpl>> rsm_map_;
|
||||
|
||||
bool Process(ShutDown && /* shut_down */) { return false; }
|
||||
|
||||
bool Process(Cron && /* cron */) {
|
||||
Cron();
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Process(ShardToInitialize &&shard_to_initialize) {
|
||||
InitializeRsm(std::forward<ShardToInitialize>(shard_to_initialize));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool Process(RouteMessage &&route_message) {
|
||||
auto &rsm = rsm_map_.at(route_message.to.unique_id);
|
||||
|
||||
rsm.Handle(std::move(route_message.message), route_message.request_id, route_message.from);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
Time Cron() {
|
||||
spdlog::info("running ShardWorker::Cron, address {}", io_.GetAddress().ToString());
|
||||
Time now = io_.Now();
|
||||
|
||||
while (!cron_schedule_.empty()) {
|
||||
const auto &[time, uuid] = cron_schedule_.top();
|
||||
|
||||
if (time <= now) {
|
||||
auto &rsm = rsm_map_.at(uuid);
|
||||
Time next_for_uuid = rsm.Cron();
|
||||
|
||||
cron_schedule_.pop();
|
||||
cron_schedule_.push(std::make_pair(next_for_uuid, uuid));
|
||||
} else {
|
||||
return time;
|
||||
}
|
||||
}
|
||||
|
||||
return now + std::chrono::microseconds(1000);
|
||||
}
|
||||
|
||||
void InitializeRsm(ShardToInitialize to_init) {
|
||||
if (rsm_map_.contains(to_init.uuid)) {
|
||||
// it's not a bug for the coordinator to send us UUIDs that we have
|
||||
// already created, because there may have been lag that caused
|
||||
// the coordinator not to hear back from us.
|
||||
return;
|
||||
}
|
||||
|
||||
auto rsm_io = io_.ForkLocal();
|
||||
auto io_addr = rsm_io.GetAddress();
|
||||
io_addr.unique_id = to_init.uuid;
|
||||
rsm_io.SetAddress(io_addr);
|
||||
|
||||
// TODO(tyler) get peers from Coordinator in HeartbeatResponse
|
||||
std::vector<Address> rsm_peers = {};
|
||||
|
||||
std::unique_ptr<Shard> shard = std::make_unique<Shard>(to_init.label_id, to_init.min_key, to_init.max_key,
|
||||
to_init.schema, to_init.config, to_init.id_to_names);
|
||||
|
||||
ShardRsm rsm_state{std::move(shard)};
|
||||
|
||||
ShardRaft<IoImpl> rsm{std::move(rsm_io), rsm_peers, std::move(rsm_state)};
|
||||
|
||||
spdlog::info("SM created a new shard with UUID {}", to_init.uuid);
|
||||
|
||||
// perform an initial Cron call for the new RSM
|
||||
Time next_cron = rsm.Cron();
|
||||
cron_schedule_.push(std::make_pair(next_cron, to_init.uuid));
|
||||
|
||||
rsm_map_.emplace(to_init.uuid, std::move(rsm));
|
||||
}
|
||||
|
||||
public:
|
||||
ShardWorker(io::Io<IoImpl> io, Queue queue) : io_(io), queue_(queue) {}
|
||||
ShardWorker(ShardWorker &&) noexcept = default;
|
||||
ShardWorker &operator=(ShardWorker &&) noexcept = default;
|
||||
ShardWorker(const ShardWorker &) = delete;
|
||||
ShardWorker &operator=(const ShardWorker &) = delete;
|
||||
~ShardWorker() = default;
|
||||
|
||||
void Run() {
|
||||
while (true) {
|
||||
Message message = queue_.Pop();
|
||||
|
||||
const bool should_continue =
|
||||
std::visit([&](auto &&msg) { return Process(std::forward<decltype(msg)>(msg)); }, std::move(message));
|
||||
|
||||
if (!should_continue) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace memgraph::storage::v3::shard_worker
|
@ -33,8 +33,9 @@ std::ostream &operator<<(std::ostream &in, const std::vector<T> &vector) {
|
||||
return in;
|
||||
}
|
||||
|
||||
template <typename K, typename V>
|
||||
std::ostream &operator<<(std::ostream &in, const std::map<K, V> &map) {
|
||||
namespace detail {
|
||||
template <typename T>
|
||||
std::ostream &MapImpl(std::ostream &in, const T &map) {
|
||||
in << "{";
|
||||
bool first = true;
|
||||
for (const auto &[a, b] : map) {
|
||||
@ -49,6 +50,17 @@ std::ostream &operator<<(std::ostream &in, const std::map<K, V> &map) {
|
||||
in << "}";
|
||||
return in;
|
||||
}
|
||||
} // namespace detail
|
||||
|
||||
template <typename K, typename V>
|
||||
std::ostream &operator<<(std::ostream &in, const std::map<K, V> &map) {
|
||||
return detail::MapImpl(in, map);
|
||||
}
|
||||
|
||||
template <typename K, typename V, typename THash, typename Cmp>
|
||||
std::ostream &operator<<(std::ostream &in, const std::unordered_map<K, V, THash, Cmp> &map) {
|
||||
return detail::MapImpl(in, map);
|
||||
}
|
||||
|
||||
template <typename K, typename V>
|
||||
std::ostream &operator<<(std::ostream &in, const std::unordered_map<K, V> &map) {
|
||||
|
@ -18,7 +18,7 @@ extern "C" {
|
||||
#include "utils/tsc.hpp"
|
||||
|
||||
namespace memgraph::utils {
|
||||
uint64_t ReadTSC() { return rdtsc(); }
|
||||
uint64_t ReadTSC() noexcept { return rdtsc(); }
|
||||
|
||||
std::optional<double> GetTSCFrequency() {
|
||||
// init is only needed for fetching frequency
|
||||
|
@ -18,7 +18,7 @@ namespace memgraph::utils {
|
||||
|
||||
// TSC stands for Time-Stamp Counter
|
||||
|
||||
uint64_t ReadTSC();
|
||||
uint64_t ReadTSC() noexcept;
|
||||
|
||||
std::optional<double> GetTSCFrequency();
|
||||
|
||||
|
@ -3,3 +3,8 @@ function(distributed_queries_e2e_python_files FILE_NAME)
|
||||
endfunction()
|
||||
|
||||
distributed_queries_e2e_python_files(distributed_queries.py)
|
||||
distributed_queries_e2e_python_files(unwind_collect.py)
|
||||
distributed_queries_e2e_python_files(order_by_and_limit.py)
|
||||
distributed_queries_e2e_python_files(distinct.py)
|
||||
distributed_queries_e2e_python_files(optional_match.py)
|
||||
distributed_queries_e2e_python_files(common.py)
|
||||
|
44
tests/e2e/distributed_queries/common.py
Normal file
44
tests/e2e/distributed_queries/common.py
Normal 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.
|
||||
|
||||
import typing
|
||||
import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def connection():
|
||||
connection = connect()
|
||||
return connection
|
||||
|
||||
|
||||
def connect(**kwargs) -> mgclient.Connection:
|
||||
connection = mgclient.connect(host="localhost", port=7687, **kwargs)
|
||||
connection.autocommit = True
|
||||
return connection
|
||||
|
||||
|
||||
def execute_and_fetch_all(cursor: mgclient.Cursor, query: str, params: dict = {}) -> typing.List[tuple]:
|
||||
cursor.execute(query, params)
|
||||
return cursor.fetchall()
|
||||
|
||||
|
||||
def has_n_result_row(cursor: mgclient.Cursor, query: str, n: int):
|
||||
results = execute_and_fetch_all(cursor, query)
|
||||
return len(results) == n
|
||||
|
||||
|
||||
def wait_for_shard_manager_to_initialize():
|
||||
# The ShardManager in memgraph takes some time to initialize
|
||||
# the shards, thus we cannot just run the queries right away
|
||||
time.sleep(3)
|
38
tests/e2e/distributed_queries/distinct.py
Normal file
38
tests/e2e/distributed_queries/distinct.py
Normal file
@ -0,0 +1,38 @@
|
||||
# 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.
|
||||
|
||||
import typing
|
||||
import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
from common import *
|
||||
|
||||
|
||||
def test_distinct(connection):
|
||||
wait_for_shard_manager_to_initialize()
|
||||
cursor = connection.cursor()
|
||||
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:0})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:1})", 0)
|
||||
assert has_n_result_row(cursor, "MATCH (n), (m) CREATE (n)-[:TO]->(m)", 0)
|
||||
assert has_n_result_row(cursor, "MATCH (n)-[r]->(m) RETURN r", 4)
|
||||
|
||||
results = execute_and_fetch_all(cursor, "MATCH (n)-[r]->(m) RETURN DISTINCT m")
|
||||
assert len(results) == 2
|
||||
for i, n in enumerate(results):
|
||||
n_props = n[0].properties
|
||||
assert len(n_props) == 1
|
||||
assert n_props["property"] == i
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main([__file__, "-rA"]))
|
@ -14,34 +14,7 @@ import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def connection():
|
||||
connection = connect()
|
||||
return connection
|
||||
|
||||
|
||||
def connect(**kwargs) -> mgclient.Connection:
|
||||
connection = mgclient.connect(host="localhost", port=7687, **kwargs)
|
||||
connection.autocommit = True
|
||||
return connection
|
||||
|
||||
|
||||
def execute_and_fetch_all(cursor: mgclient.Cursor, query: str, params: dict = {}) -> typing.List[tuple]:
|
||||
cursor.execute(query, params)
|
||||
return cursor.fetchall()
|
||||
|
||||
|
||||
def has_n_result_row(cursor: mgclient.Cursor, query: str, n: int):
|
||||
results = execute_and_fetch_all(cursor, query)
|
||||
return len(results) == n
|
||||
|
||||
|
||||
def wait_for_shard_manager_to_initialize():
|
||||
# The ShardManager in memgraph takes some time to initialize
|
||||
# the shards, thus we cannot just run the queries right away
|
||||
time.sleep(3)
|
||||
from common import *
|
||||
|
||||
|
||||
def test_vertex_creation_and_scanall(connection):
|
||||
@ -62,7 +35,7 @@ def test_vertex_creation_and_scanall(connection):
|
||||
assert len(results) == 9
|
||||
for (n, r, m) in results:
|
||||
n_props = n.properties
|
||||
assert len(n_props) == 0, "n is not expected to have properties, update the test!"
|
||||
assert len(n_props) == 1, "n is not expected to have properties, update the test!"
|
||||
assert len(n.labels) == 0, "n is not expected to have labels, update the test!"
|
||||
|
||||
assert r.type == "TO"
|
||||
|
37
tests/e2e/distributed_queries/optional_match.py
Normal file
37
tests/e2e/distributed_queries/optional_match.py
Normal 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.
|
||||
|
||||
import typing
|
||||
import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
from common import *
|
||||
|
||||
|
||||
def test_optional_match(connection):
|
||||
wait_for_shard_manager_to_initialize()
|
||||
cursor = connection.cursor()
|
||||
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:0})", 0)
|
||||
|
||||
results = execute_and_fetch_all(
|
||||
cursor, "MATCH (n:label) OPTIONAL MATCH (n:label)-[:TO]->(parent:label) RETURN parent"
|
||||
)
|
||||
assert len(results) == 1
|
||||
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:2})", 0)
|
||||
assert has_n_result_row(cursor, "MATCH (n), (m) CREATE (n)-[:TO]->(m)", 0)
|
||||
assert has_n_result_row(cursor, "MATCH (n:label) OPTIONAL MATCH (n)-[r:TO]->(m:label) RETURN r", 4)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main([__file__, "-rA"]))
|
44
tests/e2e/distributed_queries/order_by_and_limit.py
Normal file
44
tests/e2e/distributed_queries/order_by_and_limit.py
Normal 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.
|
||||
|
||||
import typing
|
||||
import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
from common import *
|
||||
|
||||
|
||||
def test_order_by_and_limit(connection):
|
||||
wait_for_shard_manager_to_initialize()
|
||||
cursor = connection.cursor()
|
||||
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:1})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:2})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:3})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:4})", 0)
|
||||
|
||||
results = execute_and_fetch_all(cursor, "MATCH (n) RETURN n ORDER BY n.property DESC")
|
||||
assert len(results) == 4
|
||||
i = 4
|
||||
for n in results:
|
||||
n_props = n[0].properties
|
||||
assert len(n_props) == 1
|
||||
assert n_props["property"] == i
|
||||
i = i - 1
|
||||
|
||||
result = execute_and_fetch_all(cursor, "MATCH (n) RETURN n ORDER BY n.property LIMIT 1")
|
||||
assert len(result) == 1
|
||||
assert result[0][0].properties["property"] == 1
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main([__file__, "-rA"]))
|
34
tests/e2e/distributed_queries/unwind_collect.py
Normal file
34
tests/e2e/distributed_queries/unwind_collect.py
Normal file
@ -0,0 +1,34 @@
|
||||
# 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.
|
||||
|
||||
import typing
|
||||
import mgclient
|
||||
import sys
|
||||
import pytest
|
||||
import time
|
||||
from common import *
|
||||
|
||||
|
||||
def test_collect_unwind(connection):
|
||||
wait_for_shard_manager_to_initialize()
|
||||
cursor = connection.cursor()
|
||||
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:1})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:2})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:3})", 0)
|
||||
assert has_n_result_row(cursor, "CREATE (n :label {property:4})", 0)
|
||||
|
||||
assert has_n_result_row(cursor, "MATCH (n) WITH collect(n) AS result RETURN result", 1)
|
||||
assert has_n_result_row(cursor, "MATCH (n) WITH collect(n) AS nd UNWIND nd AS result RETURN result", 4)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main([__file__, "-rA"]))
|
@ -11,3 +11,23 @@ workloads:
|
||||
binary: "tests/e2e/pytest_runner.sh"
|
||||
args: ["distributed_queries/distributed_queries.py"]
|
||||
<<: *template_cluster
|
||||
|
||||
- name: "Distributed unwind collect"
|
||||
binary: "tests/e2e/pytest_runner.sh"
|
||||
args: ["distributed_queries/unwind_collect.py"]
|
||||
<<: *template_cluster
|
||||
|
||||
- name: "Distributed order by and limit"
|
||||
binary: "tests/e2e/pytest_runner.sh"
|
||||
args: ["distributed_queries/order_by_and_limit.py"]
|
||||
<<: *template_cluster
|
||||
|
||||
- name: "Distributed distinct"
|
||||
binary: "tests/e2e/pytest_runner.sh"
|
||||
args: ["distributed_queries/distinct.py"]
|
||||
<<: *template_cluster
|
||||
|
||||
- name: "Distributed optional match"
|
||||
binary: "tests/e2e/pytest_runner.sh"
|
||||
args: ["distributed_queries/optional_match.py"]
|
||||
<<: *template_cluster
|
||||
|
@ -12,7 +12,7 @@ PIP_DEPS=(
|
||||
"neo4j-driver==4.1.1"
|
||||
"parse==1.18.0"
|
||||
"parse-type==0.5.2"
|
||||
"pytest==6.2.3"
|
||||
"pytest==6.2.5"
|
||||
"pyyaml==5.4.1"
|
||||
"six==1.15.0"
|
||||
)
|
||||
@ -36,12 +36,12 @@ PYTHON_MINOR=$(python3 -c 'import sys; print(sys.version_info[:][1])')
|
||||
# NOTE (2021-11-15): PyPi doesn't contain pulsar-client for Python 3.9 so we have to use
|
||||
# our manually built wheel file. When they update the repository, pulsar-client can be
|
||||
# added as a regular PIP dependancy
|
||||
if [ $PYTHON_MINOR -lt 9 ]; then
|
||||
pip --timeout 1000 install "pulsar-client==2.8.1"
|
||||
else
|
||||
pip --timeout 1000 install https://s3-eu-west-1.amazonaws.com/deps.memgraph.io/pulsar_client-2.8.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.whl
|
||||
fi
|
||||
|
||||
#if [ $PYTHON_MINOR -lt 9 ]; then
|
||||
# pip --timeout 1000 install "pulsar-client==2.8.1"
|
||||
#else
|
||||
# pip --timeout 1000 install https://s3-eu-west-1.amazonaws.com/deps.memgraph.io/pulsar_client-2.8.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.whl
|
||||
#fi
|
||||
#
|
||||
for pkg in "${PIP_DEPS[@]}"; do
|
||||
pip --timeout 1000 install "$pkg"
|
||||
done
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <rapidcheck.h>
|
||||
#include <rapidcheck/gtest.h>
|
||||
#include <spdlog/cfg/env.h>
|
||||
|
||||
#include "generated_operations.hpp"
|
||||
#include "io/simulator/simulator_config.hpp"
|
||||
@ -35,6 +36,8 @@ using storage::v3::kMaximumCronInterval;
|
||||
RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, NonEmptyOpVec ops)) {
|
||||
// TODO(tyler) set abort_time to something more restrictive than Time::max()
|
||||
|
||||
spdlog::cfg::load_env_levels();
|
||||
|
||||
SimulatorConfig sim_config{
|
||||
.drop_percent = 0,
|
||||
.perform_timeouts = false,
|
||||
|
@ -81,7 +81,7 @@ MachineManager<SimulatorTransport> MkMm(Simulator &simulator, std::vector<Addres
|
||||
|
||||
Coordinator coordinator{shard_map};
|
||||
|
||||
return MachineManager{io, config, coordinator, shard_map};
|
||||
return MachineManager{io, config, coordinator};
|
||||
}
|
||||
|
||||
void RunMachine(MachineManager<SimulatorTransport> mm) { mm.Run(); }
|
||||
@ -194,6 +194,22 @@ void ExecuteOp(msgs::ShardRequestManager<SimulatorTransport> &shard_request_mana
|
||||
}
|
||||
}
|
||||
|
||||
/// This struct exists as a way of detaching
|
||||
/// a thread if something causes an uncaught
|
||||
/// exception - because that thread would not
|
||||
/// receive a ShutDown message otherwise, and
|
||||
/// would cause the test to hang forever.
|
||||
struct DetachIfDropped {
|
||||
std::jthread &handle;
|
||||
bool detach = true;
|
||||
|
||||
~DetachIfDropped() {
|
||||
if (detach && handle.joinable()) {
|
||||
handle.detach();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig &cluster_config,
|
||||
const std::vector<Op> &ops) {
|
||||
spdlog::info("========================== NEW SIMULATION ==========================");
|
||||
@ -217,9 +233,7 @@ void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig
|
||||
|
||||
auto mm_thread_1 = std::jthread(RunMachine, std::move(mm_1));
|
||||
|
||||
// Need to detach this thread so that the destructor does not
|
||||
// block before we can propagate assertion failures.
|
||||
mm_thread_1.detach();
|
||||
auto detach_on_error = DetachIfDropped{.handle = mm_thread_1};
|
||||
|
||||
// TODO(tyler) clarify addresses of coordinator etc... as it's a mess
|
||||
|
||||
@ -236,6 +250,11 @@ void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig
|
||||
std::visit([&](auto &o) { ExecuteOp(shard_request_manager, correctness_model, o); }, op.inner);
|
||||
}
|
||||
|
||||
// We have now completed our workload without failing any assertions, so we can
|
||||
// disable detaching the worker thread, which will cause the mm_thread_1 jthread
|
||||
// to be joined when this function returns.
|
||||
detach_on_error.detach = false;
|
||||
|
||||
simulator.ShutDown();
|
||||
|
||||
SimulatorStats stats = simulator.Stats();
|
||||
|
@ -278,49 +278,6 @@ target_link_libraries(${test_prefix}utils_temporal mg-utils)
|
||||
add_unit_test(utils_histogram.cpp)
|
||||
target_link_libraries(${test_prefix}utils_histogram 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)
|
||||
|
||||
add_unit_test(property_value_v2.cpp)
|
||||
target_link_libraries(${test_prefix}property_value_v2 mg-storage-v2 mg-utils)
|
||||
|
||||
add_unit_test(storage_v2.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2 mg-storage-v2 storage_test_utils)
|
||||
|
||||
add_unit_test(storage_v2_constraints.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_constraints mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_decoder_encoder.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_decoder_encoder mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_durability.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_durability mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_edge.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_edge mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_gc.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_gc mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_indices.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_indices mg-storage-v2 mg-utils)
|
||||
|
||||
add_unit_test(storage_v2_name_id_mapper.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_name_id_mapper mg-storage-v2)
|
||||
|
||||
add_unit_test(storage_v2_property_store.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_property_store mg-storage-v2 fmt)
|
||||
|
||||
add_unit_test(storage_v2_wal_file.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_wal_file mg-storage-v2 fmt)
|
||||
|
||||
add_unit_test(storage_v2_replication.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_replication mg-storage-v2 fmt)
|
||||
|
||||
add_unit_test(storage_v2_isolation_level.cpp)
|
||||
target_link_libraries(${test_prefix}storage_v2_isolation_level mg-storage-v2)
|
||||
|
||||
# Test mg-storage-v3
|
||||
add_library(storage_v3_test_utils storage_v3_test_utils.cpp)
|
||||
target_link_libraries(storage_v3_test_utils mg-storage-v3)
|
||||
@ -442,3 +399,7 @@ target_link_libraries(${test_prefix}pretty_print_ast_to_original_expression_test
|
||||
# Tests for mg-coordinator
|
||||
add_unit_test(coordinator_shard_map.cpp)
|
||||
target_link_libraries(${test_prefix}coordinator_shard_map mg-coordinator)
|
||||
|
||||
# Tests for many shards, many creates, scan
|
||||
add_unit_test(high_density_shard_create_scan.cpp)
|
||||
target_link_libraries(${test_prefix}high_density_shard_create_scan mg-io mg-coordinator mg-storage-v3 mg-query-v2)
|
||||
|
@ -1,81 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include "storage/v2/commit_log.hpp"
|
||||
|
||||
#include "gtest/gtest.h"
|
||||
|
||||
namespace {
|
||||
inline constexpr size_t ids_per_block = 8192 * 64;
|
||||
} // namespace
|
||||
|
||||
TEST(CommitLog, Simple) {
|
||||
memgraph::storage::CommitLog log;
|
||||
EXPECT_EQ(log.OldestActive(), 0);
|
||||
|
||||
log.MarkFinished(1);
|
||||
EXPECT_EQ(log.OldestActive(), 0);
|
||||
|
||||
log.MarkFinished(0);
|
||||
EXPECT_EQ(log.OldestActive(), 2);
|
||||
}
|
||||
|
||||
TEST(CommitLog, Fields) {
|
||||
memgraph::storage::CommitLog log;
|
||||
|
||||
for (uint64_t i = 0; i < 64; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), i + 1);
|
||||
}
|
||||
|
||||
for (uint64_t i = 128; i < 192; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), 64);
|
||||
}
|
||||
|
||||
for (uint64_t i = 64; i < 128; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), i < 127 ? i + 1 : 192);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(CommitLog, Blocks) {
|
||||
memgraph::storage::CommitLog log;
|
||||
|
||||
for (uint64_t i = 0; i < ids_per_block; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), i + 1);
|
||||
}
|
||||
|
||||
for (uint64_t i = ids_per_block * 2; i < ids_per_block * 3; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), ids_per_block);
|
||||
}
|
||||
|
||||
for (uint64_t i = ids_per_block; i < ids_per_block; ++i) {
|
||||
log.MarkFinished(i);
|
||||
EXPECT_EQ(log.OldestActive(), i < ids_per_block - 1 ? i + 1 : ids_per_block * 3);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(CommitLog, TrackAfterInitialId) {
|
||||
const auto check_marking_ids = [](auto *log, auto current_oldest_active) {
|
||||
ASSERT_EQ(log->OldestActive(), current_oldest_active);
|
||||
log->MarkFinished(current_oldest_active);
|
||||
++current_oldest_active;
|
||||
ASSERT_EQ(log->OldestActive(), current_oldest_active);
|
||||
};
|
||||
|
||||
for (uint64_t i = 0; i < 2 * ids_per_block; ++i) {
|
||||
memgraph::storage::CommitLog log{i};
|
||||
check_marking_ids(&log, i);
|
||||
}
|
||||
}
|
304
tests/unit/high_density_shard_create_scan.cpp
Normal file
304
tests/unit/high_density_shard_create_scan.cpp
Normal file
@ -0,0 +1,304 @@
|
||||
// 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 <chrono>
|
||||
#include <iostream>
|
||||
#include <limits>
|
||||
#include <memory>
|
||||
#include <set>
|
||||
#include <thread>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "coordinator/coordinator_client.hpp"
|
||||
#include "coordinator/coordinator_rsm.hpp"
|
||||
#include "coordinator/shard_map.hpp"
|
||||
#include "io/address.hpp"
|
||||
#include "io/local_transport/local_system.hpp"
|
||||
#include "io/local_transport/local_transport.hpp"
|
||||
#include "io/simulator/simulator.hpp"
|
||||
#include "io/simulator/simulator_config.hpp"
|
||||
#include "io/simulator/simulator_transport.hpp"
|
||||
#include "machine_manager/machine_config.hpp"
|
||||
#include "machine_manager/machine_manager.hpp"
|
||||
#include "query/v2/requests.hpp"
|
||||
#include "query/v2/shard_request_manager.hpp"
|
||||
#include "utils/variant_helpers.hpp"
|
||||
|
||||
namespace memgraph::tests::simulation {
|
||||
|
||||
using coordinator::Coordinator;
|
||||
using coordinator::CoordinatorClient;
|
||||
using coordinator::CoordinatorReadRequests;
|
||||
using coordinator::CoordinatorWriteRequests;
|
||||
using coordinator::CoordinatorWriteResponses;
|
||||
using coordinator::GetShardMapRequest;
|
||||
using coordinator::GetShardMapResponse;
|
||||
using coordinator::Hlc;
|
||||
using coordinator::HlcResponse;
|
||||
using coordinator::Shard;
|
||||
using coordinator::ShardMap;
|
||||
using io::Address;
|
||||
using io::Io;
|
||||
using io::local_transport::LocalSystem;
|
||||
using io::local_transport::LocalTransport;
|
||||
using io::rsm::RsmClient;
|
||||
using machine_manager::MachineConfig;
|
||||
using machine_manager::MachineManager;
|
||||
using msgs::ReadRequests;
|
||||
using msgs::ReadResponses;
|
||||
using msgs::WriteRequests;
|
||||
using msgs::WriteResponses;
|
||||
using storage::v3::LabelId;
|
||||
using storage::v3::SchemaProperty;
|
||||
|
||||
using CompoundKey = std::pair<int, int>;
|
||||
using ShardClient = RsmClient<LocalTransport, WriteRequests, WriteResponses, ReadRequests, ReadResponses>;
|
||||
|
||||
struct CreateVertex {
|
||||
int first;
|
||||
int second;
|
||||
|
||||
friend std::ostream &operator<<(std::ostream &in, const CreateVertex &add) {
|
||||
in << "CreateVertex { first: " << add.first << ", second: " << add.second << " }";
|
||||
return in;
|
||||
}
|
||||
};
|
||||
|
||||
struct ScanAll {
|
||||
friend std::ostream &operator<<(std::ostream &in, const ScanAll &get) {
|
||||
in << "ScanAll {}";
|
||||
return in;
|
||||
}
|
||||
};
|
||||
|
||||
MachineManager<LocalTransport> MkMm(LocalSystem &local_system, std::vector<Address> coordinator_addresses, Address addr,
|
||||
ShardMap shard_map, size_t shard_worker_threads) {
|
||||
MachineConfig config{
|
||||
.coordinator_addresses = std::move(coordinator_addresses),
|
||||
.is_storage = true,
|
||||
.is_coordinator = true,
|
||||
.listen_ip = addr.last_known_ip,
|
||||
.listen_port = addr.last_known_port,
|
||||
.shard_worker_threads = shard_worker_threads,
|
||||
};
|
||||
|
||||
Io<LocalTransport> io = local_system.Register(addr);
|
||||
|
||||
Coordinator coordinator{shard_map};
|
||||
|
||||
return MachineManager{io, config, std::move(coordinator)};
|
||||
}
|
||||
|
||||
void RunMachine(MachineManager<LocalTransport> mm) { mm.Run(); }
|
||||
|
||||
void WaitForShardsToInitialize(CoordinatorClient<LocalTransport> &coordinator_client) {
|
||||
// Call coordinator client's read method for GetShardMap and keep
|
||||
// reading it until the shard map contains proper replicas for
|
||||
// each shard in the label space.
|
||||
|
||||
while (true) {
|
||||
GetShardMapRequest req{};
|
||||
CoordinatorReadRequests read_req = req;
|
||||
auto read_res = coordinator_client.SendReadRequest(read_req);
|
||||
if (read_res.HasError()) {
|
||||
// timed out
|
||||
continue;
|
||||
}
|
||||
auto response_result = read_res.GetValue();
|
||||
auto response = std::get<GetShardMapResponse>(response_result);
|
||||
auto shard_map = response.shard_map;
|
||||
|
||||
if (shard_map.ClusterInitialized()) {
|
||||
spdlog::info("cluster stabilized - beginning workload");
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ShardMap TestShardMap(int shards, int replication_factor, int gap_between_shards) {
|
||||
ShardMap sm{};
|
||||
|
||||
const auto label_name = std::string("test_label");
|
||||
|
||||
// register new properties
|
||||
const std::vector<std::string> property_names = {"property_1", "property_2"};
|
||||
const auto properties = sm.AllocatePropertyIds(property_names);
|
||||
const auto property_id_1 = properties.at("property_1");
|
||||
const auto property_id_2 = properties.at("property_2");
|
||||
const auto type_1 = memgraph::common::SchemaType::INT;
|
||||
const auto type_2 = memgraph::common::SchemaType::INT;
|
||||
|
||||
// register new label space
|
||||
std::vector<SchemaProperty> schema = {
|
||||
SchemaProperty{.property_id = property_id_1, .type = type_1},
|
||||
SchemaProperty{.property_id = property_id_2, .type = type_2},
|
||||
};
|
||||
|
||||
std::optional<LabelId> label_id = sm.InitializeNewLabel(label_name, schema, replication_factor, sm.shard_map_version);
|
||||
MG_ASSERT(label_id.has_value());
|
||||
|
||||
// split the shard at N split points
|
||||
for (int64_t i = 1; i < shards; ++i) {
|
||||
const auto key1 = memgraph::storage::v3::PropertyValue(i * gap_between_shards);
|
||||
const auto key2 = memgraph::storage::v3::PropertyValue(0);
|
||||
|
||||
const auto split_point = {key1, key2};
|
||||
|
||||
const bool split_success = sm.SplitShard(sm.shard_map_version, label_id.value(), split_point);
|
||||
|
||||
MG_ASSERT(split_success);
|
||||
}
|
||||
|
||||
return sm;
|
||||
}
|
||||
|
||||
void ExecuteOp(msgs::ShardRequestManager<LocalTransport> &shard_request_manager,
|
||||
std::set<CompoundKey> &correctness_model, CreateVertex create_vertex) {
|
||||
const auto key1 = memgraph::storage::v3::PropertyValue(create_vertex.first);
|
||||
const auto key2 = memgraph::storage::v3::PropertyValue(create_vertex.second);
|
||||
|
||||
std::vector<msgs::Value> primary_key = {msgs::Value(int64_t(create_vertex.first)),
|
||||
msgs::Value(int64_t(create_vertex.second))};
|
||||
|
||||
if (correctness_model.contains(std::make_pair(create_vertex.first, create_vertex.second))) {
|
||||
// TODO(tyler) remove this early-return when we have properly handled setting non-unique vertexes
|
||||
return;
|
||||
}
|
||||
|
||||
msgs::ExecutionState<msgs::CreateVerticesRequest> state;
|
||||
|
||||
auto label_id = shard_request_manager.NameToLabel("test_label");
|
||||
|
||||
msgs::NewVertex nv{.primary_key = primary_key};
|
||||
nv.label_ids.push_back({label_id});
|
||||
|
||||
std::vector<msgs::NewVertex> new_vertices;
|
||||
new_vertices.push_back(std::move(nv));
|
||||
|
||||
auto result = shard_request_manager.Request(state, std::move(new_vertices));
|
||||
|
||||
MG_ASSERT(result.size() == 1);
|
||||
MG_ASSERT(result[0].success);
|
||||
|
||||
correctness_model.emplace(std::make_pair(create_vertex.first, create_vertex.second));
|
||||
}
|
||||
|
||||
void ExecuteOp(msgs::ShardRequestManager<LocalTransport> &shard_request_manager,
|
||||
std::set<CompoundKey> &correctness_model, ScanAll scan_all) {
|
||||
msgs::ExecutionState<msgs::ScanVerticesRequest> request{.label = "test_label"};
|
||||
|
||||
auto results = shard_request_manager.Request(request);
|
||||
|
||||
MG_ASSERT(results.size() == correctness_model.size());
|
||||
|
||||
for (const auto &vertex_accessor : results) {
|
||||
const auto properties = vertex_accessor.Properties();
|
||||
const auto primary_key = vertex_accessor.Id().second;
|
||||
const CompoundKey model_key = std::make_pair(primary_key[0].int_v, primary_key[1].int_v);
|
||||
MG_ASSERT(correctness_model.contains(model_key));
|
||||
}
|
||||
}
|
||||
|
||||
void RunWorkload(int shards, int replication_factor, int create_ops, int scan_ops, int shard_worker_threads,
|
||||
int gap_between_shards) {
|
||||
spdlog::info("======================== NEW TEST ========================");
|
||||
spdlog::info("shards: ", shards);
|
||||
spdlog::info("replication factor: ", replication_factor);
|
||||
spdlog::info("create ops: ", create_ops);
|
||||
spdlog::info("scan all ops: ", scan_ops);
|
||||
spdlog::info("shard worker threads: ", shard_worker_threads);
|
||||
spdlog::info("gap between shards: ", gap_between_shards);
|
||||
|
||||
LocalSystem local_system;
|
||||
|
||||
auto cli_addr = Address::TestAddress(1);
|
||||
auto machine_1_addr = cli_addr.ForkUniqueAddress();
|
||||
|
||||
Io<LocalTransport> cli_io = local_system.Register(cli_addr);
|
||||
Io<LocalTransport> cli_io_2 = local_system.Register(Address::TestAddress(2));
|
||||
|
||||
auto coordinator_addresses = std::vector{
|
||||
machine_1_addr,
|
||||
};
|
||||
|
||||
auto time_before_shard_map_creation = cli_io_2.Now();
|
||||
ShardMap initialization_sm = TestShardMap(shards, replication_factor, gap_between_shards);
|
||||
auto time_after_shard_map_creation = cli_io_2.Now();
|
||||
|
||||
auto mm_1 = MkMm(local_system, coordinator_addresses, machine_1_addr, initialization_sm, shard_worker_threads);
|
||||
Address coordinator_address = mm_1.CoordinatorAddress();
|
||||
|
||||
auto mm_thread_1 = std::jthread(RunMachine, std::move(mm_1));
|
||||
|
||||
CoordinatorClient<LocalTransport> coordinator_client(cli_io, coordinator_address, {coordinator_address});
|
||||
|
||||
auto time_before_shard_stabilization = cli_io_2.Now();
|
||||
WaitForShardsToInitialize(coordinator_client);
|
||||
auto time_after_shard_stabilization = cli_io_2.Now();
|
||||
|
||||
msgs::ShardRequestManager<LocalTransport> shard_request_manager(std::move(coordinator_client), std::move(cli_io));
|
||||
|
||||
shard_request_manager.StartTransaction();
|
||||
|
||||
auto correctness_model = std::set<CompoundKey>{};
|
||||
|
||||
auto time_before_creates = cli_io_2.Now();
|
||||
|
||||
for (int i = 0; i < create_ops; i++) {
|
||||
ExecuteOp(shard_request_manager, correctness_model, CreateVertex{.first = i, .second = i});
|
||||
}
|
||||
|
||||
auto time_after_creates = cli_io_2.Now();
|
||||
|
||||
for (int i = 0; i < scan_ops; i++) {
|
||||
ExecuteOp(shard_request_manager, correctness_model, ScanAll{});
|
||||
}
|
||||
|
||||
auto time_after_scan = cli_io_2.Now();
|
||||
|
||||
local_system.ShutDown();
|
||||
|
||||
auto latencies = cli_io_2.ResponseLatencies();
|
||||
|
||||
spdlog::info("response latencies: \n{}", latencies.SummaryTable());
|
||||
|
||||
spdlog::info("serial time break-down: (μs)");
|
||||
|
||||
spdlog::info("{: >20}: {: >10}", "split shard map",
|
||||
(time_after_shard_map_creation - time_before_shard_map_creation).count());
|
||||
spdlog::info("{: >20}: {: >10}", "shard stabilization",
|
||||
(time_after_shard_stabilization - time_before_shard_stabilization).count());
|
||||
spdlog::info("{: >20}: {: >10}", "create nodes", (time_after_creates - time_before_creates).count());
|
||||
spdlog::info("{: >20}: {: >10}", "scan nodes", (time_after_scan - time_after_creates).count());
|
||||
|
||||
std::cout << fmt::format("{} {} {}\n", shards, shard_worker_threads, (time_after_scan - time_after_creates).count());
|
||||
}
|
||||
|
||||
TEST(MachineManager, ManyShards) {
|
||||
auto shards_attempts = {1, 64};
|
||||
auto shard_worker_thread_attempts = {1, 32};
|
||||
auto replication_factor = 1;
|
||||
auto create_ops = 128;
|
||||
auto scan_ops = 1;
|
||||
|
||||
std::cout << "splits threads scan_all_microseconds\n";
|
||||
|
||||
for (const auto shards : shards_attempts) {
|
||||
auto gap_between_shards = create_ops / shards;
|
||||
|
||||
for (const auto shard_worker_threads : shard_worker_thread_attempts) {
|
||||
RunWorkload(shards, replication_factor, create_ops, scan_ops, shard_worker_threads, gap_between_shards);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace memgraph::tests::simulation
|
@ -185,7 +185,7 @@ MachineManager<LocalTransport> MkMm(LocalSystem &local_system, std::vector<Addre
|
||||
|
||||
Coordinator coordinator{shard_map};
|
||||
|
||||
return MachineManager{io, config, coordinator, shard_map};
|
||||
return MachineManager{io, config, coordinator};
|
||||
}
|
||||
|
||||
void RunMachine(MachineManager<LocalTransport> mm) { mm.Run(); }
|
||||
|
@ -1,814 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
#include "storage/v2/property_value.hpp"
|
||||
#include "storage/v2/temporal.hpp"
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Null) {
|
||||
memgraph::storage::PropertyValue pv;
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Null);
|
||||
|
||||
ASSERT_TRUE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "null");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "null");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Bool) {
|
||||
memgraph::storage::PropertyValue pv(false);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Bool);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_TRUE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_EQ(pv.ValueBool(), false);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_EQ(cpv.ValueBool(), false);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "bool");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "false");
|
||||
}
|
||||
{
|
||||
memgraph::storage::PropertyValue pvtrue(true);
|
||||
std::stringstream ss;
|
||||
ss << pvtrue;
|
||||
ASSERT_EQ(ss.str(), "true");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Int) {
|
||||
memgraph::storage::PropertyValue pv(123L);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Int);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_TRUE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(pv.ValueInt(), 123L);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(cpv.ValueInt(), 123L);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "int");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "123");
|
||||
}
|
||||
|
||||
{
|
||||
memgraph::storage::PropertyValue pvint(123);
|
||||
ASSERT_EQ(pvint.type(), memgraph::storage::PropertyValue::Type::Int);
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Double) {
|
||||
memgraph::storage::PropertyValue pv(123.5);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Double);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_TRUE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(pv.ValueDouble(), 123.5);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(cpv.ValueDouble(), 123.5);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "double");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "123.5");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, StringCopy) {
|
||||
std::string str("nandare");
|
||||
memgraph::storage::PropertyValue pv(str);
|
||||
|
||||
ASSERT_EQ(str, "nandare");
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::String);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_TRUE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(pv.ValueString(), "nandare");
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(cpv.ValueString(), "nandare");
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "string");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "nandare");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, StringMove) {
|
||||
std::string str("nandare");
|
||||
memgraph::storage::PropertyValue pv(std::move(str));
|
||||
|
||||
ASSERT_EQ(str, "");
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::String);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_TRUE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(pv.ValueString(), "nandare");
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_EQ(cpv.ValueString(), "nandare");
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "string");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "nandare");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, ListCopy) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
memgraph::storage::PropertyValue pv(vec);
|
||||
|
||||
ASSERT_EQ(vec.size(), 2);
|
||||
ASSERT_EQ(vec[0].ValueString(), "nandare");
|
||||
ASSERT_EQ(vec[1].ValueInt(), 123);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::List);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_TRUE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = pv.ValueList();
|
||||
ASSERT_EQ(ret.size(), 2);
|
||||
ASSERT_EQ(ret[0].ValueString(), "nandare");
|
||||
ASSERT_EQ(ret[1].ValueInt(), 123);
|
||||
}
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = cpv.ValueList();
|
||||
ASSERT_EQ(ret.size(), 2);
|
||||
ASSERT_EQ(ret[0].ValueString(), "nandare");
|
||||
ASSERT_EQ(ret[1].ValueInt(), 123);
|
||||
}
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "list");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "[nandare, 123]");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, ListMove) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
memgraph::storage::PropertyValue pv(std::move(vec));
|
||||
|
||||
ASSERT_EQ(vec.size(), 0);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::List);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_TRUE(pv.IsList());
|
||||
ASSERT_FALSE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = pv.ValueList();
|
||||
ASSERT_EQ(ret.size(), 2);
|
||||
ASSERT_EQ(ret[0].ValueString(), "nandare");
|
||||
ASSERT_EQ(ret[1].ValueInt(), 123);
|
||||
}
|
||||
ASSERT_THROW(pv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = cpv.ValueList();
|
||||
ASSERT_EQ(ret.size(), 2);
|
||||
ASSERT_EQ(ret[0].ValueString(), "nandare");
|
||||
ASSERT_EQ(ret[1].ValueInt(), 123);
|
||||
}
|
||||
ASSERT_THROW(cpv.ValueMap(), memgraph::storage::PropertyValueException);
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "list");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "[nandare, 123]");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, MapCopy) {
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(123)}};
|
||||
memgraph::storage::PropertyValue pv(map);
|
||||
|
||||
ASSERT_EQ(map.size(), 1);
|
||||
ASSERT_EQ(map.at("nandare").ValueInt(), 123);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Map);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_TRUE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = pv.ValueMap();
|
||||
ASSERT_EQ(ret.size(), 1);
|
||||
ASSERT_EQ(ret.at("nandare").ValueInt(), 123);
|
||||
}
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = cpv.ValueMap();
|
||||
ASSERT_EQ(ret.size(), 1);
|
||||
ASSERT_EQ(ret.at("nandare").ValueInt(), 123);
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "map");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "{nandare: 123}");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, MapMove) {
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(123)}};
|
||||
memgraph::storage::PropertyValue pv(std::move(map));
|
||||
|
||||
ASSERT_EQ(map.size(), 0);
|
||||
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::Map);
|
||||
|
||||
ASSERT_FALSE(pv.IsNull());
|
||||
ASSERT_FALSE(pv.IsBool());
|
||||
ASSERT_FALSE(pv.IsInt());
|
||||
ASSERT_FALSE(pv.IsDouble());
|
||||
ASSERT_FALSE(pv.IsString());
|
||||
ASSERT_FALSE(pv.IsList());
|
||||
ASSERT_TRUE(pv.IsMap());
|
||||
|
||||
ASSERT_THROW(pv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(pv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = pv.ValueMap();
|
||||
ASSERT_EQ(ret.size(), 1);
|
||||
ASSERT_EQ(ret.at("nandare").ValueInt(), 123);
|
||||
}
|
||||
|
||||
const auto &cpv = pv;
|
||||
|
||||
ASSERT_THROW(cpv.ValueBool(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueInt(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueDouble(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueString(), memgraph::storage::PropertyValueException);
|
||||
ASSERT_THROW(cpv.ValueList(), memgraph::storage::PropertyValueException);
|
||||
{
|
||||
const auto &ret = cpv.ValueMap();
|
||||
ASSERT_EQ(ret.size(), 1);
|
||||
ASSERT_EQ(ret.at("nandare").ValueInt(), 123);
|
||||
}
|
||||
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv.type();
|
||||
ASSERT_EQ(ss.str(), "map");
|
||||
}
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << pv;
|
||||
ASSERT_EQ(ss.str(), "{nandare: 123}");
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, CopyConstructor) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))};
|
||||
|
||||
for (const auto &item : data) {
|
||||
memgraph::storage::PropertyValue pv(item);
|
||||
ASSERT_EQ(pv.type(), item.type());
|
||||
switch (item.type()) {
|
||||
case memgraph::storage::PropertyValue::Type::Null:
|
||||
ASSERT_TRUE(pv.IsNull());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Bool:
|
||||
ASSERT_EQ(pv.ValueBool(), item.ValueBool());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Int:
|
||||
ASSERT_EQ(pv.ValueInt(), item.ValueInt());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Double:
|
||||
ASSERT_EQ(pv.ValueDouble(), item.ValueDouble());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::String:
|
||||
ASSERT_EQ(pv.ValueString(), item.ValueString());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::List:
|
||||
ASSERT_EQ(pv.ValueList(), item.ValueList());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Map:
|
||||
ASSERT_EQ(pv.ValueMap(), item.ValueMap());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::TemporalData:
|
||||
ASSERT_EQ(pv.ValueTemporalData(), item.ValueTemporalData());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, MoveConstructor) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))};
|
||||
|
||||
for (auto &item : data) {
|
||||
memgraph::storage::PropertyValue copy(item);
|
||||
memgraph::storage::PropertyValue pv(std::move(item));
|
||||
ASSERT_EQ(item.type(), memgraph::storage::PropertyValue::Type::Null);
|
||||
ASSERT_EQ(pv.type(), copy.type());
|
||||
switch (copy.type()) {
|
||||
case memgraph::storage::PropertyValue::Type::Null:
|
||||
ASSERT_TRUE(pv.IsNull());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Bool:
|
||||
ASSERT_EQ(pv.ValueBool(), copy.ValueBool());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Int:
|
||||
ASSERT_EQ(pv.ValueInt(), copy.ValueInt());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Double:
|
||||
ASSERT_EQ(pv.ValueDouble(), copy.ValueDouble());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::String:
|
||||
ASSERT_EQ(pv.ValueString(), copy.ValueString());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::List:
|
||||
ASSERT_EQ(pv.ValueList(), copy.ValueList());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Map:
|
||||
ASSERT_EQ(pv.ValueMap(), copy.ValueMap());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::TemporalData:
|
||||
ASSERT_EQ(pv.ValueTemporalData(), copy.ValueTemporalData());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, CopyAssignment) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))};
|
||||
|
||||
for (const auto &item : data) {
|
||||
memgraph::storage::PropertyValue pv(123);
|
||||
pv = item;
|
||||
ASSERT_EQ(pv.type(), item.type());
|
||||
switch (item.type()) {
|
||||
case memgraph::storage::PropertyValue::Type::Null:
|
||||
ASSERT_TRUE(pv.IsNull());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Bool:
|
||||
ASSERT_EQ(pv.ValueBool(), item.ValueBool());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Int:
|
||||
ASSERT_EQ(pv.ValueInt(), item.ValueInt());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Double:
|
||||
ASSERT_EQ(pv.ValueDouble(), item.ValueDouble());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::String:
|
||||
ASSERT_EQ(pv.ValueString(), item.ValueString());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::List:
|
||||
ASSERT_EQ(pv.ValueList(), item.ValueList());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Map:
|
||||
ASSERT_EQ(pv.ValueMap(), item.ValueMap());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::TemporalData:
|
||||
ASSERT_EQ(pv.ValueTemporalData(), item.ValueTemporalData());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, MoveAssignment) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))};
|
||||
|
||||
for (auto &item : data) {
|
||||
memgraph::storage::PropertyValue copy(item);
|
||||
memgraph::storage::PropertyValue pv(123);
|
||||
pv = std::move(item);
|
||||
ASSERT_EQ(item.type(), memgraph::storage::PropertyValue::Type::Null);
|
||||
ASSERT_EQ(pv.type(), copy.type());
|
||||
switch (copy.type()) {
|
||||
case memgraph::storage::PropertyValue::Type::Null:
|
||||
ASSERT_TRUE(pv.IsNull());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Bool:
|
||||
ASSERT_EQ(pv.ValueBool(), copy.ValueBool());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Int:
|
||||
ASSERT_EQ(pv.ValueInt(), copy.ValueInt());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Double:
|
||||
ASSERT_EQ(pv.ValueDouble(), copy.ValueDouble());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::String:
|
||||
ASSERT_EQ(pv.ValueString(), copy.ValueString());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::List:
|
||||
ASSERT_EQ(pv.ValueList(), copy.ValueList());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::Map:
|
||||
ASSERT_EQ(pv.ValueMap(), copy.ValueMap());
|
||||
break;
|
||||
case memgraph::storage::PropertyValue::Type::TemporalData:
|
||||
ASSERT_EQ(pv.ValueTemporalData(), copy.ValueTemporalData());
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, CopyAssignmentSelf) {
|
||||
memgraph::storage::PropertyValue pv("nandare");
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wself-assign-overloaded"
|
||||
pv = pv;
|
||||
#pragma clang diagnostic pop
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::String);
|
||||
ASSERT_EQ(pv.ValueString(), "nandare");
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, MoveAssignmentSelf) {
|
||||
memgraph::storage::PropertyValue pv("nandare");
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wself-move"
|
||||
pv = std::move(pv);
|
||||
#pragma clang diagnostic pop
|
||||
ASSERT_EQ(pv.type(), memgraph::storage::PropertyValue::Type::String);
|
||||
ASSERT_EQ(pv.ValueString(), "nandare");
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Equal) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"), memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map)};
|
||||
for (const auto &item1 : data) {
|
||||
for (const auto &item2 : data) {
|
||||
if (item1.type() == item2.type()) {
|
||||
ASSERT_TRUE(item1 == item2);
|
||||
} else {
|
||||
ASSERT_FALSE(item1 == item2);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(PropertyValue, Less) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123)};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"), memgraph::storage::PropertyValue(vec),
|
||||
memgraph::storage::PropertyValue(map)};
|
||||
for (size_t i = 0; i < data.size(); ++i) {
|
||||
for (size_t j = 0; j < data.size(); ++j) {
|
||||
auto item1 = data[i];
|
||||
auto item2 = data[j];
|
||||
if (i < j) {
|
||||
ASSERT_TRUE(item1 < item2);
|
||||
} else {
|
||||
ASSERT_FALSE(item1 < item2);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyValue, NumeralTypesComparison) {
|
||||
auto v_int = memgraph::storage::PropertyValue(2);
|
||||
auto v_double = memgraph::storage::PropertyValue(2.0);
|
||||
ASSERT_TRUE(v_int.IsInt());
|
||||
ASSERT_TRUE(v_double.IsDouble());
|
||||
ASSERT_TRUE(v_int == v_double);
|
||||
ASSERT_FALSE(v_int < v_double);
|
||||
ASSERT_FALSE(v_double < v_int);
|
||||
}
|
||||
|
||||
TEST(PropertyValue, NestedNumeralTypesComparison) {
|
||||
auto v1 = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(1)});
|
||||
auto v2 = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(1.5)});
|
||||
auto v3 = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(2)});
|
||||
|
||||
auto v1alt = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(1.0)});
|
||||
auto v3alt = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(2.0)});
|
||||
|
||||
ASSERT_TRUE(v1 == v1alt);
|
||||
ASSERT_TRUE(v3 == v3alt);
|
||||
|
||||
ASSERT_FALSE(v1 == v2);
|
||||
ASSERT_FALSE(v2 == v1);
|
||||
ASSERT_FALSE(v2 == v3);
|
||||
ASSERT_FALSE(v3 == v2);
|
||||
ASSERT_FALSE(v1 == v3);
|
||||
ASSERT_FALSE(v3 == v1);
|
||||
|
||||
ASSERT_TRUE(v1 < v2);
|
||||
ASSERT_TRUE(v2 < v3);
|
||||
ASSERT_TRUE(v1 < v3);
|
||||
ASSERT_FALSE(v2 < v1);
|
||||
ASSERT_FALSE(v3 < v2);
|
||||
ASSERT_FALSE(v3 < v1);
|
||||
|
||||
ASSERT_TRUE(v1alt < v2);
|
||||
ASSERT_TRUE(v2 < v3alt);
|
||||
ASSERT_TRUE(v1alt < v3alt);
|
||||
ASSERT_FALSE(v2 < v1alt);
|
||||
ASSERT_FALSE(v3alt < v2);
|
||||
ASSERT_FALSE(v3 < v1alt);
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -1,973 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "storage/v2/storage.hpp"
|
||||
|
||||
// NOLINTNEXTLINE(google-build-using-namespace)
|
||||
using namespace memgraph::storage;
|
||||
|
||||
using testing::UnorderedElementsAre;
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define ASSERT_NO_ERROR(result) ASSERT_FALSE((result).HasError())
|
||||
|
||||
class ConstraintsTest : public testing::Test {
|
||||
protected:
|
||||
ConstraintsTest()
|
||||
: prop1(storage.NameToProperty("prop1")),
|
||||
prop2(storage.NameToProperty("prop2")),
|
||||
label1(storage.NameToLabel("label1")),
|
||||
label2(storage.NameToLabel("label2")) {}
|
||||
|
||||
Storage storage;
|
||||
PropertyId prop1;
|
||||
PropertyId prop2;
|
||||
LabelId label1;
|
||||
LabelId label2;
|
||||
};
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, ExistenceConstraintsCreateAndDrop) {
|
||||
EXPECT_EQ(storage.ListAllConstraints().existence.size(), 0);
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(label1, prop1)));
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && !res.GetValue());
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(label1, prop1)));
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label2, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().existence,
|
||||
UnorderedElementsAre(std::make_pair(label1, prop1), std::make_pair(label2, prop1)));
|
||||
EXPECT_TRUE(storage.DropExistenceConstraint(label1, prop1));
|
||||
EXPECT_FALSE(storage.DropExistenceConstraint(label1, prop1));
|
||||
EXPECT_THAT(storage.ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(label2, prop1)));
|
||||
EXPECT_TRUE(storage.DropExistenceConstraint(label2, prop1));
|
||||
EXPECT_FALSE(storage.DropExistenceConstraint(label2, prop2));
|
||||
EXPECT_EQ(storage.ListAllConstraints().existence.size(), 0);
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label2, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().existence, UnorderedElementsAre(std::make_pair(label2, prop1)));
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, ExistenceConstraintsCreateFailure1) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&vertex));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, ExistenceConstraintsCreateFailure2) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, ExistenceConstraintsViolationOnCommit) {
|
||||
{
|
||||
auto res = storage.CreateExistenceConstraint(label1, prop1);
|
||||
ASSERT_TRUE(res.HasValue() && res.GetValue());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue()));
|
||||
}
|
||||
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::EXISTENCE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue()));
|
||||
}
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&vertex));
|
||||
}
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
ASSERT_TRUE(storage.DropExistenceConstraint(label1, prop1));
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsCreateAndDropAndList) {
|
||||
EXPECT_EQ(storage.ListAllConstraints().unique.size(), 0);
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
EXPECT_TRUE(res.HasValue());
|
||||
EXPECT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique,
|
||||
UnorderedElementsAre(std::make_pair(label1, std::set<PropertyId>{prop1})));
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
EXPECT_TRUE(res.HasValue());
|
||||
EXPECT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::ALREADY_EXISTS);
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique,
|
||||
UnorderedElementsAre(std::make_pair(label1, std::set<PropertyId>{prop1})));
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label2, {prop1});
|
||||
EXPECT_TRUE(res.HasValue() && res.GetValue() == UniqueConstraints::CreationStatus::SUCCESS);
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique,
|
||||
UnorderedElementsAre(std::make_pair(label1, std::set<PropertyId>{prop1}),
|
||||
std::make_pair(label2, std::set<PropertyId>{prop1})));
|
||||
EXPECT_EQ(storage.DropUniqueConstraint(label1, {prop1}), UniqueConstraints::DeletionStatus::SUCCESS);
|
||||
EXPECT_EQ(storage.DropUniqueConstraint(label1, {prop1}), UniqueConstraints::DeletionStatus::NOT_FOUND);
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique,
|
||||
UnorderedElementsAre(std::make_pair(label2, std::set<PropertyId>{prop1})));
|
||||
EXPECT_EQ(storage.DropUniqueConstraint(label2, {prop1}), UniqueConstraints::DeletionStatus::SUCCESS);
|
||||
EXPECT_EQ(storage.DropUniqueConstraint(label2, {prop2}), UniqueConstraints::DeletionStatus::NOT_FOUND);
|
||||
EXPECT_EQ(storage.ListAllConstraints().unique.size(), 0);
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label2, {prop1});
|
||||
EXPECT_TRUE(res.HasValue());
|
||||
EXPECT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique,
|
||||
UnorderedElementsAre(std::make_pair(label2, std::set<PropertyId>{prop1})));
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsCreateFailure1) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&vertex));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsCreateFailure2) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
int value = 0;
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(value)));
|
||||
++value;
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsNoViolation1) {
|
||||
Gid gid1;
|
||||
Gid gid2;
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
auto vertex2 = acc.CreateVertex();
|
||||
gid1 = vertex1.Gid();
|
||||
gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc.FindVertex(gid2, View::OLD);
|
||||
|
||||
ASSERT_NO_ERROR(vertex1->SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex2->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop2, PropertyValue(3)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc.FindVertex(gid2, View::OLD);
|
||||
ASSERT_NO_ERROR(vertex1->SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsNoViolation2) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---SP(v1, 1)---SP(v1, 2)---OK--
|
||||
// tx2: -B---SP(v2, 2)---SP(v2, 1)---OK-
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto acc2 = storage.Access();
|
||||
auto vertex1 = acc1.CreateVertex();
|
||||
auto vertex2 = acc2.CreateVertex();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsNoViolation3) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---SP(v1, 1)---OK----------------------
|
||||
// tx2: --------------------B---SP(v1, 2)---OK--
|
||||
// tx3: ---------------------B---SP(v2, 1)---OK-
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto vertex1 = acc1.CreateVertex();
|
||||
auto gid = vertex1.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
|
||||
auto acc2 = storage.Access();
|
||||
auto acc3 = storage.Access();
|
||||
auto vertex2 = acc2.FindVertex(gid, View::NEW); // vertex1 == vertex2
|
||||
auto vertex3 = acc3.CreateVertex();
|
||||
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex3.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex3.SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
ASSERT_NO_ERROR(acc3.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsNoViolation4) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---SP(v1, 1)---OK-----------------------
|
||||
// tx2: --------------------B---SP(v2, 1)-----OK-
|
||||
// tx3: ---------------------B---SP(v1, 2)---OK--
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto vertex1 = acc1.CreateVertex();
|
||||
auto gid = vertex1.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
|
||||
auto acc2 = storage.Access();
|
||||
auto acc3 = storage.Access();
|
||||
auto vertex2 = acc2.CreateVertex();
|
||||
auto vertex3 = acc3.FindVertex(gid, View::NEW);
|
||||
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex3->SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(acc3.Commit());
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsViolationOnCommit1) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
auto vertex2 = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(1)));
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsViolationOnCommit2) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---SP(v1, 1)---SP(v2, 2)---OK-----------------------
|
||||
// tx2: -------------------------------B---SP(v1, 3)---OK----
|
||||
// tx3: --------------------------------B---SP(v2, 3)---FAIL-
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto vertex1 = acc1.CreateVertex();
|
||||
auto vertex2 = acc1.CreateVertex();
|
||||
auto gid1 = vertex1.Gid();
|
||||
auto gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
|
||||
auto acc2 = storage.Access();
|
||||
auto acc3 = storage.Access();
|
||||
auto vertex3 = acc2.FindVertex(gid1, View::NEW); // vertex3 == vertex1
|
||||
auto vertex4 = acc3.FindVertex(gid2, View::NEW); // vertex4 == vertex2
|
||||
|
||||
ASSERT_NO_ERROR(vertex3->SetProperty(prop1, PropertyValue(3)));
|
||||
ASSERT_NO_ERROR(vertex4->SetProperty(prop1, PropertyValue(3)));
|
||||
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
auto res = acc3.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsViolationOnCommit3) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---SP(v1, 1)---SP(v2, 2)---OK-----------------------
|
||||
// tx2: -------------------------------B---SP(v1, 2)---FAIL--
|
||||
// tx3: --------------------------------B---SP(v2, 1)---FAIL-
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto vertex1 = acc1.CreateVertex();
|
||||
auto vertex2 = acc1.CreateVertex();
|
||||
auto gid1 = vertex1.Gid();
|
||||
auto gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
|
||||
auto acc2 = storage.Access();
|
||||
auto acc3 = storage.Access();
|
||||
auto vertex3 = acc2.FindVertex(gid1, View::OLD); // vertex3 == vertex1
|
||||
auto vertex4 = acc3.FindVertex(gid2, View::OLD); // vertex4 == vertex2
|
||||
|
||||
// Setting `prop2` shouldn't affect the remaining code.
|
||||
ASSERT_NO_ERROR(vertex3->SetProperty(prop2, PropertyValue(3)));
|
||||
ASSERT_NO_ERROR(vertex4->SetProperty(prop2, PropertyValue(3)));
|
||||
|
||||
ASSERT_NO_ERROR(vertex3->SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex4->SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
auto res = acc2.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
res = acc3.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1}}));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsLabelAlteration) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
Gid gid1;
|
||||
Gid gid2;
|
||||
{
|
||||
// B---AL(v2)---SP(v1, 1)---SP(v2, 1)---OK
|
||||
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
auto vertex2 = acc.CreateVertex();
|
||||
gid1 = vertex1.Gid();
|
||||
gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label2));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(1)));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---AL(v1)-----OK-
|
||||
// tx2: -B---RL(v2)---OK--
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto acc2 = storage.Access();
|
||||
auto vertex1 = acc1.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc2.FindVertex(gid2, View::OLD);
|
||||
|
||||
ASSERT_NO_ERROR(vertex1->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->RemoveLabel(label1));
|
||||
|
||||
// Reapplying labels shouldn't affect the remaining code.
|
||||
ASSERT_NO_ERROR(vertex1->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1->RemoveLabel(label2));
|
||||
|
||||
// Commit the second transaction.
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
|
||||
// Reapplying labels after first commit shouldn't affect the remaining code.
|
||||
ASSERT_NO_ERROR(vertex1->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1->AddLabel(label1));
|
||||
|
||||
// Commit the first transaction.
|
||||
ASSERT_NO_ERROR(acc1.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
// B---AL(v2)---FAIL
|
||||
|
||||
auto acc = storage.Access();
|
||||
auto vertex2 = acc.FindVertex(gid2, View::OLD);
|
||||
ASSERT_NO_ERROR(vertex2->AddLabel(label1));
|
||||
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(), (ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set{prop1}}));
|
||||
}
|
||||
|
||||
{
|
||||
// B---RL(v1)---OK
|
||||
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.FindVertex(gid1, View::OLD);
|
||||
ASSERT_NO_ERROR(vertex1->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
// tx1: B---AL(v1)-----FAIL
|
||||
// tx2: -B---AL(v2)---OK---
|
||||
|
||||
auto acc1 = storage.Access();
|
||||
auto acc2 = storage.Access();
|
||||
auto vertex1 = acc1.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc2.FindVertex(gid2, View::OLD);
|
||||
|
||||
ASSERT_NO_ERROR(vertex1->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->AddLabel(label1));
|
||||
|
||||
// Reapply everything.
|
||||
ASSERT_NO_ERROR(vertex1->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->RemoveLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1->AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2->AddLabel(label1));
|
||||
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
|
||||
auto res = acc1.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(), (ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set{prop1}}));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsPropertySetSize) {
|
||||
{
|
||||
// This should fail since unique constraint cannot be created for an empty
|
||||
// property set.
|
||||
auto res = storage.CreateUniqueConstraint(label1, {});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::EMPTY_PROPERTIES);
|
||||
}
|
||||
|
||||
// Removing a constraint with empty property set should also fail.
|
||||
ASSERT_EQ(storage.DropUniqueConstraint(label1, {}), UniqueConstraints::DeletionStatus::EMPTY_PROPERTIES);
|
||||
|
||||
// Create a set of 33 properties.
|
||||
std::set<PropertyId> properties;
|
||||
for (int i = 1; i <= 33; ++i) {
|
||||
properties.insert(storage.NameToProperty("prop" + std::to_string(i)));
|
||||
}
|
||||
|
||||
{
|
||||
// This should fail since list of properties exceeds the maximum number of
|
||||
// properties, which is 32.
|
||||
auto res = storage.CreateUniqueConstraint(label1, properties);
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED);
|
||||
}
|
||||
|
||||
// An attempt to delete constraint with too large property set should fail.
|
||||
ASSERT_EQ(storage.DropUniqueConstraint(label1, properties),
|
||||
UniqueConstraints::DeletionStatus::PROPERTIES_SIZE_LIMIT_EXCEEDED);
|
||||
|
||||
// Remove one property from the set.
|
||||
properties.erase(properties.begin());
|
||||
|
||||
{
|
||||
// Creating a constraint for 32 properties should succeed.
|
||||
auto res = storage.CreateUniqueConstraint(label1, properties);
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
EXPECT_THAT(storage.ListAllConstraints().unique, UnorderedElementsAre(std::make_pair(label1, properties)));
|
||||
|
||||
// Removing a constraint with 32 properties should succeed.
|
||||
ASSERT_EQ(storage.DropUniqueConstraint(label1, properties), UniqueConstraints::DeletionStatus::SUCCESS);
|
||||
ASSERT_TRUE(storage.ListAllConstraints().unique.empty());
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsMultipleProperties) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
// An attempt to create an existing unique constraint.
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop2, prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
Gid gid1;
|
||||
Gid gid2;
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
auto vertex2 = acc.CreateVertex();
|
||||
gid1 = vertex1.Gid();
|
||||
gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop2, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop2, PropertyValue(3)));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
// Try to change property of the second vertex so it becomes the same as the
|
||||
// first vertex. It should fail.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex2 = acc.FindVertex(gid2, View::OLD);
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop2, PropertyValue(2)));
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(),
|
||||
(ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set<PropertyId>{prop1, prop2}}));
|
||||
}
|
||||
|
||||
// Then change the second property of both vertex to null. Property values of
|
||||
// both vertices should now be equal. However, this operation should succeed
|
||||
// since null value is treated as non-existing property.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc.FindVertex(gid2, View::OLD);
|
||||
ASSERT_NO_ERROR(vertex1->SetProperty(prop2, PropertyValue()));
|
||||
ASSERT_NO_ERROR(vertex2->SetProperty(prop2, PropertyValue()));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsInsertAbortInsert) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
acc.Abort();
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsInsertRemoveInsert) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
Gid gid;
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
gid = vertex.Gid();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.FindVertex(gid, View::OLD);
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&*vertex));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsInsertRemoveAbortInsert) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
Gid gid;
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
gid = vertex.Gid();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.FindVertex(gid, View::OLD);
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&*vertex));
|
||||
acc.Abort();
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
auto res = acc.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(), (ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set{prop1, prop2}}));
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsDeleteVertexSetProperty) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
Gid gid1;
|
||||
Gid gid2;
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex1 = acc.CreateVertex();
|
||||
auto vertex2 = acc.CreateVertex();
|
||||
gid1 = vertex1.Gid();
|
||||
gid2 = vertex2.Gid();
|
||||
|
||||
ASSERT_NO_ERROR(vertex1.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex2.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex1.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex2.SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc1 = storage.Access();
|
||||
auto acc2 = storage.Access();
|
||||
auto vertex1 = acc1.FindVertex(gid1, View::OLD);
|
||||
auto vertex2 = acc2.FindVertex(gid2, View::OLD);
|
||||
|
||||
ASSERT_NO_ERROR(acc2.DeleteVertex(&*vertex2));
|
||||
ASSERT_NO_ERROR(vertex1->SetProperty(prop1, PropertyValue(2)));
|
||||
|
||||
auto res = acc1.Commit();
|
||||
ASSERT_TRUE(res.HasError());
|
||||
EXPECT_EQ(res.GetError(), (ConstraintViolation{ConstraintViolation::Type::UNIQUE, label1, std::set{prop1}}));
|
||||
|
||||
ASSERT_NO_ERROR(acc2.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsInsertDropInsert) {
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
ASSERT_EQ(storage.DropUniqueConstraint(label1, {prop2, prop1}), UniqueConstraints::DeletionStatus::SUCCESS);
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ConstraintsTest, UniqueConstraintsComparePropertyValues) {
|
||||
// Purpose of this test is to make sure that extracted property values
|
||||
// are correctly compared.
|
||||
|
||||
{
|
||||
auto res = storage.CreateUniqueConstraint(label1, {prop1, prop2});
|
||||
ASSERT_TRUE(res.HasValue());
|
||||
ASSERT_EQ(res.GetValue(), UniqueConstraints::CreationStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(1)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(2)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto vertex = acc.CreateVertex();
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop2, PropertyValue(0)));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop1, PropertyValue(3)));
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
@ -1,461 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <limits>
|
||||
|
||||
#include "storage/v2/durability/serialization.hpp"
|
||||
#include "storage/v2/property_value.hpp"
|
||||
#include "storage/v2/temporal.hpp"
|
||||
|
||||
static const std::string kTestMagic{"MGtest"};
|
||||
static const uint64_t kTestVersion{1};
|
||||
|
||||
class DecoderEncoderTest : public ::testing::Test {
|
||||
public:
|
||||
void SetUp() override { Clear(); }
|
||||
|
||||
void TearDown() override { Clear(); }
|
||||
|
||||
std::filesystem::path storage_file{std::filesystem::temp_directory_path() /
|
||||
"MG_test_unit_storage_v2_decoder_encoder.bin"};
|
||||
|
||||
std::filesystem::path alternate_file{std::filesystem::temp_directory_path() /
|
||||
"MG_test_unit_storage_v2_decoder_encoder_alternate.bin"};
|
||||
|
||||
private:
|
||||
void Clear() {
|
||||
if (std::filesystem::exists(storage_file)) {
|
||||
std::filesystem::remove(storage_file);
|
||||
}
|
||||
if (std::filesystem::exists(alternate_file)) {
|
||||
std::filesystem::remove(alternate_file);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(DecoderEncoderTest, ReadMarker) {
|
||||
{
|
||||
memgraph::storage::durability::Encoder encoder;
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion);
|
||||
for (const auto &item : memgraph::storage::durability::kMarkersAll) {
|
||||
encoder.WriteMarker(item);
|
||||
}
|
||||
{
|
||||
uint8_t invalid = 1;
|
||||
encoder.Write(&invalid, sizeof(invalid));
|
||||
}
|
||||
encoder.Finalize();
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
for (const auto &item : memgraph::storage::durability::kMarkersAll) {
|
||||
auto decoded = decoder.ReadMarker();
|
||||
ASSERT_TRUE(decoded);
|
||||
ASSERT_EQ(*decoded, item);
|
||||
}
|
||||
ASSERT_FALSE(decoder.ReadMarker());
|
||||
ASSERT_FALSE(decoder.ReadMarker());
|
||||
auto pos = decoder.GetPosition();
|
||||
ASSERT_TRUE(pos);
|
||||
ASSERT_EQ(pos, decoder.GetSize());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define GENERATE_READ_TEST(name, type, ...) \
|
||||
TEST_F(DecoderEncoderTest, Read##name) { \
|
||||
std::vector<type> dataset{__VA_ARGS__}; \
|
||||
{ \
|
||||
memgraph::storage::durability::Encoder encoder; \
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion); \
|
||||
for (const auto &item : dataset) { \
|
||||
encoder.Write##name(item); \
|
||||
} \
|
||||
{ \
|
||||
uint8_t invalid = 1; \
|
||||
encoder.Write(&invalid, sizeof(invalid)); \
|
||||
} \
|
||||
encoder.Finalize(); \
|
||||
} \
|
||||
{ \
|
||||
memgraph::storage::durability::Decoder decoder; \
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic); \
|
||||
ASSERT_TRUE(version); \
|
||||
ASSERT_EQ(*version, kTestVersion); \
|
||||
for (const auto &item : dataset) { \
|
||||
auto decoded = decoder.Read##name(); \
|
||||
ASSERT_TRUE(decoded); \
|
||||
ASSERT_EQ(*decoded, item); \
|
||||
} \
|
||||
ASSERT_FALSE(decoder.Read##name()); \
|
||||
ASSERT_FALSE(decoder.Read##name()); \
|
||||
auto pos = decoder.GetPosition(); \
|
||||
ASSERT_TRUE(pos); \
|
||||
ASSERT_EQ(pos, decoder.GetSize()); \
|
||||
} \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_READ_TEST(Bool, bool, false, true);
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_READ_TEST(Uint, uint64_t, 0, 1, 1000, 123123123, std::numeric_limits<uint64_t>::max());
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_READ_TEST(Double, double, 1.123, 3.1415926535, 0, -505.505, std::numeric_limits<double>::infinity(),
|
||||
-std::numeric_limits<double>::infinity());
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_READ_TEST(String, std::string, "hello", "world", "nandare", "haihaihai", std::string(),
|
||||
std::string(100000, 'a'));
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_READ_TEST(
|
||||
PropertyValue, memgraph::storage::PropertyValue, memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(false), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123L), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue("nandare"), memgraph::storage::PropertyValue(123L)}),
|
||||
memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"nandare", memgraph::storage::PropertyValue(123)}}),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23)));
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define GENERATE_SKIP_TEST(name, type, ...) \
|
||||
TEST_F(DecoderEncoderTest, Skip##name) { \
|
||||
std::vector<type> dataset{__VA_ARGS__}; \
|
||||
{ \
|
||||
memgraph::storage::durability::Encoder encoder; \
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion); \
|
||||
for (const auto &item : dataset) { \
|
||||
encoder.Write##name(item); \
|
||||
} \
|
||||
{ \
|
||||
uint8_t invalid = 1; \
|
||||
encoder.Write(&invalid, sizeof(invalid)); \
|
||||
} \
|
||||
encoder.Finalize(); \
|
||||
} \
|
||||
{ \
|
||||
memgraph::storage::durability::Decoder decoder; \
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic); \
|
||||
ASSERT_TRUE(version); \
|
||||
ASSERT_EQ(*version, kTestVersion); \
|
||||
for (auto it = dataset.begin(); it != dataset.end(); ++it) { \
|
||||
ASSERT_TRUE(decoder.Skip##name()); \
|
||||
} \
|
||||
ASSERT_FALSE(decoder.Skip##name()); \
|
||||
ASSERT_FALSE(decoder.Skip##name()); \
|
||||
auto pos = decoder.GetPosition(); \
|
||||
ASSERT_TRUE(pos); \
|
||||
ASSERT_EQ(pos, decoder.GetSize()); \
|
||||
} \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SKIP_TEST(String, std::string, "hello", "world", "nandare", "haihaihai", std::string(500000, 'a'));
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SKIP_TEST(
|
||||
PropertyValue, memgraph::storage::PropertyValue, memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(false), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123L), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue("nandare"), memgraph::storage::PropertyValue(123L)}),
|
||||
memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"nandare", memgraph::storage::PropertyValue(123)}}),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23)));
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define GENERATE_PARTIAL_READ_TEST(name, value) \
|
||||
TEST_F(DecoderEncoderTest, PartialRead##name) { \
|
||||
{ \
|
||||
memgraph::storage::durability::Encoder encoder; \
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion); \
|
||||
encoder.Write##name(value); \
|
||||
encoder.Finalize(); \
|
||||
} \
|
||||
{ \
|
||||
memgraph::utils::InputFile ifile; \
|
||||
memgraph::utils::OutputFile ofile; \
|
||||
ASSERT_TRUE(ifile.Open(storage_file)); \
|
||||
ofile.Open(alternate_file, memgraph::utils::OutputFile::Mode::OVERWRITE_EXISTING); \
|
||||
auto size = ifile.GetSize(); \
|
||||
for (size_t i = 0; i <= size; ++i) { \
|
||||
if (i != 0) { \
|
||||
uint8_t byte; \
|
||||
ASSERT_TRUE(ifile.Read(&byte, sizeof(byte))); \
|
||||
ofile.Write(&byte, sizeof(byte)); \
|
||||
ofile.Sync(); \
|
||||
} \
|
||||
memgraph::storage::durability::Decoder decoder; \
|
||||
auto version = decoder.Initialize(alternate_file, kTestMagic); \
|
||||
if (i < kTestMagic.size() + sizeof(kTestVersion)) { \
|
||||
ASSERT_FALSE(version); \
|
||||
} else { \
|
||||
ASSERT_TRUE(version); \
|
||||
ASSERT_EQ(*version, kTestVersion); \
|
||||
} \
|
||||
if (i != size) { \
|
||||
ASSERT_FALSE(decoder.Read##name()); \
|
||||
} else { \
|
||||
auto decoded = decoder.Read##name(); \
|
||||
ASSERT_TRUE(decoded); \
|
||||
ASSERT_EQ(*decoded, value); \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(Marker, memgraph::storage::durability::Marker::SECTION_VERTEX);
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(Bool, false);
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(Uint, 123123123);
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(Double, 3.1415926535);
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(String, "nandare");
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_READ_TEST(
|
||||
PropertyValue,
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123L), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue{
|
||||
std::map<std::string, memgraph::storage::PropertyValue>{{"haihai", memgraph::storage::PropertyValue()}}},
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))}));
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define GENERATE_PARTIAL_SKIP_TEST(name, value) \
|
||||
TEST_F(DecoderEncoderTest, PartialSkip##name) { \
|
||||
{ \
|
||||
memgraph::storage::durability::Encoder encoder; \
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion); \
|
||||
encoder.Write##name(value); \
|
||||
encoder.Finalize(); \
|
||||
} \
|
||||
{ \
|
||||
memgraph::utils::InputFile ifile; \
|
||||
memgraph::utils::OutputFile ofile; \
|
||||
ASSERT_TRUE(ifile.Open(storage_file)); \
|
||||
ofile.Open(alternate_file, memgraph::utils::OutputFile::Mode::OVERWRITE_EXISTING); \
|
||||
auto size = ifile.GetSize(); \
|
||||
for (size_t i = 0; i <= size; ++i) { \
|
||||
if (i != 0) { \
|
||||
uint8_t byte; \
|
||||
ASSERT_TRUE(ifile.Read(&byte, sizeof(byte))); \
|
||||
ofile.Write(&byte, sizeof(byte)); \
|
||||
ofile.Sync(); \
|
||||
} \
|
||||
memgraph::storage::durability::Decoder decoder; \
|
||||
auto version = decoder.Initialize(alternate_file, kTestMagic); \
|
||||
if (i < kTestMagic.size() + sizeof(kTestVersion)) { \
|
||||
ASSERT_FALSE(version); \
|
||||
} else { \
|
||||
ASSERT_TRUE(version); \
|
||||
ASSERT_EQ(*version, kTestVersion); \
|
||||
} \
|
||||
if (i != size) { \
|
||||
ASSERT_FALSE(decoder.Skip##name()); \
|
||||
} else { \
|
||||
ASSERT_TRUE(decoder.Skip##name()); \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_SKIP_TEST(String, "nandare");
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_PARTIAL_SKIP_TEST(
|
||||
PropertyValue,
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(), memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123L), memgraph::storage::PropertyValue(123.5),
|
||||
memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue{
|
||||
std::map<std::string, memgraph::storage::PropertyValue>{{"haihai", memgraph::storage::PropertyValue()}}},
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23))}));
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(DecoderEncoderTest, PropertyValueInvalidMarker) {
|
||||
{
|
||||
memgraph::storage::durability::Encoder encoder;
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion);
|
||||
encoder.WritePropertyValue(memgraph::storage::PropertyValue(123L));
|
||||
encoder.Finalize();
|
||||
}
|
||||
{
|
||||
memgraph::utils::OutputFile file;
|
||||
file.Open(storage_file, memgraph::utils::OutputFile::Mode::OVERWRITE_EXISTING);
|
||||
for (auto marker : memgraph::storage::durability::kMarkersAll) {
|
||||
bool valid_marker;
|
||||
switch (marker) {
|
||||
case memgraph::storage::durability::Marker::TYPE_NULL:
|
||||
case memgraph::storage::durability::Marker::TYPE_BOOL:
|
||||
case memgraph::storage::durability::Marker::TYPE_INT:
|
||||
case memgraph::storage::durability::Marker::TYPE_DOUBLE:
|
||||
case memgraph::storage::durability::Marker::TYPE_STRING:
|
||||
case memgraph::storage::durability::Marker::TYPE_LIST:
|
||||
case memgraph::storage::durability::Marker::TYPE_MAP:
|
||||
case memgraph::storage::durability::Marker::TYPE_TEMPORAL_DATA:
|
||||
case memgraph::storage::durability::Marker::TYPE_PROPERTY_VALUE:
|
||||
valid_marker = true;
|
||||
break;
|
||||
|
||||
case memgraph::storage::durability::Marker::SECTION_VERTEX:
|
||||
case memgraph::storage::durability::Marker::SECTION_EDGE:
|
||||
case memgraph::storage::durability::Marker::SECTION_MAPPER:
|
||||
case memgraph::storage::durability::Marker::SECTION_METADATA:
|
||||
case memgraph::storage::durability::Marker::SECTION_INDICES:
|
||||
case memgraph::storage::durability::Marker::SECTION_CONSTRAINTS:
|
||||
case memgraph::storage::durability::Marker::SECTION_DELTA:
|
||||
case memgraph::storage::durability::Marker::SECTION_EPOCH_HISTORY:
|
||||
case memgraph::storage::durability::Marker::SECTION_OFFSETS:
|
||||
case memgraph::storage::durability::Marker::DELTA_VERTEX_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_VERTEX_DELETE:
|
||||
case memgraph::storage::durability::Marker::DELTA_VERTEX_ADD_LABEL:
|
||||
case memgraph::storage::durability::Marker::DELTA_VERTEX_REMOVE_LABEL:
|
||||
case memgraph::storage::durability::Marker::DELTA_VERTEX_SET_PROPERTY:
|
||||
case memgraph::storage::durability::Marker::DELTA_EDGE_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_EDGE_DELETE:
|
||||
case memgraph::storage::durability::Marker::DELTA_EDGE_SET_PROPERTY:
|
||||
case memgraph::storage::durability::Marker::DELTA_TRANSACTION_END:
|
||||
case memgraph::storage::durability::Marker::DELTA_LABEL_INDEX_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_LABEL_INDEX_DROP:
|
||||
case memgraph::storage::durability::Marker::DELTA_LABEL_PROPERTY_INDEX_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_LABEL_PROPERTY_INDEX_DROP:
|
||||
case memgraph::storage::durability::Marker::DELTA_EXISTENCE_CONSTRAINT_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_EXISTENCE_CONSTRAINT_DROP:
|
||||
case memgraph::storage::durability::Marker::DELTA_UNIQUE_CONSTRAINT_CREATE:
|
||||
case memgraph::storage::durability::Marker::DELTA_UNIQUE_CONSTRAINT_DROP:
|
||||
case memgraph::storage::durability::Marker::VALUE_FALSE:
|
||||
case memgraph::storage::durability::Marker::VALUE_TRUE:
|
||||
valid_marker = false;
|
||||
break;
|
||||
}
|
||||
// We only run this test with invalid markers.
|
||||
if (valid_marker) continue;
|
||||
{
|
||||
file.SetPosition(memgraph::utils::OutputFile::Position::RELATIVE_TO_END,
|
||||
-(sizeof(uint64_t) + sizeof(memgraph::storage::durability::Marker)));
|
||||
auto byte = static_cast<uint8_t>(marker);
|
||||
file.Write(&byte, sizeof(byte));
|
||||
file.Sync();
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
ASSERT_FALSE(decoder.SkipPropertyValue());
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
ASSERT_FALSE(decoder.ReadPropertyValue());
|
||||
}
|
||||
}
|
||||
{
|
||||
{
|
||||
file.SetPosition(memgraph::utils::OutputFile::Position::RELATIVE_TO_END,
|
||||
-(sizeof(uint64_t) + sizeof(memgraph::storage::durability::Marker)));
|
||||
uint8_t byte = 1;
|
||||
file.Write(&byte, sizeof(byte));
|
||||
file.Sync();
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
ASSERT_FALSE(decoder.SkipPropertyValue());
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
ASSERT_FALSE(decoder.ReadPropertyValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(DecoderEncoderTest, DecoderPosition) {
|
||||
{
|
||||
memgraph::storage::durability::Encoder encoder;
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion);
|
||||
encoder.WriteBool(true);
|
||||
encoder.Finalize();
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
ASSERT_TRUE(decoder.SetPosition(kTestMagic.size() + sizeof(kTestVersion)));
|
||||
auto decoded = decoder.ReadBool();
|
||||
ASSERT_TRUE(decoded);
|
||||
ASSERT_TRUE(*decoded);
|
||||
auto pos = decoder.GetPosition();
|
||||
ASSERT_TRUE(pos);
|
||||
ASSERT_EQ(pos, decoder.GetSize());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(DecoderEncoderTest, EncoderPosition) {
|
||||
{
|
||||
memgraph::storage::durability::Encoder encoder;
|
||||
encoder.Initialize(storage_file, kTestMagic, kTestVersion);
|
||||
encoder.WriteBool(false);
|
||||
encoder.SetPosition(kTestMagic.size() + sizeof(kTestVersion));
|
||||
ASSERT_EQ(encoder.GetPosition(), kTestMagic.size() + sizeof(kTestVersion));
|
||||
encoder.WriteBool(true);
|
||||
encoder.Finalize();
|
||||
}
|
||||
{
|
||||
memgraph::storage::durability::Decoder decoder;
|
||||
auto version = decoder.Initialize(storage_file, kTestMagic);
|
||||
ASSERT_TRUE(version);
|
||||
ASSERT_EQ(*version, kTestVersion);
|
||||
auto decoded = decoder.ReadBool();
|
||||
ASSERT_TRUE(decoded);
|
||||
ASSERT_TRUE(*decoded);
|
||||
auto pos = decoder.GetPosition();
|
||||
ASSERT_TRUE(pos);
|
||||
ASSERT_EQ(pos, decoder.GetSize());
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -1,200 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "storage/v2/storage.hpp"
|
||||
|
||||
using testing::UnorderedElementsAre;
|
||||
|
||||
// TODO: The point of these is not to test GC fully, these are just simple
|
||||
// sanity checks. These will be superseded by a more sophisticated stress test
|
||||
// which will verify that GC is working properly in a multithreaded environment.
|
||||
|
||||
// A simple test trying to get GC to run while a transaction is still alive and
|
||||
// then verify that GC didn't delete anything it shouldn't have.
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(StorageV2Gc, Sanity) {
|
||||
memgraph::storage::Storage storage(memgraph::storage::Config{
|
||||
.gc = {.type = memgraph::storage::Config::Gc::Type::PERIODIC, .interval = std::chrono::milliseconds(100)}});
|
||||
|
||||
std::vector<memgraph::storage::Gid> vertices;
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
// Create some vertices, but delete some of them immediately.
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.CreateVertex();
|
||||
vertices.push_back(vertex.Gid());
|
||||
}
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.FindVertex(vertices[i], memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(vertex.has_value());
|
||||
if (i % 5 == 0) {
|
||||
EXPECT_FALSE(acc.DeleteVertex(&vertex.value()).HasError());
|
||||
}
|
||||
}
|
||||
|
||||
// Wait for GC.
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(300));
|
||||
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex_old = acc.FindVertex(vertices[i], memgraph::storage::View::OLD);
|
||||
auto vertex_new = acc.FindVertex(vertices[i], memgraph::storage::View::NEW);
|
||||
EXPECT_TRUE(vertex_old.has_value());
|
||||
EXPECT_EQ(vertex_new.has_value(), i % 5 != 0);
|
||||
}
|
||||
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// Verify existing vertices and add labels to some of them.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.FindVertex(vertices[i], memgraph::storage::View::OLD);
|
||||
EXPECT_EQ(vertex.has_value(), i % 5 != 0);
|
||||
|
||||
if (vertex.has_value()) {
|
||||
EXPECT_FALSE(vertex->AddLabel(memgraph::storage::LabelId::FromUint(3 * i)).HasError());
|
||||
EXPECT_FALSE(vertex->AddLabel(memgraph::storage::LabelId::FromUint(3 * i + 1)).HasError());
|
||||
EXPECT_FALSE(vertex->AddLabel(memgraph::storage::LabelId::FromUint(3 * i + 2)).HasError());
|
||||
}
|
||||
}
|
||||
|
||||
// Wait for GC.
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(300));
|
||||
|
||||
// Verify labels.
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.FindVertex(vertices[i], memgraph::storage::View::NEW);
|
||||
EXPECT_EQ(vertex.has_value(), i % 5 != 0);
|
||||
|
||||
if (vertex.has_value()) {
|
||||
auto labels_old = vertex->Labels(memgraph::storage::View::OLD);
|
||||
EXPECT_TRUE(labels_old.HasValue());
|
||||
EXPECT_TRUE(labels_old->empty());
|
||||
|
||||
auto labels_new = vertex->Labels(memgraph::storage::View::NEW);
|
||||
EXPECT_TRUE(labels_new.HasValue());
|
||||
EXPECT_THAT(labels_new.GetValue(), UnorderedElementsAre(memgraph::storage::LabelId::FromUint(3 * i),
|
||||
memgraph::storage::LabelId::FromUint(3 * i + 1),
|
||||
memgraph::storage::LabelId::FromUint(3 * i + 2)));
|
||||
}
|
||||
}
|
||||
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// Add and remove some edges.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto from_vertex = acc.FindVertex(vertices[i], memgraph::storage::View::OLD);
|
||||
auto to_vertex = acc.FindVertex(vertices[(i + 1) % 1000], memgraph::storage::View::OLD);
|
||||
EXPECT_EQ(from_vertex.has_value(), i % 5 != 0);
|
||||
EXPECT_EQ(to_vertex.has_value(), (i + 1) % 5 != 0);
|
||||
|
||||
if (from_vertex.has_value() && to_vertex.has_value()) {
|
||||
EXPECT_FALSE(
|
||||
acc.CreateEdge(&from_vertex.value(), &to_vertex.value(), memgraph::storage::EdgeTypeId::FromUint(i))
|
||||
.HasError());
|
||||
}
|
||||
}
|
||||
|
||||
// Detach delete some vertices.
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.FindVertex(vertices[i], memgraph::storage::View::NEW);
|
||||
EXPECT_EQ(vertex.has_value(), i % 5 != 0);
|
||||
if (vertex.has_value()) {
|
||||
if (i % 3 == 0) {
|
||||
EXPECT_FALSE(acc.DetachDeleteVertex(&vertex.value()).HasError());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Wait for GC.
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(300));
|
||||
|
||||
// Vertify edges.
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc.FindVertex(vertices[i], memgraph::storage::View::NEW);
|
||||
EXPECT_EQ(vertex.has_value(), i % 5 != 0 && i % 3 != 0);
|
||||
if (vertex.has_value()) {
|
||||
auto out_edges = vertex->OutEdges(memgraph::storage::View::NEW);
|
||||
if (i % 5 != 4 && i % 3 != 2) {
|
||||
EXPECT_EQ(out_edges.GetValue().size(), 1);
|
||||
EXPECT_EQ(*vertex->OutDegree(memgraph::storage::View::NEW), 1);
|
||||
EXPECT_EQ(out_edges.GetValue().at(0).EdgeType().AsUint(), i);
|
||||
} else {
|
||||
EXPECT_TRUE(out_edges->empty());
|
||||
}
|
||||
|
||||
auto in_edges = vertex->InEdges(memgraph::storage::View::NEW);
|
||||
if (i % 5 != 1 && i % 3 != 1) {
|
||||
EXPECT_EQ(in_edges.GetValue().size(), 1);
|
||||
EXPECT_EQ(*vertex->InDegree(memgraph::storage::View::NEW), 1);
|
||||
EXPECT_EQ(in_edges.GetValue().at(0).EdgeType().AsUint(), (i + 999) % 1000);
|
||||
} else {
|
||||
EXPECT_TRUE(in_edges->empty());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
// A simple sanity check for index GC:
|
||||
// 1. Start transaction 0, create some vertices, add a label to them and
|
||||
// commit.
|
||||
// 2. Start transaction 1.
|
||||
// 3. Start transaction 2, remove the labels and commit;
|
||||
// 4. Wait for GC. GC shouldn't remove the vertices from index because
|
||||
// transaction 1 can still see them with that label.
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(StorageV2Gc, Indices) {
|
||||
memgraph::storage::Storage storage(memgraph::storage::Config{
|
||||
.gc = {.type = memgraph::storage::Config::Gc::Type::PERIODIC, .interval = std::chrono::milliseconds(100)}});
|
||||
|
||||
ASSERT_TRUE(storage.CreateIndex(storage.NameToLabel("label")));
|
||||
|
||||
{
|
||||
auto acc0 = storage.Access();
|
||||
for (uint64_t i = 0; i < 1000; ++i) {
|
||||
auto vertex = acc0.CreateVertex();
|
||||
ASSERT_TRUE(*vertex.AddLabel(acc0.NameToLabel("label")));
|
||||
}
|
||||
ASSERT_FALSE(acc0.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc1 = storage.Access();
|
||||
|
||||
auto acc2 = storage.Access();
|
||||
for (auto vertex : acc2.Vertices(memgraph::storage::View::OLD)) {
|
||||
ASSERT_TRUE(*vertex.RemoveLabel(acc2.NameToLabel("label")));
|
||||
}
|
||||
ASSERT_FALSE(acc2.Commit().HasError());
|
||||
|
||||
// Wait for GC.
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(300));
|
||||
|
||||
std::set<memgraph::storage::Gid> gids;
|
||||
for (auto vertex : acc1.Vertices(acc1.NameToLabel("label"), memgraph::storage::View::OLD)) {
|
||||
gids.insert(vertex.Gid());
|
||||
}
|
||||
EXPECT_EQ(gids.size(), 1000);
|
||||
}
|
||||
}
|
@ -1,832 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "storage/v2/property_value.hpp"
|
||||
#include "storage/v2/storage.hpp"
|
||||
#include "storage/v2/temporal.hpp"
|
||||
|
||||
// NOLINTNEXTLINE(google-build-using-namespace)
|
||||
using namespace memgraph::storage;
|
||||
|
||||
using testing::IsEmpty;
|
||||
using testing::UnorderedElementsAre;
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define ASSERT_NO_ERROR(result) ASSERT_FALSE((result).HasError())
|
||||
|
||||
class IndexTest : public testing::Test {
|
||||
protected:
|
||||
void SetUp() override {
|
||||
auto acc = storage.Access();
|
||||
prop_id = acc.NameToProperty("id");
|
||||
prop_val = acc.NameToProperty("val");
|
||||
label1 = acc.NameToLabel("label1");
|
||||
label2 = acc.NameToLabel("label2");
|
||||
vertex_id = 0;
|
||||
}
|
||||
|
||||
Storage storage;
|
||||
PropertyId prop_id;
|
||||
PropertyId prop_val;
|
||||
LabelId label1;
|
||||
LabelId label2;
|
||||
|
||||
VertexAccessor CreateVertex(Storage::Accessor *accessor) {
|
||||
VertexAccessor vertex = accessor->CreateVertex();
|
||||
MG_ASSERT(!vertex.SetProperty(prop_id, PropertyValue(vertex_id++)).HasError());
|
||||
return vertex;
|
||||
}
|
||||
|
||||
template <class TIterable>
|
||||
std::vector<int64_t> GetIds(TIterable iterable, View view = View::OLD) {
|
||||
std::vector<int64_t> ret;
|
||||
for (auto vertex : iterable) {
|
||||
ret.push_back(vertex.GetProperty(prop_id, view)->ValueInt());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private:
|
||||
int vertex_id;
|
||||
};
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexCreate) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelIndexExists(label1));
|
||||
}
|
||||
EXPECT_EQ(storage.ListAllIndices().label.size(), 0);
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 10; i < 20; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
|
||||
acc.Abort();
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 10; i < 20; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 21, 23, 25, 27, 29));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexDrop) {
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelIndexExists(label1));
|
||||
}
|
||||
EXPECT_EQ(storage.ListAllIndices().label.size(), 0);
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
}
|
||||
|
||||
EXPECT_TRUE(storage.DropIndex(label1));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelIndexExists(label1));
|
||||
}
|
||||
EXPECT_EQ(storage.ListAllIndices().label.size(), 0);
|
||||
|
||||
EXPECT_FALSE(storage.DropIndex(label1));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelIndexExists(label1));
|
||||
}
|
||||
EXPECT_EQ(storage.ListAllIndices().label.size(), 0);
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 10; i < 20; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_TRUE(acc.LabelIndexExists(label1));
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllIndices().label, UnorderedElementsAre(label1));
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(1, 3, 5, 7, 9, 11, 13, 15, 17, 19));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexBasic) {
|
||||
// The following steps are performed and index correctness is validated after
|
||||
// each step:
|
||||
// 1. Create 10 vertices numbered from 0 to 9.
|
||||
// 2. Add Label1 to odd numbered, and Label2 to even numbered vertices.
|
||||
// 3. Remove Label1 from odd numbered vertices, and add it to even numbered
|
||||
// vertices.
|
||||
// 4. Delete even numbered vertices.
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
EXPECT_TRUE(storage.CreateIndex(label2));
|
||||
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(storage.ListAllIndices().label, UnorderedElementsAre(label1, label2));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
int64_t id = vertex.GetProperty(prop_id, View::OLD)->ValueInt();
|
||||
if (id % 2) {
|
||||
ASSERT_NO_ERROR(vertex.RemoveLabel(label1));
|
||||
} else {
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
int64_t id = vertex.GetProperty(prop_id, View::OLD)->ValueInt();
|
||||
if (id % 2 == 0) {
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&vertex));
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, View::NEW), View::NEW), IsEmpty());
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexDuplicateVersions) {
|
||||
// By removing labels and adding them again we create duplicate entries for
|
||||
// the same vertex in the index (they only differ by the timestamp). This test
|
||||
// checks that duplicates are properly filtered out.
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
EXPECT_TRUE(storage.CreateIndex(label2));
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.RemoveLabel(label1));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
}
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexTransactionalIsolation) {
|
||||
// Check that transactions only see entries they are supposed to see.
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
EXPECT_TRUE(storage.CreateIndex(label2));
|
||||
|
||||
auto acc_before = storage.Access();
|
||||
auto acc = storage.Access();
|
||||
auto acc_after = storage.Access();
|
||||
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc_before.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
|
||||
auto acc_after_commit = storage.Access();
|
||||
|
||||
EXPECT_THAT(GetIds(acc_before.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after.Vertices(label1, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after_commit.Vertices(label1, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelIndexCountEstimate) {
|
||||
EXPECT_TRUE(storage.CreateIndex(label1));
|
||||
EXPECT_TRUE(storage.CreateIndex(label2));
|
||||
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 20; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 3 ? label1 : label2));
|
||||
}
|
||||
|
||||
EXPECT_EQ(acc.ApproximateVertexCount(label1), 13);
|
||||
EXPECT_EQ(acc.ApproximateVertexCount(label2), 7);
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexCreateAndDrop) {
|
||||
EXPECT_EQ(storage.ListAllIndices().label_property.size(), 0);
|
||||
EXPECT_TRUE(storage.CreateIndex(label1, prop_id));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_TRUE(acc.LabelPropertyIndexExists(label1, prop_id));
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllIndices().label_property, UnorderedElementsAre(std::make_pair(label1, prop_id)));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelPropertyIndexExists(label2, prop_id));
|
||||
}
|
||||
EXPECT_FALSE(storage.CreateIndex(label1, prop_id));
|
||||
EXPECT_THAT(storage.ListAllIndices().label_property, UnorderedElementsAre(std::make_pair(label1, prop_id)));
|
||||
|
||||
EXPECT_TRUE(storage.CreateIndex(label2, prop_id));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_TRUE(acc.LabelPropertyIndexExists(label2, prop_id));
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllIndices().label_property,
|
||||
UnorderedElementsAre(std::make_pair(label1, prop_id), std::make_pair(label2, prop_id)));
|
||||
|
||||
EXPECT_TRUE(storage.DropIndex(label1, prop_id));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelPropertyIndexExists(label1, prop_id));
|
||||
}
|
||||
EXPECT_THAT(storage.ListAllIndices().label_property, UnorderedElementsAre(std::make_pair(label2, prop_id)));
|
||||
EXPECT_FALSE(storage.DropIndex(label1, prop_id));
|
||||
|
||||
EXPECT_TRUE(storage.DropIndex(label2, prop_id));
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_FALSE(acc.LabelPropertyIndexExists(label2, prop_id));
|
||||
}
|
||||
EXPECT_EQ(storage.ListAllIndices().label_property.size(), 0);
|
||||
}
|
||||
|
||||
// The following three tests are almost an exact copy-paste of the corresponding
|
||||
// label index tests. We request all vertices with given label and property from
|
||||
// the index, without range filtering. Range filtering is tested in a separate
|
||||
// test.
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexBasic) {
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
storage.CreateIndex(label2, prop_val);
|
||||
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), IsEmpty());
|
||||
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(i % 2 ? label1 : label2));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue(i)));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
int64_t id = vertex.GetProperty(prop_id, View::OLD)->ValueInt();
|
||||
if (id % 2) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue()));
|
||||
} else {
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
int64_t id = vertex.GetProperty(prop_id, View::OLD)->ValueInt();
|
||||
if (id % 2 == 0) {
|
||||
ASSERT_NO_ERROR(acc.DeleteVertex(&vertex));
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(1, 3, 5, 7, 9));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 2, 4, 6, 8));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
acc.AdvanceCommand();
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::OLD), View::OLD), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label2, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexDuplicateVersions) {
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue(i)));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue()));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
for (auto vertex : acc.Vertices(View::OLD)) {
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue(42)));
|
||||
}
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::OLD), View::OLD), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexTransactionalIsolation) {
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
|
||||
auto acc_before = storage.Access();
|
||||
auto acc = storage.Access();
|
||||
auto acc_after = storage.Access();
|
||||
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue(i)));
|
||||
}
|
||||
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, View::NEW), View::NEW), UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
EXPECT_THAT(GetIds(acc_before.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
|
||||
auto acc_after_commit = storage.Access();
|
||||
|
||||
EXPECT_THAT(GetIds(acc_before.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after.Vertices(label1, prop_val, View::NEW), View::NEW), IsEmpty());
|
||||
EXPECT_THAT(GetIds(acc_after_commit.Vertices(label1, prop_val, View::NEW), View::NEW),
|
||||
UnorderedElementsAre(0, 1, 2, 3, 4));
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexFiltering) {
|
||||
// We insert vertices with values:
|
||||
// 0 0.0 1 1.0 2 2.0 3 3.0 4 4.0
|
||||
// Then we check all combinations of inclusive and exclusive bounds.
|
||||
// We also have a mix of doubles and integers to verify that they are sorted
|
||||
// properly.
|
||||
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, i % 2 ? PropertyValue(i / 2) : PropertyValue(i / 2.0)));
|
||||
}
|
||||
ASSERT_NO_ERROR(acc.Commit());
|
||||
}
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (int i = 0; i < 5; ++i) {
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, PropertyValue(i), View::OLD)),
|
||||
UnorderedElementsAre(2 * i, 2 * i + 1));
|
||||
}
|
||||
|
||||
// [1, +inf>
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundInclusive(PropertyValue(1)),
|
||||
std::nullopt, View::OLD)),
|
||||
UnorderedElementsAre(2, 3, 4, 5, 6, 7, 8, 9));
|
||||
// <1, +inf>
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundExclusive(PropertyValue(1)),
|
||||
std::nullopt, View::OLD)),
|
||||
UnorderedElementsAre(4, 5, 6, 7, 8, 9));
|
||||
|
||||
// <-inf, 3]
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, std::nullopt,
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(0, 1, 2, 3, 4, 5, 6, 7));
|
||||
// <-inf, 3>
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, std::nullopt,
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(0, 1, 2, 3, 4, 5));
|
||||
|
||||
// [1, 3]
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundInclusive(PropertyValue(1)),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(2, 3, 4, 5, 6, 7));
|
||||
// <1, 3]
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundExclusive(PropertyValue(1)),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(4, 5, 6, 7));
|
||||
// [1, 3>
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundInclusive(PropertyValue(1)),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(2, 3, 4, 5));
|
||||
// <1, 3>
|
||||
EXPECT_THAT(GetIds(acc.Vertices(label1, prop_val, memgraph::utils::MakeBoundExclusive(PropertyValue(1)),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(3)), View::OLD)),
|
||||
UnorderedElementsAre(4, 5));
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_F(IndexTest, LabelPropertyIndexCountEstimate) {
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
|
||||
auto acc = storage.Access();
|
||||
for (int i = 1; i <= 10; ++i) {
|
||||
for (int j = 0; j < i; ++j) {
|
||||
auto vertex = CreateVertex(&acc);
|
||||
ASSERT_NO_ERROR(vertex.AddLabel(label1));
|
||||
ASSERT_NO_ERROR(vertex.SetProperty(prop_val, PropertyValue(i)));
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_EQ(acc.ApproximateVertexCount(label1, prop_val), 55);
|
||||
for (int i = 1; i <= 10; ++i) {
|
||||
EXPECT_EQ(acc.ApproximateVertexCount(label1, prop_val, PropertyValue(i)), i);
|
||||
}
|
||||
|
||||
EXPECT_EQ(acc.ApproximateVertexCount(label1, prop_val, memgraph::utils::MakeBoundInclusive(PropertyValue(2)),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(6))),
|
||||
2 + 3 + 4 + 5 + 6);
|
||||
}
|
||||
|
||||
TEST_F(IndexTest, LabelPropertyIndexMixedIteration) {
|
||||
storage.CreateIndex(label1, prop_val);
|
||||
|
||||
const std::array temporals{TemporalData{TemporalType::Date, 23}, TemporalData{TemporalType::Date, 28},
|
||||
TemporalData{TemporalType::LocalDateTime, 20}};
|
||||
|
||||
std::vector<PropertyValue> values = {
|
||||
PropertyValue(false),
|
||||
PropertyValue(true),
|
||||
PropertyValue(-std::numeric_limits<double>::infinity()),
|
||||
PropertyValue(std::numeric_limits<int64_t>::min()),
|
||||
PropertyValue(-1),
|
||||
PropertyValue(-0.5),
|
||||
PropertyValue(0),
|
||||
PropertyValue(0.5),
|
||||
PropertyValue(1),
|
||||
PropertyValue(1.5),
|
||||
PropertyValue(2),
|
||||
PropertyValue(std::numeric_limits<int64_t>::max()),
|
||||
PropertyValue(std::numeric_limits<double>::infinity()),
|
||||
PropertyValue(""),
|
||||
PropertyValue("a"),
|
||||
PropertyValue("b"),
|
||||
PropertyValue("c"),
|
||||
PropertyValue(std::vector<PropertyValue>()),
|
||||
PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)}),
|
||||
PropertyValue(std::vector<PropertyValue>{PropertyValue(2)}),
|
||||
PropertyValue(std::map<std::string, PropertyValue>()),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5)}}),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}}),
|
||||
PropertyValue(temporals[0]),
|
||||
PropertyValue(temporals[1]),
|
||||
PropertyValue(temporals[2]),
|
||||
};
|
||||
|
||||
// Create vertices, each with one of the values above.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
for (const auto &value : values) {
|
||||
auto v = acc.CreateVertex();
|
||||
ASSERT_TRUE(v.AddLabel(label1).HasValue());
|
||||
ASSERT_TRUE(v.SetProperty(prop_val, value).HasValue());
|
||||
}
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// Verify that all nodes are in the index.
|
||||
{
|
||||
auto acc = storage.Access();
|
||||
auto iterable = acc.Vertices(label1, prop_val, View::OLD);
|
||||
auto it = iterable.begin();
|
||||
for (const auto &value : values) {
|
||||
ASSERT_NE(it, iterable.end());
|
||||
auto vertex = *it;
|
||||
auto maybe_value = vertex.GetProperty(prop_val, View::OLD);
|
||||
ASSERT_TRUE(maybe_value.HasValue());
|
||||
ASSERT_EQ(value, *maybe_value);
|
||||
++it;
|
||||
}
|
||||
ASSERT_EQ(it, iterable.end());
|
||||
}
|
||||
|
||||
auto verify = [&](const std::optional<memgraph::utils::Bound<PropertyValue>> &from,
|
||||
const std::optional<memgraph::utils::Bound<PropertyValue>> &to,
|
||||
const std::vector<PropertyValue> &expected) {
|
||||
auto acc = storage.Access();
|
||||
auto iterable = acc.Vertices(label1, prop_val, from, to, View::OLD);
|
||||
size_t i = 0;
|
||||
for (auto it = iterable.begin(); it != iterable.end(); ++it, ++i) {
|
||||
auto vertex = *it;
|
||||
auto maybe_value = vertex.GetProperty(prop_val, View::OLD);
|
||||
ASSERT_TRUE(maybe_value.HasValue());
|
||||
ASSERT_EQ(*maybe_value, expected[i]);
|
||||
}
|
||||
ASSERT_EQ(i, expected.size());
|
||||
};
|
||||
|
||||
// Range iteration with two specified bounds that have the same type should
|
||||
// yield the naturally expected items.
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(false)),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(true)), {});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(false)),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(true)), {PropertyValue(true)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(false)),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(true)), {PropertyValue(false)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(false)),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(true)), {PropertyValue(false), PropertyValue(true)});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(0)), memgraph::utils::MakeBoundExclusive(PropertyValue(1.8)),
|
||||
{PropertyValue(0.5), PropertyValue(1), PropertyValue(1.5)});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(0)), memgraph::utils::MakeBoundInclusive(PropertyValue(1.8)),
|
||||
{PropertyValue(0.5), PropertyValue(1), PropertyValue(1.5)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(0)), memgraph::utils::MakeBoundExclusive(PropertyValue(1.8)),
|
||||
{PropertyValue(0), PropertyValue(0.5), PropertyValue(1), PropertyValue(1.5)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(0)), memgraph::utils::MakeBoundInclusive(PropertyValue(1.8)),
|
||||
{PropertyValue(0), PropertyValue(0.5), PropertyValue(1), PropertyValue(1.5)});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue("b")),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue("memgraph")), {PropertyValue("c")});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue("b")),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue("memgraph")), {PropertyValue("c")});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue("b")),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue("memgraph")), {PropertyValue("b"), PropertyValue("c")});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue("b")),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue("memgraph")), {PropertyValue("b"), PropertyValue("c")});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)})),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue("b")})),
|
||||
{PropertyValue(std::vector<PropertyValue>{PropertyValue(2)})});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)})),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue("b")})),
|
||||
{PropertyValue(std::vector<PropertyValue>{PropertyValue(2)})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)})),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue("b")})),
|
||||
{PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)}),
|
||||
PropertyValue(std::vector<PropertyValue>{PropertyValue(2)})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)})),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue("b")})),
|
||||
{PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)}),
|
||||
PropertyValue(std::vector<PropertyValue>{PropertyValue(2)})});
|
||||
verify(memgraph::utils::MakeBoundExclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5.0)}})),
|
||||
memgraph::utils::MakeBoundExclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue("b")}})),
|
||||
{PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}})});
|
||||
verify(memgraph::utils::MakeBoundExclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5.0)}})),
|
||||
memgraph::utils::MakeBoundInclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue("b")}})),
|
||||
{PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5.0)}})),
|
||||
memgraph::utils::MakeBoundExclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue("b")}})),
|
||||
{PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5)}}),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5.0)}})),
|
||||
memgraph::utils::MakeBoundInclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue("b")}})),
|
||||
{PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5)}}),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}})});
|
||||
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(temporals[0])),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(TemporalData{TemporalType::Date, 200})),
|
||||
// LocalDateTime has a "higher" type number so it is not part of the range
|
||||
{PropertyValue(temporals[1])});
|
||||
verify(memgraph::utils::MakeBoundExclusive(PropertyValue(temporals[0])),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(temporals[2])),
|
||||
{PropertyValue(temporals[1]), PropertyValue(temporals[2])});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(temporals[0])),
|
||||
memgraph::utils::MakeBoundExclusive(PropertyValue(temporals[2])),
|
||||
{PropertyValue(temporals[0]), PropertyValue(temporals[1])});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(temporals[0])),
|
||||
memgraph::utils::MakeBoundInclusive(PropertyValue(temporals[2])),
|
||||
{PropertyValue(temporals[0]), PropertyValue(temporals[1]), PropertyValue(temporals[2])});
|
||||
|
||||
// Range iteration with one unspecified bound should only yield items that
|
||||
// have the same type as the specified bound.
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(false)), std::nullopt,
|
||||
{PropertyValue(false), PropertyValue(true)});
|
||||
verify(std::nullopt, memgraph::utils::MakeBoundExclusive(PropertyValue(true)), {PropertyValue(false)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(1)), std::nullopt,
|
||||
{PropertyValue(1), PropertyValue(1.5), PropertyValue(2), PropertyValue(std::numeric_limits<int64_t>::max()),
|
||||
PropertyValue(std::numeric_limits<double>::infinity())});
|
||||
verify(std::nullopt, memgraph::utils::MakeBoundExclusive(PropertyValue(0)),
|
||||
{PropertyValue(-std::numeric_limits<double>::infinity()), PropertyValue(std::numeric_limits<int64_t>::min()),
|
||||
PropertyValue(-1), PropertyValue(-0.5)});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue("b")), std::nullopt,
|
||||
{PropertyValue("b"), PropertyValue("c")});
|
||||
verify(std::nullopt, memgraph::utils::MakeBoundExclusive(PropertyValue("b")),
|
||||
{PropertyValue(""), PropertyValue("a")});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(false)})),
|
||||
std::nullopt,
|
||||
{PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)}),
|
||||
PropertyValue(std::vector<PropertyValue>{PropertyValue(2)})});
|
||||
verify(std::nullopt, memgraph::utils::MakeBoundExclusive(PropertyValue(std::vector<PropertyValue>{PropertyValue(1)})),
|
||||
{PropertyValue(std::vector<PropertyValue>()), PropertyValue(std::vector<PropertyValue>{PropertyValue(0.8)})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(false)}})),
|
||||
std::nullopt,
|
||||
{PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5)}}),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(10)}})});
|
||||
verify(std::nullopt,
|
||||
memgraph::utils::MakeBoundExclusive(
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(7.5)}})),
|
||||
{PropertyValue(std::map<std::string, PropertyValue>()),
|
||||
PropertyValue(std::map<std::string, PropertyValue>{{"id", PropertyValue(5)}})});
|
||||
verify(memgraph::utils::MakeBoundInclusive(PropertyValue(TemporalData(TemporalType::Date, 10))), std::nullopt,
|
||||
{PropertyValue(temporals[0]), PropertyValue(temporals[1]), PropertyValue(temporals[2])});
|
||||
verify(std::nullopt, memgraph::utils::MakeBoundExclusive(PropertyValue(TemporalData(TemporalType::Duration, 0))),
|
||||
{PropertyValue(temporals[0]), PropertyValue(temporals[1]), PropertyValue(temporals[2])});
|
||||
|
||||
// Range iteration with two specified bounds that don't have the same type
|
||||
// should yield no items.
|
||||
for (size_t i = 0; i < values.size(); ++i) {
|
||||
for (size_t j = i; j < values.size(); ++j) {
|
||||
if (PropertyValue::AreComparableTypes(values[i].type(), values[j].type())) {
|
||||
verify(memgraph::utils::MakeBoundInclusive(values[i]), memgraph::utils::MakeBoundInclusive(values[j]),
|
||||
{values.begin() + i, values.begin() + j + 1});
|
||||
} else {
|
||||
verify(memgraph::utils::MakeBoundInclusive(values[i]), memgraph::utils::MakeBoundInclusive(values[j]), {});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Iteration without any bounds should return all items of the index.
|
||||
verify(std::nullopt, std::nullopt, values);
|
||||
}
|
@ -1,110 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "storage/v2/isolation_level.hpp"
|
||||
#include "storage/v2/storage.hpp"
|
||||
|
||||
namespace {
|
||||
int64_t VerticesCount(memgraph::storage::Storage::Accessor &accessor) {
|
||||
int64_t count{0};
|
||||
for ([[maybe_unused]] const auto &vertex : accessor.Vertices(memgraph::storage::View::NEW)) {
|
||||
++count;
|
||||
}
|
||||
|
||||
return count;
|
||||
}
|
||||
|
||||
inline constexpr std::array isolation_levels{memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION,
|
||||
memgraph::storage::IsolationLevel::READ_COMMITTED,
|
||||
memgraph::storage::IsolationLevel::READ_UNCOMMITTED};
|
||||
|
||||
std::string_view IsolationLevelToString(const memgraph::storage::IsolationLevel isolation_level) {
|
||||
switch (isolation_level) {
|
||||
case memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION:
|
||||
return "SNAPSHOT_ISOLATION";
|
||||
case memgraph::storage::IsolationLevel::READ_COMMITTED:
|
||||
return "READ_COMMITTED";
|
||||
case memgraph::storage::IsolationLevel::READ_UNCOMMITTED:
|
||||
return "READ_UNCOMMITTED";
|
||||
}
|
||||
}
|
||||
} // namespace
|
||||
|
||||
class StorageIsolationLevelTest : public ::testing::TestWithParam<memgraph::storage::IsolationLevel> {
|
||||
public:
|
||||
struct PrintToStringParamName {
|
||||
std::string operator()(const testing::TestParamInfo<memgraph::storage::IsolationLevel> &info) {
|
||||
return std::string(IsolationLevelToString(static_cast<memgraph::storage::IsolationLevel>(info.param)));
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
TEST_P(StorageIsolationLevelTest, Visibility) {
|
||||
const auto default_isolation_level = GetParam();
|
||||
|
||||
for (const auto override_isolation_level : isolation_levels) {
|
||||
memgraph::storage::Storage storage{
|
||||
memgraph::storage::Config{.transaction = {.isolation_level = default_isolation_level}}};
|
||||
auto creator = storage.Access();
|
||||
auto default_isolation_level_reader = storage.Access();
|
||||
auto override_isolation_level_reader = storage.Access(override_isolation_level);
|
||||
|
||||
ASSERT_EQ(VerticesCount(default_isolation_level_reader), 0);
|
||||
ASSERT_EQ(VerticesCount(override_isolation_level_reader), 0);
|
||||
|
||||
static constexpr auto iteration_count = 10;
|
||||
{
|
||||
SCOPED_TRACE(fmt::format(
|
||||
"Visibility while the creator transaction is active "
|
||||
"(default isolation level = {}, override isolation level = {})",
|
||||
IsolationLevelToString(default_isolation_level), IsolationLevelToString(override_isolation_level)));
|
||||
for (size_t i = 1; i <= iteration_count; ++i) {
|
||||
creator.CreateVertex();
|
||||
|
||||
const auto check_vertices_count = [i](auto &accessor, const auto isolation_level) {
|
||||
const auto expected_count = isolation_level == memgraph::storage::IsolationLevel::READ_UNCOMMITTED ? i : 0;
|
||||
EXPECT_EQ(VerticesCount(accessor), expected_count);
|
||||
};
|
||||
check_vertices_count(default_isolation_level_reader, default_isolation_level);
|
||||
check_vertices_count(override_isolation_level_reader, override_isolation_level);
|
||||
}
|
||||
}
|
||||
|
||||
ASSERT_FALSE(creator.Commit().HasError());
|
||||
{
|
||||
SCOPED_TRACE(fmt::format(
|
||||
"Visibility after the creator transaction is committed "
|
||||
"(default isolation level = {}, override isolation level = {})",
|
||||
IsolationLevelToString(default_isolation_level), IsolationLevelToString(override_isolation_level)));
|
||||
const auto check_vertices_count = [](auto &accessor, const auto isolation_level) {
|
||||
const auto expected_count =
|
||||
isolation_level == memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION ? 0 : iteration_count;
|
||||
ASSERT_EQ(VerticesCount(accessor), expected_count);
|
||||
};
|
||||
|
||||
check_vertices_count(default_isolation_level_reader, default_isolation_level);
|
||||
check_vertices_count(override_isolation_level_reader, override_isolation_level);
|
||||
}
|
||||
|
||||
ASSERT_FALSE(default_isolation_level_reader.Commit().HasError());
|
||||
ASSERT_FALSE(override_isolation_level_reader.Commit().HasError());
|
||||
|
||||
SCOPED_TRACE("Visibility after a new transaction is started");
|
||||
auto verifier = storage.Access();
|
||||
ASSERT_EQ(VerticesCount(verifier), iteration_count);
|
||||
ASSERT_FALSE(verifier.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(ParameterizedStorageIsolationLevelTests, StorageIsolationLevelTest,
|
||||
::testing::ValuesIn(isolation_levels), StorageIsolationLevelTest::PrintToStringParamName());
|
@ -1,47 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include "storage/v2/name_id_mapper.hpp"
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(NameIdMapper, Basic) {
|
||||
memgraph::storage::NameIdMapper mapper;
|
||||
|
||||
ASSERT_EQ(mapper.NameToId("n1"), 0);
|
||||
ASSERT_EQ(mapper.NameToId("n2"), 1);
|
||||
ASSERT_EQ(mapper.NameToId("n1"), 0);
|
||||
ASSERT_EQ(mapper.NameToId("n2"), 1);
|
||||
ASSERT_EQ(mapper.NameToId("n3"), 2);
|
||||
|
||||
ASSERT_EQ(mapper.IdToName(0), "n1");
|
||||
ASSERT_EQ(mapper.IdToName(1), "n2");
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST(NameIdMapper, Correctness) {
|
||||
memgraph::storage::NameIdMapper mapper;
|
||||
|
||||
ASSERT_DEATH(mapper.IdToName(0), "");
|
||||
ASSERT_EQ(mapper.NameToId("n1"), 0);
|
||||
ASSERT_EQ(mapper.IdToName(0), "n1");
|
||||
|
||||
ASSERT_DEATH(mapper.IdToName(1), "");
|
||||
ASSERT_EQ(mapper.NameToId("n2"), 1);
|
||||
ASSERT_EQ(mapper.IdToName(1), "n2");
|
||||
|
||||
ASSERT_EQ(mapper.NameToId("n1"), 0);
|
||||
ASSERT_EQ(mapper.NameToId("n2"), 1);
|
||||
|
||||
ASSERT_EQ(mapper.IdToName(1), "n2");
|
||||
ASSERT_EQ(mapper.IdToName(0), "n1");
|
||||
}
|
@ -1,651 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
#include "storage/v2/property_store.hpp"
|
||||
#include "storage/v2/property_value.hpp"
|
||||
#include "storage/v2/temporal.hpp"
|
||||
|
||||
using testing::UnorderedElementsAre;
|
||||
|
||||
const memgraph::storage::PropertyValue kSampleValues[] = {
|
||||
memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(false),
|
||||
memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(0),
|
||||
memgraph::storage::PropertyValue(33),
|
||||
memgraph::storage::PropertyValue(-33),
|
||||
memgraph::storage::PropertyValue(-3137),
|
||||
memgraph::storage::PropertyValue(3137),
|
||||
memgraph::storage::PropertyValue(310000007),
|
||||
memgraph::storage::PropertyValue(-310000007),
|
||||
memgraph::storage::PropertyValue(3100000000007L),
|
||||
memgraph::storage::PropertyValue(-3100000000007L),
|
||||
memgraph::storage::PropertyValue(0.0),
|
||||
memgraph::storage::PropertyValue(33.33),
|
||||
memgraph::storage::PropertyValue(-33.33),
|
||||
memgraph::storage::PropertyValue(3137.3137),
|
||||
memgraph::storage::PropertyValue(-3137.3137),
|
||||
memgraph::storage::PropertyValue("sample"),
|
||||
memgraph::storage::PropertyValue(std::string(404, 'n')),
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(33), memgraph::storage::PropertyValue(std::string("sample")),
|
||||
memgraph::storage::PropertyValue(-33.33)}),
|
||||
memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(), memgraph::storage::PropertyValue(false)}),
|
||||
memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"sample", memgraph::storage::PropertyValue()}, {"key", memgraph::storage::PropertyValue(false)}}),
|
||||
memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"test", memgraph::storage::PropertyValue(33)},
|
||||
{"map", memgraph::storage::PropertyValue(std::string("sample"))},
|
||||
{"item", memgraph::storage::PropertyValue(-33.33)}}),
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23)),
|
||||
};
|
||||
|
||||
void TestIsPropertyEqual(const memgraph::storage::PropertyStore &store, memgraph::storage::PropertyId property,
|
||||
const memgraph::storage::PropertyValue &value) {
|
||||
ASSERT_TRUE(store.IsPropertyEqual(property, value));
|
||||
for (const auto &sample : kSampleValues) {
|
||||
if (sample == value) {
|
||||
ASSERT_TRUE(store.IsPropertyEqual(property, sample));
|
||||
} else {
|
||||
ASSERT_FALSE(store.IsPropertyEqual(property, sample));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyStore, Simple) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(42);
|
||||
ASSERT_TRUE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
|
||||
ASSERT_FALSE(props.SetProperty(prop, memgraph::storage::PropertyValue()));
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, SimpleLarge) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
{
|
||||
auto value = memgraph::storage::PropertyValue(std::string(10000, 'a'));
|
||||
ASSERT_TRUE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
{
|
||||
auto value =
|
||||
memgraph::storage::PropertyValue(memgraph::storage::TemporalData(memgraph::storage::TemporalType::Date, 23));
|
||||
ASSERT_FALSE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
|
||||
ASSERT_FALSE(props.SetProperty(prop, memgraph::storage::PropertyValue()));
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, EmptySetToNull) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
ASSERT_TRUE(props.SetProperty(prop, memgraph::storage::PropertyValue()));
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, Clear) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(42);
|
||||
ASSERT_TRUE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
ASSERT_TRUE(props.ClearProperties());
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, EmptyClear) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
ASSERT_FALSE(props.ClearProperties());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, MoveConstruct) {
|
||||
memgraph::storage::PropertyStore props1;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(42);
|
||||
ASSERT_TRUE(props1.SetProperty(prop, value));
|
||||
ASSERT_EQ(props1.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, value);
|
||||
ASSERT_THAT(props1.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
{
|
||||
memgraph::storage::PropertyStore props2(std::move(props1));
|
||||
ASSERT_EQ(props2.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
// NOLINTNEXTLINE(bugprone-use-after-move,clang-analyzer-cplusplus.Move,hicpp-invalid-access-moved)
|
||||
ASSERT_TRUE(props1.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props1.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, MoveConstructLarge) {
|
||||
memgraph::storage::PropertyStore props1;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(std::string(10000, 'a'));
|
||||
ASSERT_TRUE(props1.SetProperty(prop, value));
|
||||
ASSERT_EQ(props1.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, value);
|
||||
ASSERT_THAT(props1.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
{
|
||||
memgraph::storage::PropertyStore props2(std::move(props1));
|
||||
ASSERT_EQ(props2.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
// NOLINTNEXTLINE(bugprone-use-after-move,clang-analyzer-cplusplus.Move,hicpp-invalid-access-moved)
|
||||
ASSERT_TRUE(props1.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props1.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, MoveAssign) {
|
||||
memgraph::storage::PropertyStore props1;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(42);
|
||||
ASSERT_TRUE(props1.SetProperty(prop, value));
|
||||
ASSERT_EQ(props1.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, value);
|
||||
ASSERT_THAT(props1.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
{
|
||||
auto value2 = memgraph::storage::PropertyValue(68);
|
||||
memgraph::storage::PropertyStore props2;
|
||||
ASSERT_TRUE(props2.SetProperty(prop, value2));
|
||||
ASSERT_EQ(props2.GetProperty(prop), value2);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value2);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value2)));
|
||||
props2 = std::move(props1);
|
||||
ASSERT_EQ(props2.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
// NOLINTNEXTLINE(bugprone-use-after-move,clang-analyzer-cplusplus.Move,hicpp-invalid-access-moved)
|
||||
ASSERT_TRUE(props1.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props1.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, MoveAssignLarge) {
|
||||
memgraph::storage::PropertyStore props1;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
auto value = memgraph::storage::PropertyValue(std::string(10000, 'a'));
|
||||
ASSERT_TRUE(props1.SetProperty(prop, value));
|
||||
ASSERT_EQ(props1.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, value);
|
||||
ASSERT_THAT(props1.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
{
|
||||
auto value2 = memgraph::storage::PropertyValue(std::string(10000, 'b'));
|
||||
memgraph::storage::PropertyStore props2;
|
||||
ASSERT_TRUE(props2.SetProperty(prop, value2));
|
||||
ASSERT_EQ(props2.GetProperty(prop), value2);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value2);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value2)));
|
||||
props2 = std::move(props1);
|
||||
ASSERT_EQ(props2.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props2.HasProperty(prop));
|
||||
TestIsPropertyEqual(props2, prop, value);
|
||||
ASSERT_THAT(props2.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
}
|
||||
// NOLINTNEXTLINE(bugprone-use-after-move,clang-analyzer-cplusplus.Move,hicpp-invalid-access-moved)
|
||||
ASSERT_TRUE(props1.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props1.HasProperty(prop));
|
||||
TestIsPropertyEqual(props1, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props1.Properties().size(), 0);
|
||||
}
|
||||
|
||||
TEST(PropertyStore, EmptySet) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue()};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
const memgraph::storage::TemporalData temporal{memgraph::storage::TemporalType::LocalDateTime, 23};
|
||||
std::vector<memgraph::storage::PropertyValue> data{
|
||||
memgraph::storage::PropertyValue(true), memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue(123.5), memgraph::storage::PropertyValue("nandare"),
|
||||
memgraph::storage::PropertyValue(vec), memgraph::storage::PropertyValue(map),
|
||||
memgraph::storage::PropertyValue(temporal)};
|
||||
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
for (const auto &value : data) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
|
||||
ASSERT_TRUE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
ASSERT_FALSE(props.SetProperty(prop, value));
|
||||
ASSERT_EQ(props.GetProperty(prop), value);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, value);
|
||||
ASSERT_THAT(props.Properties(), UnorderedElementsAre(std::pair(prop, value)));
|
||||
ASSERT_FALSE(props.SetProperty(prop, memgraph::storage::PropertyValue()));
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
ASSERT_TRUE(props.SetProperty(prop, memgraph::storage::PropertyValue()));
|
||||
ASSERT_TRUE(props.GetProperty(prop).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(prop));
|
||||
TestIsPropertyEqual(props, prop, memgraph::storage::PropertyValue());
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyStore, FullSet) {
|
||||
std::vector<memgraph::storage::PropertyValue> vec{memgraph::storage::PropertyValue(true),
|
||||
memgraph::storage::PropertyValue(123),
|
||||
memgraph::storage::PropertyValue()};
|
||||
std::map<std::string, memgraph::storage::PropertyValue> map{{"nandare", memgraph::storage::PropertyValue(false)}};
|
||||
const memgraph::storage::TemporalData temporal{memgraph::storage::TemporalType::LocalDateTime, 23};
|
||||
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> data{
|
||||
{memgraph::storage::PropertyId::FromInt(1), memgraph::storage::PropertyValue(true)},
|
||||
{memgraph::storage::PropertyId::FromInt(2), memgraph::storage::PropertyValue(123)},
|
||||
{memgraph::storage::PropertyId::FromInt(3), memgraph::storage::PropertyValue(123.5)},
|
||||
{memgraph::storage::PropertyId::FromInt(4), memgraph::storage::PropertyValue("nandare")},
|
||||
{memgraph::storage::PropertyId::FromInt(5), memgraph::storage::PropertyValue(vec)},
|
||||
{memgraph::storage::PropertyId::FromInt(6), memgraph::storage::PropertyValue(map)},
|
||||
{memgraph::storage::PropertyId::FromInt(7), memgraph::storage::PropertyValue(temporal)}};
|
||||
|
||||
std::vector<memgraph::storage::PropertyValue> alt{memgraph::storage::PropertyValue(),
|
||||
memgraph::storage::PropertyValue(std::string()),
|
||||
memgraph::storage::PropertyValue(std::string(10, 'a')),
|
||||
memgraph::storage::PropertyValue(std::string(100, 'a')),
|
||||
memgraph::storage::PropertyValue(std::string(1000, 'a')),
|
||||
memgraph::storage::PropertyValue(std::string(10000, 'a')),
|
||||
memgraph::storage::PropertyValue(std::string(100000, 'a'))};
|
||||
|
||||
memgraph::storage::PropertyStore props;
|
||||
for (const auto &target : data) {
|
||||
for (const auto &item : data) {
|
||||
ASSERT_TRUE(props.SetProperty(item.first, item.second));
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < alt.size(); ++i) {
|
||||
if (i == 1) {
|
||||
ASSERT_TRUE(props.SetProperty(target.first, alt[i]));
|
||||
} else {
|
||||
ASSERT_FALSE(props.SetProperty(target.first, alt[i]));
|
||||
}
|
||||
for (const auto &item : data) {
|
||||
if (item.first == target.first) {
|
||||
ASSERT_EQ(props.GetProperty(item.first), alt[i]);
|
||||
if (alt[i].IsNull()) {
|
||||
ASSERT_FALSE(props.HasProperty(item.first));
|
||||
} else {
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
}
|
||||
TestIsPropertyEqual(props, item.first, alt[i]);
|
||||
} else {
|
||||
ASSERT_EQ(props.GetProperty(item.first), item.second);
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, item.second);
|
||||
}
|
||||
}
|
||||
auto current = data;
|
||||
if (alt[i].IsNull()) {
|
||||
current.erase(target.first);
|
||||
} else {
|
||||
current[target.first] = alt[i];
|
||||
}
|
||||
ASSERT_EQ(props.Properties(), current);
|
||||
}
|
||||
|
||||
for (ssize_t i = alt.size() - 1; i >= 0; --i) {
|
||||
ASSERT_FALSE(props.SetProperty(target.first, alt[i]));
|
||||
for (const auto &item : data) {
|
||||
if (item.first == target.first) {
|
||||
ASSERT_EQ(props.GetProperty(item.first), alt[i]);
|
||||
if (alt[i].IsNull()) {
|
||||
ASSERT_FALSE(props.HasProperty(item.first));
|
||||
} else {
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
}
|
||||
TestIsPropertyEqual(props, item.first, alt[i]);
|
||||
} else {
|
||||
ASSERT_EQ(props.GetProperty(item.first), item.second);
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, item.second);
|
||||
}
|
||||
}
|
||||
auto current = data;
|
||||
if (alt[i].IsNull()) {
|
||||
current.erase(target.first);
|
||||
} else {
|
||||
current[target.first] = alt[i];
|
||||
}
|
||||
ASSERT_EQ(props.Properties(), current);
|
||||
}
|
||||
|
||||
ASSERT_TRUE(props.SetProperty(target.first, target.second));
|
||||
ASSERT_EQ(props.GetProperty(target.first), target.second);
|
||||
ASSERT_TRUE(props.HasProperty(target.first));
|
||||
TestIsPropertyEqual(props, target.first, target.second);
|
||||
|
||||
props.ClearProperties();
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
for (const auto &item : data) {
|
||||
ASSERT_TRUE(props.GetProperty(item.first).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, memgraph::storage::PropertyValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IntEncoding) {
|
||||
std::map<memgraph::storage::PropertyId, memgraph::storage::PropertyValue> data{
|
||||
{memgraph::storage::PropertyId::FromUint(0UL),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int64_t>::min())},
|
||||
{memgraph::storage::PropertyId::FromUint(10UL), memgraph::storage::PropertyValue(-137438953472L)},
|
||||
{memgraph::storage::PropertyId::FromUint(std::numeric_limits<uint8_t>::max()),
|
||||
memgraph::storage::PropertyValue(-4294967297L)},
|
||||
{memgraph::storage::PropertyId::FromUint(256UL),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int32_t>::min())},
|
||||
{memgraph::storage::PropertyId::FromUint(1024UL), memgraph::storage::PropertyValue(-1048576L)},
|
||||
{memgraph::storage::PropertyId::FromUint(1025UL), memgraph::storage::PropertyValue(-65537L)},
|
||||
{memgraph::storage::PropertyId::FromUint(1026UL),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int16_t>::min())},
|
||||
{memgraph::storage::PropertyId::FromUint(1027UL), memgraph::storage::PropertyValue(-1024L)},
|
||||
{memgraph::storage::PropertyId::FromUint(2000UL), memgraph::storage::PropertyValue(-257L)},
|
||||
{memgraph::storage::PropertyId::FromUint(3000UL),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int8_t>::min())},
|
||||
{memgraph::storage::PropertyId::FromUint(4000UL), memgraph::storage::PropertyValue(-1L)},
|
||||
{memgraph::storage::PropertyId::FromUint(10000UL), memgraph::storage::PropertyValue(0L)},
|
||||
{memgraph::storage::PropertyId::FromUint(20000UL), memgraph::storage::PropertyValue(1L)},
|
||||
{memgraph::storage::PropertyId::FromUint(30000UL),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int8_t>::max())},
|
||||
{memgraph::storage::PropertyId::FromUint(40000UL), memgraph::storage::PropertyValue(256L)},
|
||||
{memgraph::storage::PropertyId::FromUint(50000UL), memgraph::storage::PropertyValue(1024L)},
|
||||
{memgraph::storage::PropertyId::FromUint(std::numeric_limits<uint16_t>::max()),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int16_t>::max())},
|
||||
{memgraph::storage::PropertyId::FromUint(65536UL), memgraph::storage::PropertyValue(65536L)},
|
||||
{memgraph::storage::PropertyId::FromUint(1048576UL), memgraph::storage::PropertyValue(1048576L)},
|
||||
{memgraph::storage::PropertyId::FromUint(std::numeric_limits<uint32_t>::max()),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int32_t>::max())},
|
||||
{memgraph::storage::PropertyId::FromUint(4294967296UL), memgraph::storage::PropertyValue(4294967296L)},
|
||||
{memgraph::storage::PropertyId::FromUint(137438953472UL), memgraph::storage::PropertyValue(137438953472L)},
|
||||
{memgraph::storage::PropertyId::FromUint(std::numeric_limits<uint64_t>::max()),
|
||||
memgraph::storage::PropertyValue(std::numeric_limits<int64_t>::max())}};
|
||||
|
||||
memgraph::storage::PropertyStore props;
|
||||
for (const auto &item : data) {
|
||||
ASSERT_TRUE(props.SetProperty(item.first, item.second));
|
||||
ASSERT_EQ(props.GetProperty(item.first), item.second);
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, item.second);
|
||||
}
|
||||
for (auto it = data.rbegin(); it != data.rend(); ++it) {
|
||||
const auto &item = *it;
|
||||
ASSERT_FALSE(props.SetProperty(item.first, item.second));
|
||||
ASSERT_EQ(props.GetProperty(item.first), item.second);
|
||||
ASSERT_TRUE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, item.second);
|
||||
}
|
||||
|
||||
ASSERT_EQ(props.Properties(), data);
|
||||
|
||||
props.ClearProperties();
|
||||
ASSERT_EQ(props.Properties().size(), 0);
|
||||
for (const auto &item : data) {
|
||||
ASSERT_TRUE(props.GetProperty(item.first).IsNull());
|
||||
ASSERT_FALSE(props.HasProperty(item.first));
|
||||
TestIsPropertyEqual(props, item.first, memgraph::storage::PropertyValue());
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IsPropertyEqualIntAndDouble) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
|
||||
ASSERT_TRUE(props.SetProperty(prop, memgraph::storage::PropertyValue(42)));
|
||||
|
||||
std::vector<std::pair<memgraph::storage::PropertyValue, memgraph::storage::PropertyValue>> tests{
|
||||
{memgraph::storage::PropertyValue(0), memgraph::storage::PropertyValue(0.0)},
|
||||
{memgraph::storage::PropertyValue(123), memgraph::storage::PropertyValue(123.0)},
|
||||
{memgraph::storage::PropertyValue(12345), memgraph::storage::PropertyValue(12345.0)},
|
||||
{memgraph::storage::PropertyValue(12345678), memgraph::storage::PropertyValue(12345678.0)},
|
||||
{memgraph::storage::PropertyValue(1234567890123L), memgraph::storage::PropertyValue(1234567890123.0)},
|
||||
};
|
||||
|
||||
// Test equality with raw values.
|
||||
for (auto test : tests) {
|
||||
ASSERT_EQ(test.first, test.second);
|
||||
|
||||
// Test first, second
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.first));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.first);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
|
||||
// Test second, first
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.second));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.second);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
|
||||
// Make both negative
|
||||
test.first = memgraph::storage::PropertyValue(test.first.ValueInt() * -1);
|
||||
test.second = memgraph::storage::PropertyValue(test.second.ValueDouble() * -1.0);
|
||||
ASSERT_EQ(test.first, test.second);
|
||||
|
||||
// Test -first, -second
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.first));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.first);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
|
||||
// Test -second, -first
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.second));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.second);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
}
|
||||
|
||||
// Test equality with values wrapped in lists.
|
||||
for (auto test : tests) {
|
||||
test.first = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(test.first.ValueInt())});
|
||||
test.second = memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(test.second.ValueDouble())});
|
||||
ASSERT_EQ(test.first, test.second);
|
||||
|
||||
// Test first, second
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.first));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.first);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
|
||||
// Test second, first
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.second));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.second);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
|
||||
// Make both negative
|
||||
test.first = memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(test.first.ValueList()[0].ValueInt() * -1)});
|
||||
test.second = memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(test.second.ValueList()[0].ValueDouble() * -1.0)});
|
||||
ASSERT_EQ(test.first, test.second);
|
||||
|
||||
// Test -first, -second
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.first));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.first);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
|
||||
// Test -second, -first
|
||||
ASSERT_FALSE(props.SetProperty(prop, test.second));
|
||||
ASSERT_EQ(props.GetProperty(prop), test.second);
|
||||
ASSERT_TRUE(props.HasProperty(prop));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.second));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, test.first));
|
||||
}
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IsPropertyEqualString) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
ASSERT_TRUE(props.SetProperty(prop, memgraph::storage::PropertyValue("test")));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue("test")));
|
||||
|
||||
// Different length.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue("helloworld")));
|
||||
|
||||
// Same length, different value.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue("asdf")));
|
||||
|
||||
// Shortened and extended.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue("tes")));
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue("testt")));
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IsPropertyEqualList) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
ASSERT_TRUE(
|
||||
props.SetProperty(prop, memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(42), memgraph::storage::PropertyValue("test")})));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(42), memgraph::storage::PropertyValue("test")})));
|
||||
|
||||
// Different length.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(24)})));
|
||||
|
||||
// Same length, different value.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(42), memgraph::storage::PropertyValue("asdf")})));
|
||||
|
||||
// Shortened and extended.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(
|
||||
std::vector<memgraph::storage::PropertyValue>{memgraph::storage::PropertyValue(42)})));
|
||||
ASSERT_FALSE(
|
||||
props.IsPropertyEqual(prop, memgraph::storage::PropertyValue(std::vector<memgraph::storage::PropertyValue>{
|
||||
memgraph::storage::PropertyValue(42), memgraph::storage::PropertyValue("test"),
|
||||
memgraph::storage::PropertyValue(true)})));
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IsPropertyEqualMap) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
ASSERT_TRUE(props.SetProperty(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}, {"zyx", memgraph::storage::PropertyValue("test")}})));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}, {"zyx", memgraph::storage::PropertyValue("test")}})));
|
||||
|
||||
// Different length.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"fgh", memgraph::storage::PropertyValue(24)}})));
|
||||
|
||||
// Same length, different value.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}, {"zyx", memgraph::storage::PropertyValue("testt")}})));
|
||||
|
||||
// Same length, different key (different length).
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}, {"zyxw", memgraph::storage::PropertyValue("test")}})));
|
||||
|
||||
// Same length, different key (same length).
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}, {"zyw", memgraph::storage::PropertyValue("test")}})));
|
||||
|
||||
// Shortened and extended.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)}})));
|
||||
ASSERT_FALSE(props.IsPropertyEqual(
|
||||
prop, memgraph::storage::PropertyValue(std::map<std::string, memgraph::storage::PropertyValue>{
|
||||
{"abc", memgraph::storage::PropertyValue(42)},
|
||||
{"sdf", memgraph::storage::PropertyValue(true)},
|
||||
{"zyx", memgraph::storage::PropertyValue("test")}})));
|
||||
}
|
||||
|
||||
TEST(PropertyStore, IsPropertyEqualTemporalData) {
|
||||
memgraph::storage::PropertyStore props;
|
||||
auto prop = memgraph::storage::PropertyId::FromInt(42);
|
||||
const memgraph::storage::TemporalData temporal{memgraph::storage::TemporalType::Date, 23};
|
||||
ASSERT_TRUE(props.SetProperty(prop, memgraph::storage::PropertyValue(temporal)));
|
||||
ASSERT_TRUE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue(temporal)));
|
||||
|
||||
// Different type.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue(memgraph::storage::TemporalData{
|
||||
memgraph::storage::TemporalType::Duration, 23})));
|
||||
|
||||
// Same type, different value.
|
||||
ASSERT_FALSE(props.IsPropertyEqual(prop, memgraph::storage::PropertyValue(memgraph::storage::TemporalData{
|
||||
memgraph::storage::TemporalType::Date, 30})));
|
||||
}
|
@ -1,828 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <chrono>
|
||||
#include <thread>
|
||||
|
||||
#include <fmt/format.h>
|
||||
#include <gmock/gmock-generated-matchers.h>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <storage/v2/property_value.hpp>
|
||||
#include <storage/v2/replication/enums.hpp>
|
||||
#include <storage/v2/storage.hpp>
|
||||
#include "storage/v2/view.hpp"
|
||||
|
||||
using testing::UnorderedElementsAre;
|
||||
|
||||
class ReplicationTest : public ::testing::Test {
|
||||
protected:
|
||||
std::filesystem::path storage_directory{std::filesystem::temp_directory_path() /
|
||||
"MG_test_unit_storage_v2_replication"};
|
||||
void SetUp() override { Clear(); }
|
||||
|
||||
void TearDown() override { Clear(); }
|
||||
|
||||
memgraph::storage::Config configuration{
|
||||
.items = {.properties_on_edges = true},
|
||||
.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
}};
|
||||
|
||||
const std::string local_host = ("127.0.0.1");
|
||||
const std::array<uint16_t, 2> ports{10000, 20000};
|
||||
const std::array<std::string, 2> replicas = {"REPLICA1", "REPLICA2"};
|
||||
|
||||
private:
|
||||
void Clear() {
|
||||
if (!std::filesystem::exists(storage_directory)) return;
|
||||
std::filesystem::remove_all(storage_directory);
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(ReplicationTest, BasicSynchronousReplicationTest) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store(configuration);
|
||||
replica_store.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica("REPLICA", memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
// vertex create
|
||||
// vertex add label
|
||||
// vertex set property
|
||||
const auto *vertex_label = "vertex_label";
|
||||
const auto *vertex_property = "vertex_property";
|
||||
const auto *vertex_property_value = "vertex_property_value";
|
||||
std::optional<memgraph::storage::Gid> vertex_gid;
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_TRUE(v.AddLabel(main_store.NameToLabel(vertex_label)).HasValue());
|
||||
ASSERT_TRUE(v.SetProperty(main_store.NameToProperty(vertex_property),
|
||||
memgraph::storage::PropertyValue(vertex_property_value))
|
||||
.HasValue());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto labels = v->Labels(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(labels.HasValue());
|
||||
ASSERT_EQ(labels->size(), 1);
|
||||
ASSERT_THAT(*labels, UnorderedElementsAre(replica_store.NameToLabel(vertex_label)));
|
||||
const auto properties = v->Properties(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(properties.HasValue());
|
||||
ASSERT_EQ(properties->size(), 1);
|
||||
ASSERT_THAT(*properties,
|
||||
UnorderedElementsAre(std::make_pair(replica_store.NameToProperty(vertex_property),
|
||||
memgraph::storage::PropertyValue(vertex_property_value))));
|
||||
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// vertex remove label
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_TRUE(v->RemoveLabel(main_store.NameToLabel(vertex_label)).HasValue());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto labels = v->Labels(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(labels.HasValue());
|
||||
ASSERT_EQ(labels->size(), 0);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// vertex delete
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_TRUE(acc.DeleteVertex(&*v).HasValue());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_FALSE(v);
|
||||
vertex_gid.reset();
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// edge create
|
||||
// edge set property
|
||||
const auto *edge_type = "edge_type";
|
||||
const auto *edge_property = "edge_property";
|
||||
const auto *edge_property_value = "edge_property_value";
|
||||
std::optional<memgraph::storage::Gid> edge_gid;
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
auto edge = acc.CreateEdge(&v, &v, main_store.NameToEdgeType(edge_type));
|
||||
ASSERT_TRUE(edge.HasValue());
|
||||
ASSERT_TRUE(edge->SetProperty(main_store.NameToProperty(edge_property),
|
||||
memgraph::storage::PropertyValue(edge_property_value))
|
||||
.HasValue());
|
||||
edge_gid.emplace(edge->Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
const auto find_edge = [&](const auto &edges,
|
||||
const memgraph::storage::Gid edge_gid) -> std::optional<memgraph::storage::EdgeAccessor> {
|
||||
for (const auto &edge : edges) {
|
||||
if (edge.Gid() == edge_gid) {
|
||||
return edge;
|
||||
}
|
||||
}
|
||||
return std::nullopt;
|
||||
};
|
||||
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto out_edges = v->OutEdges(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(out_edges.HasValue());
|
||||
const auto edge = find_edge(*out_edges, *edge_gid);
|
||||
ASSERT_EQ(edge->EdgeType(), replica_store.NameToEdgeType(edge_type));
|
||||
const auto properties = edge->Properties(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(properties.HasValue());
|
||||
ASSERT_EQ(properties->size(), 1);
|
||||
ASSERT_THAT(*properties,
|
||||
UnorderedElementsAre(std::make_pair(replica_store.NameToProperty(edge_property),
|
||||
memgraph::storage::PropertyValue(edge_property_value))));
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// delete edge
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
auto out_edges = v->OutEdges(memgraph::storage::View::OLD);
|
||||
auto edge = find_edge(*out_edges, *edge_gid);
|
||||
ASSERT_TRUE(edge);
|
||||
ASSERT_TRUE(acc.DeleteEdge(&*edge).HasValue());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto out_edges = v->OutEdges(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(out_edges.HasValue());
|
||||
ASSERT_FALSE(find_edge(*out_edges, *edge_gid));
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// label index create
|
||||
// label property index create
|
||||
// existence constraint create
|
||||
// unique constriant create
|
||||
const auto *label = "label";
|
||||
const auto *property = "property";
|
||||
const auto *property_extra = "property_extra";
|
||||
{
|
||||
ASSERT_TRUE(main_store.CreateIndex(main_store.NameToLabel(label)));
|
||||
ASSERT_TRUE(main_store.CreateIndex(main_store.NameToLabel(label), main_store.NameToProperty(property)));
|
||||
ASSERT_FALSE(
|
||||
main_store.CreateExistenceConstraint(main_store.NameToLabel(label), main_store.NameToProperty(property))
|
||||
.HasError());
|
||||
ASSERT_FALSE(main_store
|
||||
.CreateUniqueConstraint(main_store.NameToLabel(label), {main_store.NameToProperty(property),
|
||||
main_store.NameToProperty(property_extra)})
|
||||
.HasError());
|
||||
}
|
||||
|
||||
{
|
||||
const auto indices = replica_store.ListAllIndices();
|
||||
ASSERT_THAT(indices.label, UnorderedElementsAre(replica_store.NameToLabel(label)));
|
||||
ASSERT_THAT(indices.label_property, UnorderedElementsAre(std::make_pair(replica_store.NameToLabel(label),
|
||||
replica_store.NameToProperty(property))));
|
||||
|
||||
const auto constraints = replica_store.ListAllConstraints();
|
||||
ASSERT_THAT(constraints.existence, UnorderedElementsAre(std::make_pair(replica_store.NameToLabel(label),
|
||||
replica_store.NameToProperty(property))));
|
||||
ASSERT_THAT(constraints.unique,
|
||||
UnorderedElementsAre(std::make_pair(
|
||||
replica_store.NameToLabel(label),
|
||||
std::set{replica_store.NameToProperty(property), replica_store.NameToProperty(property_extra)})));
|
||||
}
|
||||
|
||||
// label index drop
|
||||
// label property index drop
|
||||
// existence constraint drop
|
||||
// unique constriant drop
|
||||
{
|
||||
ASSERT_TRUE(main_store.DropIndex(main_store.NameToLabel(label)));
|
||||
ASSERT_TRUE(main_store.DropIndex(main_store.NameToLabel(label), main_store.NameToProperty(property)));
|
||||
ASSERT_TRUE(main_store.DropExistenceConstraint(main_store.NameToLabel(label), main_store.NameToProperty(property)));
|
||||
ASSERT_EQ(
|
||||
main_store.DropUniqueConstraint(main_store.NameToLabel(label), {main_store.NameToProperty(property),
|
||||
main_store.NameToProperty(property_extra)}),
|
||||
memgraph::storage::UniqueConstraints::DeletionStatus::SUCCESS);
|
||||
}
|
||||
|
||||
{
|
||||
const auto indices = replica_store.ListAllIndices();
|
||||
ASSERT_EQ(indices.label.size(), 0);
|
||||
ASSERT_EQ(indices.label_property.size(), 0);
|
||||
|
||||
const auto constraints = replica_store.ListAllConstraints();
|
||||
ASSERT_EQ(constraints.existence.size(), 0);
|
||||
ASSERT_EQ(constraints.unique.size(), 0);
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, MultipleSynchronousReplicationTest) {
|
||||
memgraph::storage::Storage main_store(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
}});
|
||||
|
||||
memgraph::storage::Storage replica_store1(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
}});
|
||||
replica_store1.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
memgraph::storage::Storage replica_store2(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
}});
|
||||
replica_store2.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[1]});
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replicas[1], memgraph::io::network::Endpoint{local_host, ports[1]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
const auto *vertex_label = "label";
|
||||
const auto *vertex_property = "property";
|
||||
const auto *vertex_property_value = "property_value";
|
||||
std::optional<memgraph::storage::Gid> vertex_gid;
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
ASSERT_TRUE(v.AddLabel(main_store.NameToLabel(vertex_label)).HasValue());
|
||||
ASSERT_TRUE(v.SetProperty(main_store.NameToProperty(vertex_property),
|
||||
memgraph::storage::PropertyValue(vertex_property_value))
|
||||
.HasValue());
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
const auto check_replica = [&](memgraph::storage::Storage *replica_store) {
|
||||
auto acc = replica_store->Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto labels = v->Labels(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(labels.HasValue());
|
||||
ASSERT_THAT(*labels, UnorderedElementsAre(replica_store->NameToLabel(vertex_label)));
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
};
|
||||
|
||||
check_replica(&replica_store1);
|
||||
check_replica(&replica_store2);
|
||||
|
||||
main_store.UnregisterReplica(replicas[1]);
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// REPLICA1 should contain the new vertex
|
||||
{
|
||||
auto acc = replica_store1.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
// REPLICA2 should not contain the new vertex
|
||||
{
|
||||
auto acc = replica_store2.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_FALSE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, RecoveryProcess) {
|
||||
std::vector<memgraph::storage::Gid> vertex_gids;
|
||||
// Force the creation of snapshot
|
||||
{
|
||||
memgraph::storage::Storage main_store(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.recover_on_startup = true,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
.snapshot_on_exit = true,
|
||||
}});
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
// Create the vertex before registering a replica
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gids.emplace_back(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// Create second WAL
|
||||
memgraph::storage::Storage main_store(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.recover_on_startup = true,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL}});
|
||||
// Create vertices in 2 different transactions
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gids.emplace_back(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gids.emplace_back(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
memgraph::storage::Storage main_store(
|
||||
{.durability = {
|
||||
.storage_directory = storage_directory,
|
||||
.recover_on_startup = true,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL,
|
||||
}});
|
||||
|
||||
static constexpr const auto *property_name = "property_name";
|
||||
static constexpr const auto property_value = 1;
|
||||
{
|
||||
// Force the creation of current WAL file
|
||||
auto acc = main_store.Access();
|
||||
for (const auto &vertex_gid : vertex_gids) {
|
||||
auto v = acc.FindVertex(vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_TRUE(
|
||||
v->SetProperty(main_store.NameToProperty(property_name), memgraph::storage::PropertyValue(property_value))
|
||||
.HasValue());
|
||||
}
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
std::filesystem::path replica_storage_directory{std::filesystem::temp_directory_path() /
|
||||
"MG_test_unit_storage_v2_replication_replica"};
|
||||
memgraph::utils::OnScopeExit replica_directory_cleaner(
|
||||
[&]() { std::filesystem::remove_all(replica_storage_directory); });
|
||||
|
||||
static constexpr const auto *vertex_label = "vertex_label";
|
||||
{
|
||||
memgraph::storage::Storage replica_store(
|
||||
{.durability = {
|
||||
.storage_directory = replica_storage_directory,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL}});
|
||||
|
||||
replica_store.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
ASSERT_EQ(main_store.GetReplicaState(replicas[0]), memgraph::storage::replication::ReplicaState::RECOVERY);
|
||||
|
||||
while (main_store.GetReplicaState(replicas[0]) != memgraph::storage::replication::ReplicaState::READY) {
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||
}
|
||||
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
for (const auto &vertex_gid : vertex_gids) {
|
||||
auto v = acc.FindVertex(vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_TRUE(v->AddLabel(main_store.NameToLabel(vertex_label)).HasValue());
|
||||
}
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
for (const auto &vertex_gid : vertex_gids) {
|
||||
auto v = acc.FindVertex(vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto labels = v->Labels(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(labels.HasValue());
|
||||
ASSERT_THAT(*labels, UnorderedElementsAre(replica_store.NameToLabel(vertex_label)));
|
||||
const auto properties = v->Properties(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(properties.HasValue());
|
||||
ASSERT_THAT(*properties,
|
||||
UnorderedElementsAre(std::make_pair(replica_store.NameToProperty(property_name),
|
||||
memgraph::storage::PropertyValue(property_value))));
|
||||
}
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
{
|
||||
memgraph::storage::Storage replica_store(
|
||||
{.durability = {
|
||||
.storage_directory = replica_storage_directory,
|
||||
.recover_on_startup = true,
|
||||
.snapshot_wal_mode = memgraph::storage::Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL}});
|
||||
{
|
||||
auto acc = replica_store.Access();
|
||||
for (const auto &vertex_gid : vertex_gids) {
|
||||
auto v = acc.FindVertex(vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
const auto labels = v->Labels(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(labels.HasValue());
|
||||
ASSERT_THAT(*labels, UnorderedElementsAre(replica_store.NameToLabel(vertex_label)));
|
||||
const auto properties = v->Properties(memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(properties.HasValue());
|
||||
ASSERT_THAT(*properties,
|
||||
UnorderedElementsAre(std::make_pair(replica_store.NameToProperty(property_name),
|
||||
memgraph::storage::PropertyValue(property_value))));
|
||||
}
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, BasicAsynchronousReplicationTest) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store_async(configuration);
|
||||
|
||||
replica_store_async.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[1]});
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica("REPLICA_ASYNC", memgraph::io::network::Endpoint{local_host, ports[1]},
|
||||
memgraph::storage::replication::ReplicationMode::ASYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
static constexpr size_t vertices_create_num = 10;
|
||||
std::vector<memgraph::storage::Gid> created_vertices;
|
||||
for (size_t i = 0; i < vertices_create_num; ++i) {
|
||||
auto acc = main_store.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
created_vertices.push_back(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
|
||||
if (i == 0) {
|
||||
ASSERT_EQ(main_store.GetReplicaState("REPLICA_ASYNC"), memgraph::storage::replication::ReplicaState::REPLICATING);
|
||||
} else {
|
||||
ASSERT_EQ(main_store.GetReplicaState("REPLICA_ASYNC"), memgraph::storage::replication::ReplicaState::RECOVERY);
|
||||
}
|
||||
}
|
||||
|
||||
while (main_store.GetReplicaState("REPLICA_ASYNC") != memgraph::storage::replication::ReplicaState::READY) {
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||
}
|
||||
|
||||
ASSERT_TRUE(std::all_of(created_vertices.begin(), created_vertices.end(), [&](const auto vertex_gid) {
|
||||
auto acc = replica_store_async.Access();
|
||||
auto v = acc.FindVertex(vertex_gid, memgraph::storage::View::OLD);
|
||||
const bool exists = v.has_value();
|
||||
EXPECT_FALSE(acc.Commit().HasError());
|
||||
return exists;
|
||||
}));
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, EpochTest) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
|
||||
replica_store1.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
|
||||
replica_store2.SetReplicaRole(memgraph::io::network::Endpoint{local_host, 10001});
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replicas[1], memgraph::io::network::Endpoint{local_host, 10001},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
std::optional<memgraph::storage::Gid> vertex_gid;
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
const auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc = replica_store1.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc = replica_store2.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
main_store.UnregisterReplica(replicas[0]);
|
||||
main_store.UnregisterReplica(replicas[1]);
|
||||
|
||||
replica_store1.SetMainReplicationRole();
|
||||
ASSERT_FALSE(replica_store1
|
||||
.RegisterReplica(replicas[1], memgraph::io::network::Endpoint{local_host, 10001},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
acc.CreateVertex();
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
{
|
||||
auto acc = replica_store1.Access();
|
||||
auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
// Replica1 should forward it's vertex to Replica2
|
||||
{
|
||||
auto acc = replica_store2.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_TRUE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
|
||||
replica_store1.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
ASSERT_TRUE(main_store
|
||||
.RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
{
|
||||
auto acc = main_store.Access();
|
||||
const auto v = acc.CreateVertex();
|
||||
vertex_gid.emplace(v.Gid());
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
// Replica1 is not compatible with the main so it shouldn't contain
|
||||
// it's newest vertex
|
||||
{
|
||||
auto acc = replica_store1.Access();
|
||||
const auto v = acc.FindVertex(*vertex_gid, memgraph::storage::View::OLD);
|
||||
ASSERT_FALSE(v);
|
||||
ASSERT_FALSE(acc.Commit().HasError());
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, ReplicationInformation) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
|
||||
const memgraph::io::network::Endpoint replica1_endpoint{local_host, 10001};
|
||||
replica_store1.SetReplicaRole(replica1_endpoint);
|
||||
|
||||
const memgraph::io::network::Endpoint replica2_endpoint{local_host, 10002};
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
|
||||
replica_store2.SetReplicaRole(replica2_endpoint);
|
||||
|
||||
const std::string replica1_name{replicas[0]};
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replica1_name, replica1_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
const std::string replica2_name{replicas[1]};
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replica2_name, replica2_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::ASYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
ASSERT_EQ(main_store.GetReplicationRole(), memgraph::storage::ReplicationRole::MAIN);
|
||||
ASSERT_EQ(replica_store1.GetReplicationRole(), memgraph::storage::ReplicationRole::REPLICA);
|
||||
ASSERT_EQ(replica_store2.GetReplicationRole(), memgraph::storage::ReplicationRole::REPLICA);
|
||||
|
||||
const auto replicas_info = main_store.ReplicasInfo();
|
||||
ASSERT_EQ(replicas_info.size(), 2);
|
||||
|
||||
const auto &first_info = replicas_info[0];
|
||||
ASSERT_EQ(first_info.name, replica1_name);
|
||||
ASSERT_EQ(first_info.mode, memgraph::storage::replication::ReplicationMode::SYNC);
|
||||
ASSERT_EQ(first_info.endpoint, replica1_endpoint);
|
||||
ASSERT_EQ(first_info.state, memgraph::storage::replication::ReplicaState::READY);
|
||||
|
||||
const auto &second_info = replicas_info[1];
|
||||
ASSERT_EQ(second_info.name, replica2_name);
|
||||
ASSERT_EQ(second_info.mode, memgraph::storage::replication::ReplicationMode::ASYNC);
|
||||
ASSERT_EQ(second_info.endpoint, replica2_endpoint);
|
||||
ASSERT_EQ(second_info.state, memgraph::storage::replication::ReplicaState::READY);
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, ReplicationReplicaWithExistingName) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
|
||||
const memgraph::io::network::Endpoint replica1_endpoint{local_host, 10001};
|
||||
replica_store1.SetReplicaRole(replica1_endpoint);
|
||||
|
||||
const memgraph::io::network::Endpoint replica2_endpoint{local_host, 10002};
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
|
||||
replica_store2.SetReplicaRole(replica2_endpoint);
|
||||
|
||||
const std::string replica1_name{replicas[0]};
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replica1_name, replica1_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
const std::string replica2_name{replicas[0]};
|
||||
ASSERT_TRUE(main_store
|
||||
.RegisterReplica(replica2_name, replica2_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::ASYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.GetError() == memgraph::storage::Storage::RegisterReplicaError::NAME_EXISTS);
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, ReplicationReplicaWithExistingEndPoint) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
|
||||
const memgraph::io::network::Endpoint replica1_endpoint{local_host, 10001};
|
||||
replica_store1.SetReplicaRole(replica1_endpoint);
|
||||
|
||||
const memgraph::io::network::Endpoint replica2_endpoint{local_host, 10001};
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
|
||||
replica_store2.SetReplicaRole(replica2_endpoint);
|
||||
|
||||
const std::string replica1_name{replicas[0]};
|
||||
ASSERT_FALSE(main_store
|
||||
.RegisterReplica(replica1_name, replica1_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.HasError());
|
||||
|
||||
const std::string replica2_name{replicas[1]};
|
||||
ASSERT_TRUE(main_store
|
||||
.RegisterReplica(replica2_name, replica2_endpoint,
|
||||
memgraph::storage::replication::ReplicationMode::ASYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.GetError() == memgraph::storage::Storage::RegisterReplicaError::END_POINT_EXISTS);
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, RestoringReplicationAtStartupAftgerDroppingReplica) {
|
||||
auto main_config = configuration;
|
||||
main_config.durability.restore_replicas_on_startup = true;
|
||||
auto main_store = std::make_unique<memgraph::storage::Storage>(main_config);
|
||||
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
replica_store1.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
replica_store2.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[1]});
|
||||
|
||||
auto res = main_store->RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID);
|
||||
ASSERT_FALSE(res.HasError());
|
||||
res = main_store->RegisterReplica(replicas[1], memgraph::io::network::Endpoint{local_host, ports[1]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID);
|
||||
ASSERT_FALSE(res.HasError());
|
||||
|
||||
auto replica_infos = main_store->ReplicasInfo();
|
||||
|
||||
ASSERT_EQ(replica_infos.size(), 2);
|
||||
ASSERT_EQ(replica_infos[0].name, replicas[0]);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.port, ports[0]);
|
||||
ASSERT_EQ(replica_infos[1].name, replicas[1]);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.port, ports[1]);
|
||||
|
||||
main_store.reset();
|
||||
|
||||
auto other_main_store = std::make_unique<memgraph::storage::Storage>(main_config);
|
||||
replica_infos = other_main_store->ReplicasInfo();
|
||||
ASSERT_EQ(replica_infos.size(), 2);
|
||||
ASSERT_EQ(replica_infos[0].name, replicas[0]);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.port, ports[0]);
|
||||
ASSERT_EQ(replica_infos[1].name, replicas[1]);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.port, ports[1]);
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, RestoringReplicationAtStartup) {
|
||||
auto main_config = configuration;
|
||||
main_config.durability.restore_replicas_on_startup = true;
|
||||
auto main_store = std::make_unique<memgraph::storage::Storage>(main_config);
|
||||
memgraph::storage::Storage replica_store1(configuration);
|
||||
replica_store1.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[0]});
|
||||
|
||||
memgraph::storage::Storage replica_store2(configuration);
|
||||
replica_store2.SetReplicaRole(memgraph::io::network::Endpoint{local_host, ports[1]});
|
||||
|
||||
auto res = main_store->RegisterReplica(replicas[0], memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID);
|
||||
ASSERT_FALSE(res.HasError());
|
||||
res = main_store->RegisterReplica(replicas[1], memgraph::io::network::Endpoint{local_host, ports[1]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID);
|
||||
ASSERT_FALSE(res.HasError());
|
||||
|
||||
auto replica_infos = main_store->ReplicasInfo();
|
||||
|
||||
ASSERT_EQ(replica_infos.size(), 2);
|
||||
ASSERT_EQ(replica_infos[0].name, replicas[0]);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.port, ports[0]);
|
||||
ASSERT_EQ(replica_infos[1].name, replicas[1]);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[1].endpoint.port, ports[1]);
|
||||
|
||||
const auto unregister_res = main_store->UnregisterReplica(replicas[0]);
|
||||
ASSERT_TRUE(unregister_res);
|
||||
|
||||
replica_infos = main_store->ReplicasInfo();
|
||||
ASSERT_EQ(replica_infos.size(), 1);
|
||||
ASSERT_EQ(replica_infos[0].name, replicas[1]);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.port, ports[1]);
|
||||
|
||||
main_store.reset();
|
||||
|
||||
auto other_main_store = std::make_unique<memgraph::storage::Storage>(main_config);
|
||||
replica_infos = other_main_store->ReplicasInfo();
|
||||
ASSERT_EQ(replica_infos.size(), 1);
|
||||
ASSERT_EQ(replica_infos[0].name, replicas[1]);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.address, local_host);
|
||||
ASSERT_EQ(replica_infos[0].endpoint.port, ports[1]);
|
||||
}
|
||||
|
||||
TEST_F(ReplicationTest, AddingInvalidReplica) {
|
||||
memgraph::storage::Storage main_store(configuration);
|
||||
|
||||
ASSERT_TRUE(main_store
|
||||
.RegisterReplica("REPLICA", memgraph::io::network::Endpoint{local_host, ports[0]},
|
||||
memgraph::storage::replication::ReplicationMode::SYNC,
|
||||
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID)
|
||||
.GetError() == memgraph::storage::Storage::RegisterReplicaError::CONNECTION_FAILED);
|
||||
}
|
@ -1,623 +0,0 @@
|
||||
// Copyright 2022 Memgraph Ltd.
|
||||
//
|
||||
// Use of this software is governed by the Business Source License
|
||||
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
|
||||
// License, and you may not use this file except in compliance with the Business Source License.
|
||||
//
|
||||
// As of the Change Date specified in that file, in accordance with
|
||||
// the Business Source License, use of this software will be governed
|
||||
// by the Apache License, Version 2.0, included in the file
|
||||
// licenses/APL.txt.
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <fmt/format.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <filesystem>
|
||||
#include <string_view>
|
||||
|
||||
#include "storage/v2/durability/exceptions.hpp"
|
||||
#include "storage/v2/durability/version.hpp"
|
||||
#include "storage/v2/durability/wal.hpp"
|
||||
#include "storage/v2/mvcc.hpp"
|
||||
#include "storage/v2/name_id_mapper.hpp"
|
||||
#include "utils/file.hpp"
|
||||
#include "utils/file_locker.hpp"
|
||||
#include "utils/uuid.hpp"
|
||||
|
||||
// Helper function used to convert between enum types.
|
||||
memgraph::storage::durability::WalDeltaData::Type StorageGlobalOperationToWalDeltaDataType(
|
||||
memgraph::storage::durability::StorageGlobalOperation operation) {
|
||||
switch (operation) {
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_INDEX_CREATE:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::LABEL_INDEX_CREATE;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_INDEX_DROP:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::LABEL_INDEX_DROP;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP:
|
||||
return memgraph::storage::durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP;
|
||||
}
|
||||
}
|
||||
|
||||
// This class mimics the internals of the storage to generate the deltas.
|
||||
class DeltaGenerator final {
|
||||
public:
|
||||
class Transaction final {
|
||||
private:
|
||||
friend class DeltaGenerator;
|
||||
|
||||
explicit Transaction(DeltaGenerator *gen)
|
||||
: gen_(gen),
|
||||
transaction_(gen->transaction_id_++, gen->timestamp_++,
|
||||
memgraph::storage::IsolationLevel::SNAPSHOT_ISOLATION) {}
|
||||
|
||||
public:
|
||||
memgraph::storage::Vertex *CreateVertex() {
|
||||
auto gid = memgraph::storage::Gid::FromUint(gen_->vertices_count_++);
|
||||
auto delta = memgraph::storage::CreateDeleteObjectDelta(&transaction_);
|
||||
auto &it = gen_->vertices_.emplace_back(gid, delta);
|
||||
delta->prev.Set(&it);
|
||||
{
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = memgraph::storage::durability::WalDeltaData::Type::VERTEX_CREATE;
|
||||
data.vertex_create_delete.gid = gid;
|
||||
data_.push_back(data);
|
||||
}
|
||||
return ⁢
|
||||
}
|
||||
|
||||
void DeleteVertex(memgraph::storage::Vertex *vertex) {
|
||||
memgraph::storage::CreateAndLinkDelta(&transaction_, &*vertex, memgraph::storage::Delta::RecreateObjectTag());
|
||||
{
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = memgraph::storage::durability::WalDeltaData::Type::VERTEX_DELETE;
|
||||
data.vertex_create_delete.gid = vertex->gid;
|
||||
data_.push_back(data);
|
||||
}
|
||||
}
|
||||
|
||||
void AddLabel(memgraph::storage::Vertex *vertex, const std::string &label) {
|
||||
auto label_id = memgraph::storage::LabelId::FromUint(gen_->mapper_.NameToId(label));
|
||||
vertex->labels.push_back(label_id);
|
||||
memgraph::storage::CreateAndLinkDelta(&transaction_, &*vertex, memgraph::storage::Delta::RemoveLabelTag(),
|
||||
label_id);
|
||||
{
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = memgraph::storage::durability::WalDeltaData::Type::VERTEX_ADD_LABEL;
|
||||
data.vertex_add_remove_label.gid = vertex->gid;
|
||||
data.vertex_add_remove_label.label = label;
|
||||
data_.push_back(data);
|
||||
}
|
||||
}
|
||||
|
||||
void RemoveLabel(memgraph::storage::Vertex *vertex, const std::string &label) {
|
||||
auto label_id = memgraph::storage::LabelId::FromUint(gen_->mapper_.NameToId(label));
|
||||
vertex->labels.erase(std::find(vertex->labels.begin(), vertex->labels.end(), label_id));
|
||||
memgraph::storage::CreateAndLinkDelta(&transaction_, &*vertex, memgraph::storage::Delta::AddLabelTag(), label_id);
|
||||
{
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = memgraph::storage::durability::WalDeltaData::Type::VERTEX_REMOVE_LABEL;
|
||||
data.vertex_add_remove_label.gid = vertex->gid;
|
||||
data.vertex_add_remove_label.label = label;
|
||||
data_.push_back(data);
|
||||
}
|
||||
}
|
||||
|
||||
void SetProperty(memgraph::storage::Vertex *vertex, const std::string &property,
|
||||
const memgraph::storage::PropertyValue &value) {
|
||||
auto property_id = memgraph::storage::PropertyId::FromUint(gen_->mapper_.NameToId(property));
|
||||
auto &props = vertex->properties;
|
||||
auto old_value = props.GetProperty(property_id);
|
||||
memgraph::storage::CreateAndLinkDelta(&transaction_, &*vertex, memgraph::storage::Delta::SetPropertyTag(),
|
||||
property_id, old_value);
|
||||
props.SetProperty(property_id, value);
|
||||
{
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = memgraph::storage::durability::WalDeltaData::Type::VERTEX_SET_PROPERTY;
|
||||
data.vertex_edge_set_property.gid = vertex->gid;
|
||||
data.vertex_edge_set_property.property = property;
|
||||
// We don't store the property value here. That is because the storage
|
||||
// generates multiple `SetProperty` deltas using only the final values
|
||||
// of the property. The intermediate values aren't encoded. The value is
|
||||
// later determined in the `Finalize` function.
|
||||
data_.push_back(data);
|
||||
}
|
||||
}
|
||||
|
||||
void Finalize(bool append_transaction_end = true) {
|
||||
auto commit_timestamp = gen_->timestamp_++;
|
||||
for (const auto &delta : transaction_.deltas) {
|
||||
auto owner = delta.prev.Get();
|
||||
while (owner.type == memgraph::storage::PreviousPtr::Type::DELTA) {
|
||||
owner = owner.delta->prev.Get();
|
||||
}
|
||||
if (owner.type == memgraph::storage::PreviousPtr::Type::VERTEX) {
|
||||
gen_->wal_file_.AppendDelta(delta, *owner.vertex, commit_timestamp);
|
||||
} else if (owner.type == memgraph::storage::PreviousPtr::Type::EDGE) {
|
||||
gen_->wal_file_.AppendDelta(delta, *owner.edge, commit_timestamp);
|
||||
} else {
|
||||
LOG_FATAL("Invalid delta owner!");
|
||||
}
|
||||
}
|
||||
if (append_transaction_end) {
|
||||
gen_->wal_file_.AppendTransactionEnd(commit_timestamp);
|
||||
if (gen_->valid_) {
|
||||
gen_->UpdateStats(commit_timestamp, transaction_.deltas.size() + 1);
|
||||
for (auto &data : data_) {
|
||||
if (data.type == memgraph::storage::durability::WalDeltaData::Type::VERTEX_SET_PROPERTY) {
|
||||
// We need to put the final property value into the SET_PROPERTY
|
||||
// delta.
|
||||
auto vertex =
|
||||
std::find(gen_->vertices_.begin(), gen_->vertices_.end(), data.vertex_edge_set_property.gid);
|
||||
ASSERT_NE(vertex, gen_->vertices_.end());
|
||||
auto property_id = memgraph::storage::PropertyId::FromUint(
|
||||
gen_->mapper_.NameToId(data.vertex_edge_set_property.property));
|
||||
data.vertex_edge_set_property.value = vertex->properties.GetProperty(property_id);
|
||||
}
|
||||
gen_->data_.emplace_back(commit_timestamp, data);
|
||||
}
|
||||
memgraph::storage::durability::WalDeltaData data{
|
||||
.type = memgraph::storage::durability::WalDeltaData::Type::TRANSACTION_END};
|
||||
gen_->data_.emplace_back(commit_timestamp, data);
|
||||
}
|
||||
} else {
|
||||
gen_->valid_ = false;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
DeltaGenerator *gen_;
|
||||
memgraph::storage::Transaction transaction_;
|
||||
std::vector<memgraph::storage::durability::WalDeltaData> data_;
|
||||
};
|
||||
|
||||
using DataT = std::vector<std::pair<uint64_t, memgraph::storage::durability::WalDeltaData>>;
|
||||
|
||||
DeltaGenerator(const std::filesystem::path &data_directory, bool properties_on_edges, uint64_t seq_num)
|
||||
: uuid_(memgraph::utils::GenerateUUID()),
|
||||
epoch_id_(memgraph::utils::GenerateUUID()),
|
||||
seq_num_(seq_num),
|
||||
wal_file_(data_directory, uuid_, epoch_id_, {.properties_on_edges = properties_on_edges}, &mapper_, seq_num,
|
||||
&file_retainer_) {}
|
||||
|
||||
Transaction CreateTransaction() { return Transaction(this); }
|
||||
|
||||
void ResetTransactionIds() {
|
||||
transaction_id_ = memgraph::storage::kTransactionInitialId;
|
||||
timestamp_ = memgraph::storage::kTimestampInitialId;
|
||||
valid_ = false;
|
||||
}
|
||||
|
||||
void AppendOperation(memgraph::storage::durability::StorageGlobalOperation operation, const std::string &label,
|
||||
const std::set<std::string> properties = {}) {
|
||||
auto label_id = memgraph::storage::LabelId::FromUint(mapper_.NameToId(label));
|
||||
std::set<memgraph::storage::PropertyId> property_ids;
|
||||
for (const auto &property : properties) {
|
||||
property_ids.insert(memgraph::storage::PropertyId::FromUint(mapper_.NameToId(property)));
|
||||
}
|
||||
wal_file_.AppendOperation(operation, label_id, property_ids, timestamp_);
|
||||
if (valid_) {
|
||||
UpdateStats(timestamp_, 1);
|
||||
memgraph::storage::durability::WalDeltaData data;
|
||||
data.type = StorageGlobalOperationToWalDeltaDataType(operation);
|
||||
switch (operation) {
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_INDEX_CREATE:
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_INDEX_DROP:
|
||||
data.operation_label.label = label;
|
||||
break;
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE:
|
||||
case memgraph::storage::durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP:
|
||||
case memgraph::storage::durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE:
|
||||
case memgraph::storage::durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP:
|
||||
data.operation_label_property.label = label;
|
||||
data.operation_label_property.property = *properties.begin();
|
||||
case memgraph::storage::durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE:
|
||||
case memgraph::storage::durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP:
|
||||
data.operation_label_properties.label = label;
|
||||
data.operation_label_properties.properties = properties;
|
||||
}
|
||||
data_.emplace_back(timestamp_, data);
|
||||
}
|
||||
}
|
||||
|
||||
uint64_t GetPosition() { return wal_file_.GetSize(); }
|
||||
|
||||
memgraph::storage::durability::WalInfo GetInfo() {
|
||||
return {.offset_metadata = 0,
|
||||
.offset_deltas = 0,
|
||||
.uuid = uuid_,
|
||||
.epoch_id = epoch_id_,
|
||||
.seq_num = seq_num_,
|
||||
.from_timestamp = tx_from_,
|
||||
.to_timestamp = tx_to_,
|
||||
.num_deltas = deltas_count_};
|
||||
}
|
||||
|
||||
DataT GetData() { return data_; }
|
||||
|
||||
private:
|
||||
void UpdateStats(uint64_t timestamp, uint64_t count) {
|
||||
if (deltas_count_ == 0) {
|
||||
tx_from_ = timestamp;
|
||||
}
|
||||
tx_to_ = timestamp;
|
||||
deltas_count_ += count;
|
||||
}
|
||||
|
||||
std::string uuid_;
|
||||
std::string epoch_id_;
|
||||
uint64_t seq_num_;
|
||||
|
||||
uint64_t transaction_id_{memgraph::storage::kTransactionInitialId};
|
||||
uint64_t timestamp_{memgraph::storage::kTimestampInitialId};
|
||||
uint64_t vertices_count_{0};
|
||||
std::list<memgraph::storage::Vertex> vertices_;
|
||||
memgraph::storage::NameIdMapper mapper_;
|
||||
|
||||
memgraph::storage::durability::WalFile wal_file_;
|
||||
|
||||
DataT data_;
|
||||
|
||||
uint64_t deltas_count_{0};
|
||||
uint64_t tx_from_{0};
|
||||
uint64_t tx_to_{0};
|
||||
uint64_t valid_{true};
|
||||
|
||||
memgraph::utils::FileRetainer file_retainer_;
|
||||
};
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define TRANSACTION(append_transaction_end, ops) \
|
||||
{ \
|
||||
auto tx = gen.CreateTransaction(); \
|
||||
ops; \
|
||||
tx.Finalize(append_transaction_end); \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define OPERATION(op, ...) gen.AppendOperation(memgraph::storage::durability::StorageGlobalOperation::op, __VA_ARGS__)
|
||||
|
||||
void AssertWalInfoEqual(const memgraph::storage::durability::WalInfo &a,
|
||||
const memgraph::storage::durability::WalInfo &b) {
|
||||
ASSERT_EQ(a.uuid, b.uuid);
|
||||
ASSERT_EQ(a.epoch_id, b.epoch_id);
|
||||
ASSERT_EQ(a.seq_num, b.seq_num);
|
||||
ASSERT_EQ(a.from_timestamp, b.from_timestamp);
|
||||
ASSERT_EQ(a.to_timestamp, b.to_timestamp);
|
||||
ASSERT_EQ(a.num_deltas, b.num_deltas);
|
||||
}
|
||||
|
||||
void AssertWalDataEqual(const DeltaGenerator::DataT &data, const std::filesystem::path &path) {
|
||||
auto info = memgraph::storage::durability::ReadWalInfo(path);
|
||||
memgraph::storage::durability::Decoder wal;
|
||||
wal.Initialize(path, memgraph::storage::durability::kWalMagic);
|
||||
wal.SetPosition(info.offset_deltas);
|
||||
DeltaGenerator::DataT current;
|
||||
for (uint64_t i = 0; i < info.num_deltas; ++i) {
|
||||
auto timestamp = memgraph::storage::durability::ReadWalDeltaHeader(&wal);
|
||||
current.emplace_back(timestamp, memgraph::storage::durability::ReadWalDeltaData(&wal));
|
||||
}
|
||||
ASSERT_EQ(data.size(), current.size());
|
||||
ASSERT_EQ(data, current);
|
||||
}
|
||||
|
||||
class WalFileTest : public ::testing::TestWithParam<bool> {
|
||||
public:
|
||||
WalFileTest() {}
|
||||
|
||||
void SetUp() override { Clear(); }
|
||||
|
||||
void TearDown() override { Clear(); }
|
||||
|
||||
std::vector<std::filesystem::path> GetFilesList() {
|
||||
std::vector<std::filesystem::path> ret;
|
||||
for (auto &item : std::filesystem::directory_iterator(storage_directory)) {
|
||||
ret.push_back(item.path());
|
||||
}
|
||||
std::sort(ret.begin(), ret.end());
|
||||
std::reverse(ret.begin(), ret.end());
|
||||
return ret;
|
||||
}
|
||||
|
||||
std::filesystem::path storage_directory{std::filesystem::temp_directory_path() / "MG_test_unit_storage_v2_wal_file"};
|
||||
|
||||
private:
|
||||
void Clear() {
|
||||
if (!std::filesystem::exists(storage_directory)) return;
|
||||
std::filesystem::remove_all(storage_directory);
|
||||
}
|
||||
};
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(EdgesWithProperties, WalFileTest, ::testing::Values(true));
|
||||
INSTANTIATE_TEST_CASE_P(EdgesWithoutProperties, WalFileTest, ::testing::Values(false));
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_P(WalFileTest, EmptyFile) {
|
||||
{ DeltaGenerator gen(storage_directory, GetParam(), 5); }
|
||||
auto wal_files = GetFilesList();
|
||||
ASSERT_EQ(wal_files.size(), 0);
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
|
||||
#define GENERATE_SIMPLE_TEST(name, ops) \
|
||||
TEST_P(WalFileTest, name) { \
|
||||
memgraph::storage::durability::WalInfo info; \
|
||||
DeltaGenerator::DataT data; \
|
||||
\
|
||||
{ \
|
||||
DeltaGenerator gen(storage_directory, GetParam(), 5); \
|
||||
ops; \
|
||||
info = gen.GetInfo(); \
|
||||
data = gen.GetData(); \
|
||||
} \
|
||||
\
|
||||
auto wal_files = GetFilesList(); \
|
||||
ASSERT_EQ(wal_files.size(), 1); \
|
||||
\
|
||||
if (info.num_deltas == 0) { \
|
||||
ASSERT_THROW(memgraph::storage::durability::ReadWalInfo(wal_files.front()), \
|
||||
memgraph::storage::durability::RecoveryFailure); \
|
||||
} else { \
|
||||
AssertWalInfoEqual(info, memgraph::storage::durability::ReadWalInfo(wal_files.front())); \
|
||||
AssertWalDataEqual(data, wal_files.front()); \
|
||||
} \
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionWithEnd, { TRANSACTION(true, { tx.CreateVertex(); }); });
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionWithoutEnd, { TRANSACTION(false, { tx.CreateVertex(); }); });
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(OperationSingle, { OPERATION(LABEL_INDEX_CREATE, "hello"); });
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsEnd00, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsEnd01, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsEnd10, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsEnd11, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation_00, {
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation_01, {
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation_10, {
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation_11, {
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation0_0, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation0_1, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation1_0, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation1_1, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation00_, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation01_, {
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation10_, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(false, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(TransactionsWithOperation11_, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(AllTransactionOperationsWithEnd, {
|
||||
TRANSACTION(true, {
|
||||
auto vertex1 = tx.CreateVertex();
|
||||
auto vertex2 = tx.CreateVertex();
|
||||
tx.AddLabel(vertex1, "test");
|
||||
tx.AddLabel(vertex2, "hello");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue("nandare"));
|
||||
tx.RemoveLabel(vertex1, "test");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue(123));
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue());
|
||||
tx.DeleteVertex(vertex1);
|
||||
});
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(AllTransactionOperationsWithoutEnd, {
|
||||
TRANSACTION(false, {
|
||||
auto vertex1 = tx.CreateVertex();
|
||||
auto vertex2 = tx.CreateVertex();
|
||||
tx.AddLabel(vertex1, "test");
|
||||
tx.AddLabel(vertex2, "hello");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue("nandare"));
|
||||
tx.RemoveLabel(vertex1, "test");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue(123));
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue());
|
||||
tx.DeleteVertex(vertex1);
|
||||
});
|
||||
});
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(AllGlobalOperations, {
|
||||
OPERATION(LABEL_INDEX_CREATE, "hello");
|
||||
OPERATION(LABEL_INDEX_DROP, "hello");
|
||||
OPERATION(LABEL_PROPERTY_INDEX_CREATE, "hello", {"world"});
|
||||
OPERATION(LABEL_PROPERTY_INDEX_DROP, "hello", {"world"});
|
||||
OPERATION(EXISTENCE_CONSTRAINT_CREATE, "hello", {"world"});
|
||||
OPERATION(EXISTENCE_CONSTRAINT_DROP, "hello", {"world"});
|
||||
OPERATION(UNIQUE_CONSTRAINT_CREATE, "hello", {"world", "and", "universe"});
|
||||
OPERATION(UNIQUE_CONSTRAINT_DROP, "hello", {"world", "and", "universe"});
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
GENERATE_SIMPLE_TEST(InvalidTransactionOrdering, {
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
gen.ResetTransactionIds();
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
});
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_P(WalFileTest, InvalidMarker) {
|
||||
memgraph::storage::durability::WalInfo info;
|
||||
|
||||
{
|
||||
DeltaGenerator gen(storage_directory, GetParam(), 5);
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
info = gen.GetInfo();
|
||||
}
|
||||
|
||||
auto wal_files = GetFilesList();
|
||||
ASSERT_EQ(wal_files.size(), 1);
|
||||
const auto &wal_file = wal_files.front();
|
||||
|
||||
auto final_info = memgraph::storage::durability::ReadWalInfo(wal_file);
|
||||
AssertWalInfoEqual(info, final_info);
|
||||
|
||||
size_t i = 0;
|
||||
for (auto marker : memgraph::storage::durability::kMarkersAll) {
|
||||
if (marker == memgraph::storage::durability::Marker::SECTION_DELTA) continue;
|
||||
auto current_file = storage_directory / fmt::format("temporary_{}", i);
|
||||
ASSERT_TRUE(std::filesystem::copy_file(wal_file, current_file));
|
||||
memgraph::utils::OutputFile file;
|
||||
file.Open(current_file, memgraph::utils::OutputFile::Mode::OVERWRITE_EXISTING);
|
||||
file.SetPosition(memgraph::utils::OutputFile::Position::SET, final_info.offset_deltas);
|
||||
auto value = static_cast<uint8_t>(marker);
|
||||
file.Write(&value, sizeof(value));
|
||||
file.Sync();
|
||||
file.Close();
|
||||
ASSERT_THROW(memgraph::storage::durability::ReadWalInfo(current_file),
|
||||
memgraph::storage::durability::RecoveryFailure);
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
// NOLINTNEXTLINE(hicpp-special-member-functions)
|
||||
TEST_P(WalFileTest, PartialData) {
|
||||
std::vector<std::pair<uint64_t, memgraph::storage::durability::WalInfo>> infos;
|
||||
|
||||
{
|
||||
DeltaGenerator gen(storage_directory, GetParam(), 5);
|
||||
TRANSACTION(true, { tx.CreateVertex(); });
|
||||
infos.emplace_back(gen.GetPosition(), gen.GetInfo());
|
||||
TRANSACTION(true, {
|
||||
auto vertex = tx.CreateVertex();
|
||||
tx.AddLabel(vertex, "hello");
|
||||
});
|
||||
infos.emplace_back(gen.GetPosition(), gen.GetInfo());
|
||||
OPERATION(LABEL_PROPERTY_INDEX_CREATE, "hello", {"world"});
|
||||
infos.emplace_back(gen.GetPosition(), gen.GetInfo());
|
||||
TRANSACTION(true, {
|
||||
auto vertex1 = tx.CreateVertex();
|
||||
auto vertex2 = tx.CreateVertex();
|
||||
tx.AddLabel(vertex1, "test");
|
||||
tx.AddLabel(vertex2, "hello");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue("nandare"));
|
||||
tx.RemoveLabel(vertex1, "test");
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue(123));
|
||||
tx.SetProperty(vertex2, "hello", memgraph::storage::PropertyValue());
|
||||
tx.DeleteVertex(vertex1);
|
||||
});
|
||||
infos.emplace_back(gen.GetPosition(), gen.GetInfo());
|
||||
}
|
||||
|
||||
auto wal_files = GetFilesList();
|
||||
ASSERT_EQ(wal_files.size(), 1);
|
||||
const auto &wal_file = wal_files.front();
|
||||
|
||||
AssertWalInfoEqual(infos.back().second, memgraph::storage::durability::ReadWalInfo(wal_file));
|
||||
|
||||
auto current_file = storage_directory / "temporary";
|
||||
memgraph::utils::InputFile infile;
|
||||
infile.Open(wal_file);
|
||||
|
||||
uint64_t pos = 0;
|
||||
for (size_t i = 0; i < infile.GetSize(); ++i) {
|
||||
if (i < infos.front().first) {
|
||||
ASSERT_THROW(memgraph::storage::durability::ReadWalInfo(current_file),
|
||||
memgraph::storage::durability::RecoveryFailure);
|
||||
} else {
|
||||
if (i >= infos[pos + 1].first) ++pos;
|
||||
AssertWalInfoEqual(infos[pos].second, memgraph::storage::durability::ReadWalInfo(current_file));
|
||||
}
|
||||
{
|
||||
memgraph::utils::OutputFile outfile;
|
||||
outfile.Open(current_file, memgraph::utils::OutputFile::Mode::APPEND_TO_EXISTING);
|
||||
uint8_t value;
|
||||
ASSERT_TRUE(infile.Read(&value, sizeof(value)));
|
||||
outfile.Write(&value, sizeof(value));
|
||||
outfile.Sync();
|
||||
outfile.Close();
|
||||
}
|
||||
}
|
||||
ASSERT_EQ(pos, infos.size() - 2);
|
||||
AssertWalInfoEqual(infos[infos.size() - 1].second, memgraph::storage::durability::ReadWalInfo(current_file));
|
||||
}
|
Loading…
Reference in New Issue
Block a user