Merge branch 'project-pineapples' into T1167-MG-create-scanbyprimarykey-operator

This commit is contained in:
gvolfing 2023-01-23 08:42:14 +01:00
commit 89d26c36c9
23 changed files with 473 additions and 72 deletions

View File

@ -171,7 +171,7 @@ benchmark_tag="v1.6.0"
repo_clone_try_double "${primary_urls[gbenchmark]}" "${secondary_urls[gbenchmark]}" "benchmark" "$benchmark_tag" true
# google test
googletest_tag="release-1.8.0"
googletest_tag="release-1.12.1"
repo_clone_try_double "${primary_urls[gtest]}" "${secondary_urls[gtest]}" "googletest" "$googletest_tag" true
# libbcrypt

View File

@ -734,7 +734,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
// Returns true if a result was pulled.
const auto pull_result = [&]() -> bool {
cursor_->PullMultiple(multi_frame_, ctx_);
return multi_frame_.HasValidFrame();
return !multi_frame_.HasInvalidFrame();
};
const auto stream_values = [&output_symbols, &stream](const Frame &frame) {

View File

@ -45,7 +45,11 @@ void MultiFrame::MakeAllFramesInvalid() noexcept {
}
bool MultiFrame::HasValidFrame() const noexcept {
return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); });
return std::any_of(frames_.begin(), frames_.end(), [](const auto &frame) { return frame.IsValid(); });
}
bool MultiFrame::HasInvalidFrame() const noexcept {
return std::any_of(frames_.rbegin(), frames_.rend(), [](const auto &frame) { return !frame.IsValid(); });
}
// NOLINTNEXTLINE (bugprone-exception-escape)

View File

@ -81,6 +81,7 @@ class MultiFrame {
void MakeAllFramesInvalid() noexcept;
bool HasValidFrame() const noexcept;
bool HasInvalidFrame() const noexcept;
inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); }
@ -168,7 +169,7 @@ class ValidFramesModifier {
Iterator &operator++() {
do {
ptr_++;
} while (*this != iterator_wrapper_->end() && ptr_->IsValid());
} while (*this != iterator_wrapper_->end() && !ptr_->IsValid());
return *this;
}

View File

@ -171,9 +171,8 @@ uint64_t ComputeProfilingKey(const T *obj) {
class DistributedCreateNodeCursor : public Cursor {
public:
using InputOperator = std::shared_ptr<memgraph::query::v2::plan::LogicalOperator>;
DistributedCreateNodeCursor(const InputOperator &op, utils::MemoryResource *mem,
std::vector<const NodeCreationInfo *> nodes_info)
: input_cursor_(op->MakeCursor(mem)), nodes_info_(std::move(nodes_info)) {}
DistributedCreateNodeCursor(const InputOperator &op, utils::MemoryResource *mem, const NodeCreationInfo &node_info)
: input_cursor_(op->MakeCursor(mem)), node_info_(node_info) {}
bool Pull(Frame &frame, ExecutionContext &context) override {
SCOPED_PROFILE_OP("CreateNode");
@ -190,33 +189,92 @@ class DistributedCreateNodeCursor : public Cursor {
return false;
}
void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
SCOPED_PROFILE_OP("CreateNodeMF");
input_cursor_->PullMultiple(multi_frame, context);
auto *request_router = context.request_router;
{
SCOPED_REQUEST_WAIT_PROFILE;
request_router->CreateVertices(NodeCreationInfoToRequests(context, multi_frame));
}
PlaceNodesOnTheMultiFrame(multi_frame, context);
}
void Shutdown() override { input_cursor_->Shutdown(); }
void Reset() override {}
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]};
const auto primary_label = msgs::Label{.id = node_info_.labels[0]};
msgs::Vertex v{.id = std::make_pair(primary_label, primary_keys_[0])};
frame[nodes_info_.front()->symbol] =
frame[node_info_.symbol] =
TypedValue(query::v2::accessors::VertexAccessor(std::move(v), src_vertex_props_[0], context.request_router));
}
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");
const auto primary_label = node_info_.labels[0];
// TODO(jbajic) Send also the properties that are not part of primary key
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.request_router->IsPrimaryKey(primary_label, key)) {
rqst.primary_key.push_back(TypedValueToValue(val));
pk.push_back(TypedValueToValue(val));
}
}
} else {
auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(node_info_.properties)).ValueMap();
for (const auto &[key, value] : property_map) {
auto key_str = std::string(key);
auto property_id = context.request_router->NameToProperty(key_str);
if (context.request_router->IsPrimaryKey(primary_label, property_id)) {
rqst.primary_key.push_back(TypedValueToValue(value));
pk.push_back(TypedValueToValue(value));
}
}
}
// 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;
}
void PlaceNodesOnTheMultiFrame(MultiFrame &multi_frame, ExecutionContext &context) {
auto multi_frame_modifier = multi_frame.GetValidFramesModifier();
size_t i = 0;
MG_ASSERT(std::distance(multi_frame_modifier.begin(), multi_frame_modifier.end()));
for (auto &frame : multi_frame_modifier) {
const auto primary_label = msgs::Label{.id = node_info_.labels[0]};
msgs::Vertex v{.id = std::make_pair(primary_label, primary_keys_[i])};
frame[node_info_.symbol] = TypedValue(
query::v2::accessors::VertexAccessor(std::move(v), src_vertex_props_[i++], context.request_router));
}
}
std::vector<msgs::NewVertex> NodeCreationInfoToRequests(ExecutionContext &context, MultiFrame &multi_frame) {
std::vector<msgs::NewVertex> requests;
auto multi_frame_modifier = multi_frame.GetValidFramesModifier();
for (auto &frame : multi_frame_modifier) {
msgs::PrimaryKey pk;
msgs::NewVertex rqst;
MG_ASSERT(!node_info->labels.empty(), "Cannot determine primary label");
const auto primary_label = node_info->labels[0];
// TODO(jbajic) Fix properties not send,
// suggestion: ignore distinction between properties and primary keys
// since schema validation is done on storage side
MG_ASSERT(!node_info_.labels.empty(), "Cannot determine primary label");
const auto primary_label = node_info_.labels[0];
MG_ASSERT(context.request_router->IsPrimaryLabel(primary_label), "First label has to be a primary label!");
// TODO(jbajic) Send also the properties that are not part of primary key
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)) {
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.request_router->IsPrimaryKey(primary_label, key)) {
@ -225,7 +283,7 @@ class DistributedCreateNodeCursor : public Cursor {
}
}
} else {
auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(node_info->properties)).ValueMap();
auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(node_info_.properties)).ValueMap();
for (const auto &[key, value] : property_map) {
auto key_str = std::string(key);
auto property_id = context.request_router->NameToProperty(key_str);
@ -236,21 +294,19 @@ class DistributedCreateNodeCursor : public Cursor {
}
}
if (node_info->labels.empty()) {
throw QueryRuntimeException("Primary label must be defined!");
}
// 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));
}
primary_keys_.push_back(std::move(pk));
return requests;
}
private:
const UniqueCursorPtr input_cursor_;
std::vector<const NodeCreationInfo *> nodes_info_;
NodeCreationInfo node_info_;
std::vector<std::vector<std::pair<storage::v3::PropertyId, msgs::Value>>> src_vertex_props_;
std::vector<msgs::PrimaryKey> primary_keys_;
};
@ -295,7 +351,7 @@ ACCEPT_WITH_INPUT(CreateNode)
UniqueCursorPtr CreateNode::MakeCursor(utils::MemoryResource *mem) const {
EventCounter::IncrementCounter(EventCounter::CreateNodeOperator);
return MakeUniqueCursorPtr<DistributedCreateNodeCursor>(mem, input_, mem, std::vector{&this->node_info_});
return MakeUniqueCursorPtr<DistributedCreateNodeCursor>(mem, input_, mem, this->node_info_);
}
std::vector<Symbol> CreateNode::ModifiedSymbols(const SymbolTable &table) const {
@ -2464,6 +2520,22 @@ class DistributedCreateExpandCursor : public Cursor {
return true;
}
void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
SCOPED_PROFILE_OP("CreateExpandMF");
input_cursor_->PullMultiple(multi_frame, context);
auto request_vertices = ExpandCreationInfoToRequests(multi_frame, context);
{
SCOPED_REQUEST_WAIT_PROFILE;
auto &request_router = context.request_router;
auto results = request_router->CreateExpand(std::move(request_vertices));
for (const auto &result : results) {
if (result.error) {
throw std::runtime_error("CreateExpand Request failed");
}
}
}
}
void Shutdown() override { input_cursor_->Shutdown(); }
void Reset() override {
@ -2538,6 +2610,63 @@ class DistributedCreateExpandCursor : public Cursor {
return edge_requests;
}
std::vector<msgs::NewExpand> ExpandCreationInfoToRequests(MultiFrame &multi_frame, ExecutionContext &context) const {
std::vector<msgs::NewExpand> edge_requests;
auto frames_modifier = multi_frame.GetValidFramesModifier();
for (auto &frame : frames_modifier) {
const auto &edge_info = self_.edge_info_;
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};
if (const auto *edge_info_properties = std::get_if<PropertiesMapList>(&edge_info.properties)) {
for (const auto &[property, value_expression] : *edge_info_properties) {
TypedValue val = value_expression->Accept(evaluator);
request.properties.emplace_back(property, storage::v3::TypedValueToValue(val));
}
} else {
// handle parameter
auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(edge_info.properties)).ValueMap();
for (const auto &[property, value] : property_map) {
const auto property_id = context.request_router->NameToProperty(std::string(property));
request.properties.emplace_back(property_id, storage::v3::TypedValueToValue(value));
}
}
TypedValue &v1_value = frame[self_.input_symbol_];
const auto &v1 = v1_value.ValueVertex();
const auto &v2 = OtherVertex(frame);
msgs::Edge edge{.src = request.src_vertex,
.dst = request.dest_vertex,
.properties = request.properties,
.id = request.id,
.type = request.type};
frame[self_.edge_info_.symbol] = TypedValue(accessors::EdgeAccessor(std::move(edge), context.request_router));
// Set src and dest vertices
// TODO(jbajic) Currently we are only handling scenario where vertices
// are matched
switch (edge_info.direction) {
case EdgeAtom::Direction::IN: {
request.src_vertex = v2.Id();
request.dest_vertex = v1.Id();
break;
}
case EdgeAtom::Direction::OUT: {
request.src_vertex = v1.Id();
request.dest_vertex = v2.Id();
break;
}
case EdgeAtom::Direction::BOTH:
LOG_FATAL("Must indicate exact expansion direction here");
}
edge_requests.push_back(std::move(request));
}
return edge_requests;
}
private:
void ResetExecutionState() {}

View File

@ -305,7 +305,8 @@ class RequestRouter : public RequestRouterInterface {
MG_ASSERT(!new_edges.empty());
// create requests
std::vector<ShardRequestState<msgs::CreateExpandRequest>> requests_to_be_sent = RequestsForCreateExpand(new_edges);
std::vector<ShardRequestState<msgs::CreateExpandRequest>> requests_to_be_sent =
RequestsForCreateExpand(std::move(new_edges));
// begin all requests in parallel
RunningRequests<msgs::CreateExpandRequest> running_requests = {};
@ -437,7 +438,7 @@ class RequestRouter : public RequestRouterInterface {
}
std::vector<ShardRequestState<msgs::CreateExpandRequest>> RequestsForCreateExpand(
const std::vector<msgs::NewExpand> &new_expands) {
std::vector<msgs::NewExpand> new_expands) {
std::map<ShardMetadata, msgs::CreateExpandRequest> per_shard_request_table;
auto ensure_shard_exists_in_table = [&per_shard_request_table,
transaction_id = transaction_id_](const ShardMetadata &shard) {

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -36,6 +36,7 @@ struct Value;
struct Label {
LabelId id;
friend bool operator==(const Label &lhs, const Label &rhs) { return lhs.id == rhs.id; }
friend bool operator==(const Label &lhs, const LabelId &rhs) { return lhs.id == rhs; }
};
// TODO(kostasrim) update this with CompoundKey, same for the rest of the file.

View File

@ -410,3 +410,10 @@ target_link_libraries(${test_prefix}high_density_shard_create_scan mg-io mg-coor
# Tests for awesome_memgraph_functions
add_unit_test(query_v2_expression_evaluator.cpp)
target_link_libraries(${test_prefix}query_v2_expression_evaluator mg-query-v2)
# Tests for multiframes
add_unit_test(query_v2_create_expand_multiframe.cpp)
target_link_libraries(${test_prefix}query_v2_create_expand_multiframe mg-query-v2)
add_unit_test(query_v2_create_node_multiframe.cpp)
target_link_libraries(${test_prefix}query_v2_create_node_multiframe mg-query-v2)

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -88,14 +88,14 @@ TEST_P(SingleNodeBfsTest, All) {
std::unique_ptr<SingleNodeDb> SingleNodeBfsTest::db_{nullptr};
INSTANTIATE_TEST_CASE_P(DirectionAndExpansionDepth, SingleNodeBfsTest,
testing::Combine(testing::Range(-1, kVertexCount), testing::Range(-1, kVertexCount),
testing::Values(EdgeAtom::Direction::OUT, EdgeAtom::Direction::IN,
EdgeAtom::Direction::BOTH),
testing::Values(std::vector<std::string>{}), testing::Bool(),
testing::Values(FilterLambdaType::NONE)));
INSTANTIATE_TEST_SUITE_P(DirectionAndExpansionDepth, SingleNodeBfsTest,
testing::Combine(testing::Range(-1, kVertexCount), testing::Range(-1, kVertexCount),
testing::Values(EdgeAtom::Direction::OUT, EdgeAtom::Direction::IN,
EdgeAtom::Direction::BOTH),
testing::Values(std::vector<std::string>{}), testing::Bool(),
testing::Values(FilterLambdaType::NONE)));
INSTANTIATE_TEST_CASE_P(
INSTANTIATE_TEST_SUITE_P(
EdgeType, SingleNodeBfsTest,
testing::Combine(testing::Values(-1), testing::Values(-1),
testing::Values(EdgeAtom::Direction::OUT, EdgeAtom::Direction::IN, EdgeAtom::Direction::BOTH),
@ -103,11 +103,11 @@ INSTANTIATE_TEST_CASE_P(
std::vector<std::string>{"b"}, std::vector<std::string>{"a", "b"}),
testing::Bool(), testing::Values(FilterLambdaType::NONE)));
INSTANTIATE_TEST_CASE_P(FilterLambda, SingleNodeBfsTest,
testing::Combine(testing::Values(-1), testing::Values(-1),
testing::Values(EdgeAtom::Direction::OUT, EdgeAtom::Direction::IN,
EdgeAtom::Direction::BOTH),
testing::Values(std::vector<std::string>{}), testing::Bool(),
testing::Values(FilterLambdaType::NONE, FilterLambdaType::USE_FRAME,
FilterLambdaType::USE_FRAME_NULL, FilterLambdaType::USE_CTX,
FilterLambdaType::ERROR)));
INSTANTIATE_TEST_SUITE_P(FilterLambda, SingleNodeBfsTest,
testing::Combine(testing::Values(-1), testing::Values(-1),
testing::Values(EdgeAtom::Direction::OUT, EdgeAtom::Direction::IN,
EdgeAtom::Direction::BOTH),
testing::Values(std::vector<std::string>{}), testing::Bool(),
testing::Values(FilterLambdaType::NONE, FilterLambdaType::USE_FRAME,
FilterLambdaType::USE_FRAME_NULL, FilterLambdaType::USE_CTX,
FilterLambdaType::ERROR)));

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -294,7 +294,7 @@ std::shared_ptr<Base> gAstGeneratorTypes[] = {
std::make_shared<CachedAstGenerator>(),
};
INSTANTIATE_TEST_CASE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::ValuesIn(gAstGeneratorTypes));
INSTANTIATE_TEST_SUITE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::ValuesIn(gAstGeneratorTypes));
// NOTE: The above used to use *Typed Tests* functionality of gtest library.
// Unfortunately, the compilation time of this test increased to full 2 minutes!
@ -308,7 +308,7 @@ INSTANTIATE_TEST_CASE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::Val
// ClonedAstGenerator, CachedAstGenerator>
// AstGeneratorTypes;
//
// TYPED_TEST_CASE(CypherMainVisitorTest, AstGeneratorTypes);
// TYPED_TEST_SUITE(CypherMainVisitorTest, AstGeneratorTypes);
TEST_P(CypherMainVisitorTest, SyntaxException) {
auto &ast_generator = *GetParam();

View File

@ -0,0 +1,83 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <gmock/gmock.h>
#include <gtest/gtest.h>
#include "query/v2/common.hpp"
#include "query/v2/context.hpp"
#include "query/v2/plan/operator.hpp"
#include "query/v2/request_router.hpp"
namespace memgraph::query::v2::tests {
class MockedRequestRouter : public RequestRouterInterface {
public:
MOCK_METHOD(std::vector<VertexAccessor>, ScanVertices, (std::optional<std::string> label));
MOCK_METHOD(std::vector<msgs::CreateVerticesResponse>, CreateVertices, (std::vector<msgs::NewVertex>));
MOCK_METHOD(std::vector<msgs::ExpandOneResultRow>, ExpandOne, (msgs::ExpandOneRequest));
MOCK_METHOD(std::vector<msgs::CreateExpandResponse>, CreateExpand, (std::vector<msgs::NewExpand>));
MOCK_METHOD(std::vector<msgs::GetPropertiesResultRow>, GetProperties, (msgs::GetPropertiesRequest));
MOCK_METHOD(void, StartTransaction, ());
MOCK_METHOD(void, Commit, ());
MOCK_METHOD(storage::v3::EdgeTypeId, NameToEdgeType, (const std::string &), (const));
MOCK_METHOD(storage::v3::PropertyId, NameToProperty, (const std::string &), (const));
MOCK_METHOD(storage::v3::LabelId, NameToLabel, (const std::string &), (const));
MOCK_METHOD(storage::v3::LabelId, LabelToName, (const std::string &), (const));
MOCK_METHOD(const std::string &, PropertyToName, (storage::v3::PropertyId), (const));
MOCK_METHOD(const std::string &, LabelToName, (storage::v3::LabelId label), (const));
MOCK_METHOD(const std::string &, EdgeTypeToName, (storage::v3::EdgeTypeId type), (const));
MOCK_METHOD(std::optional<storage::v3::PropertyId>, MaybeNameToProperty, (const std::string &), (const));
MOCK_METHOD(std::optional<storage::v3::EdgeTypeId>, MaybeNameToEdgeType, (const std::string &), (const));
MOCK_METHOD(std::optional<storage::v3::LabelId>, MaybeNameToLabel, (const std::string &), (const));
MOCK_METHOD(bool, IsPrimaryLabel, (storage::v3::LabelId), (const));
MOCK_METHOD(bool, IsPrimaryKey, (storage::v3::LabelId, storage::v3::PropertyId), (const));
};
class MockedLogicalOperator : public plan::LogicalOperator {
public:
MOCK_METHOD(plan::UniqueCursorPtr, MakeCursor, (utils::MemoryResource *), (const));
MOCK_METHOD(std::vector<expr::Symbol>, ModifiedSymbols, (const expr::SymbolTable &), (const));
MOCK_METHOD(bool, HasSingleInput, (), (const));
MOCK_METHOD(std::shared_ptr<LogicalOperator>, input, (), (const));
MOCK_METHOD(void, set_input, (std::shared_ptr<LogicalOperator>));
MOCK_METHOD(std::unique_ptr<LogicalOperator>, Clone, (AstStorage * storage), (const));
MOCK_METHOD(bool, Accept, (plan::HierarchicalLogicalOperatorVisitor & visitor));
};
class MockedCursor : public plan::Cursor {
public:
MOCK_METHOD(bool, Pull, (Frame &, expr::ExecutionContext &));
MOCK_METHOD(void, PullMultiple, (MultiFrame &, expr::ExecutionContext &));
MOCK_METHOD(void, Reset, ());
MOCK_METHOD(void, Shutdown, ());
};
inline expr::ExecutionContext MakeContext(const expr::AstStorage &storage, const expr::SymbolTable &symbol_table,
RequestRouterInterface *router, IdAllocator *id_alloc) {
expr::ExecutionContext context;
context.symbol_table = symbol_table;
context.evaluation_context.properties = NamesToProperties(storage.properties_, router);
context.evaluation_context.labels = NamesToLabels(storage.labels_, router);
context.edge_ids_alloc = id_alloc;
context.request_router = router;
return context;
}
inline MockedLogicalOperator &BaseToMock(plan::LogicalOperator &op) {
return dynamic_cast<MockedLogicalOperator &>(op);
}
inline MockedCursor &BaseToMock(plan::Cursor &cursor) { return dynamic_cast<MockedCursor &>(cursor); }
} // namespace memgraph::query::v2::tests

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -67,7 +67,7 @@ TEST_P(ExpressiontoStringTest, Example) {
EXPECT_EQ(rewritten_expression, rewritten_expression2);
}
INSTANTIATE_TEST_CASE_P(
INSTANTIATE_TEST_SUITE_P(
PARAMETER, ExpressiontoStringTest,
::testing::Values(
std::make_pair(std::string("2 / 1"), std::string("(2 / 1)")),

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -90,7 +90,7 @@ void DeleteListContent(std::list<BaseOpChecker *> *list) {
delete ptr;
}
}
TYPED_TEST_CASE(TestPlanner, PlannerTypes);
TYPED_TEST_SUITE(TestPlanner, PlannerTypes);
TYPED_TEST(TestPlanner, MatchNodeReturn) {
// Test MATCH (n) RETURN n

View File

@ -0,0 +1,93 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <memory>
#include "mock_helpers.hpp"
#include "query/v2/bindings/frame.hpp"
#include "query/v2/bindings/symbol_table.hpp"
#include "query/v2/common.hpp"
#include "query/v2/context.hpp"
#include "query/v2/plan/operator.hpp"
#include "query/v2/requests.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/shard.hpp"
#include "utils/logging.hpp"
#include "utils/memory.hpp"
namespace memgraph::query::v2::tests {
MultiFrame CreateMultiFrame(const size_t max_pos, const Symbol &src, const Symbol &dst, MockedRequestRouter *router) {
static constexpr size_t number_of_frames = 100;
MultiFrame multi_frame(max_pos, number_of_frames, utils::NewDeleteResource());
auto frames_populator = multi_frame.GetInvalidFramesPopulator();
size_t i = 0;
for (auto &frame : frames_populator) {
auto &src_acc = frame.at(src);
auto &dst_acc = frame.at(dst);
auto v1 = msgs::Vertex{.id = {{msgs::LabelId::FromUint(1)}, {msgs::Value(static_cast<int64_t>(i++))}}};
auto v2 = msgs::Vertex{.id = {{msgs::LabelId::FromUint(1)}, {msgs::Value(static_cast<int64_t>(i++))}}};
std::map<msgs::PropertyId, msgs::Value> mp;
src_acc = TypedValue(query::v2::accessors::VertexAccessor(v1, mp, router));
dst_acc = TypedValue(query::v2::accessors::VertexAccessor(v2, mp, router));
}
multi_frame.MakeAllFramesInvalid();
return multi_frame;
}
TEST(CreateExpandTest, Cursor) {
using testing::_;
using testing::Return;
AstStorage ast;
SymbolTable symbol_table;
plan::NodeCreationInfo node;
plan::EdgeCreationInfo edge;
edge.edge_type = msgs::EdgeTypeId::FromUint(1);
edge.direction = EdgeAtom::Direction::IN;
edge.symbol = symbol_table.CreateSymbol("e", true);
auto id_alloc = IdAllocator(0, 100);
const auto &src = symbol_table.CreateSymbol("n", true);
node.symbol = symbol_table.CreateSymbol("u", true);
auto once_op = std::make_shared<plan::Once>();
auto create_expand = plan::CreateExpand(node, edge, once_op, src, true);
auto cursor = create_expand.MakeCursor(utils::NewDeleteResource());
MockedRequestRouter router;
EXPECT_CALL(router, CreateExpand(_))
.Times(1)
.WillOnce(Return(std::vector<msgs::CreateExpandResponse>{msgs::CreateExpandResponse{}}));
auto context = MakeContext(ast, symbol_table, &router, &id_alloc);
auto multi_frame = CreateMultiFrame(context.symbol_table.max_position(), src, node.symbol, &router);
cursor->PullMultiple(multi_frame, context);
auto frames = multi_frame.GetValidFramesReader();
auto number_of_valid_frames = 0;
for (auto &frame : frames) {
++number_of_valid_frames;
EXPECT_EQ(frame[edge.symbol].IsEdge(), true);
const auto &e = frame[edge.symbol].ValueEdge();
EXPECT_EQ(e.EdgeType(), edge.edge_type);
}
EXPECT_EQ(number_of_valid_frames, 1);
auto invalid_frames = multi_frame.GetInvalidFramesPopulator();
auto number_of_invalid_frames = std::distance(invalid_frames.begin(), invalid_frames.end());
EXPECT_EQ(number_of_invalid_frames, 99);
}
} // namespace memgraph::query::v2::tests

View File

@ -0,0 +1,82 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "gmock/gmock.h"
#include "mock_helpers.hpp"
#include "query/v2/bindings/frame.hpp"
#include "query/v2/bindings/symbol_table.hpp"
#include "query/v2/common.hpp"
#include "query/v2/context.hpp"
#include "query/v2/frontend/ast/ast.hpp"
#include "query/v2/plan/operator.hpp"
#include "query/v2/requests.hpp"
#include "storage/v3/property_value.hpp"
#include "storage/v3/shard.hpp"
#include "utils/memory.hpp"
namespace memgraph::query::v2::tests {
MultiFrame CreateMultiFrame(const size_t max_pos) {
static constexpr size_t frame_size = 100;
MultiFrame multi_frame(max_pos, frame_size, utils::NewDeleteResource());
return multi_frame;
}
TEST(CreateNodeTest, CreateNodeCursor) {
using testing::_;
using testing::IsEmpty;
using testing::Return;
AstStorage ast;
SymbolTable symbol_table;
plan::NodeCreationInfo node;
auto id_alloc = IdAllocator(0, 100);
node.symbol = symbol_table.CreateSymbol("n", true);
const auto primary_label_id = msgs::LabelId::FromUint(2);
node.labels.push_back(primary_label_id);
auto literal = PrimitiveLiteral();
literal.value_ = TypedValue(static_cast<int64_t>(200));
auto p = plan::PropertiesMapList{};
p.push_back(std::make_pair(msgs::PropertyId::FromUint(3), &literal));
node.properties.emplace<0>(std::move(p));
auto once_op = std::make_shared<plan::Once>();
auto create_expand = plan::CreateNode(once_op, node);
auto cursor = create_expand.MakeCursor(utils::NewDeleteResource());
MockedRequestRouter router;
EXPECT_CALL(router, CreateVertices(_)).Times(1).WillOnce(Return(std::vector<msgs::CreateVerticesResponse>{}));
EXPECT_CALL(router, IsPrimaryLabel(_)).WillRepeatedly(Return(true));
EXPECT_CALL(router, IsPrimaryKey(_, _)).WillRepeatedly(Return(true));
auto context = MakeContext(ast, symbol_table, &router, &id_alloc);
auto multi_frame = CreateMultiFrame(context.symbol_table.max_position());
cursor->PullMultiple(multi_frame, context);
auto frames = multi_frame.GetValidFramesReader();
auto number_of_valid_frames = 0;
for (auto &frame : frames) {
++number_of_valid_frames;
EXPECT_EQ(frame[node.symbol].IsVertex(), true);
const auto &n = frame[node.symbol].ValueVertex();
EXPECT_THAT(n.Labels(), IsEmpty());
EXPECT_EQ(n.PrimaryLabel(), primary_label_id);
// TODO(antaljanosbenjamin): Check primary key
}
EXPECT_EQ(number_of_valid_frames, 1);
auto invalid_frames = multi_frame.GetInvalidFramesPopulator();
auto number_of_invalid_frames = std::distance(invalid_frames.begin(), invalid_frames.end());
EXPECT_EQ(number_of_invalid_frames, 99);
}
} // namespace memgraph::query::v2::tests

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -299,7 +299,7 @@ std::shared_ptr<Base> gAstGeneratorTypes[] = {
std::make_shared<CachedAstGenerator>(),
};
INSTANTIATE_TEST_CASE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::ValuesIn(gAstGeneratorTypes));
INSTANTIATE_TEST_SUITE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::ValuesIn(gAstGeneratorTypes));
// NOTE: The above used to use *Typed Tests* functionality of gtest library.
// Unfortunately, the compilation time of this test increased to full 2 minutes!
@ -313,7 +313,7 @@ INSTANTIATE_TEST_CASE_P(AstGeneratorTypes, CypherMainVisitorTest, ::testing::Val
// ClonedAstGenerator, CachedAstGenerator>
// AstGeneratorTypes;
//
// TYPED_TEST_CASE(CypherMainVisitorTest, AstGeneratorTypes);
// TYPED_TEST_SUITE(CypherMainVisitorTest, AstGeneratorTypes);
TEST_P(CypherMainVisitorTest, SyntaxException) {
auto &ast_generator = *GetParam();

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -82,8 +82,8 @@ class StorageV3 : public ::testing::TestWithParam<bool> {
Config{.gc = {.reclamation_interval = reclamation_interval}}};
coordinator::Hlc last_hlc{0, io::Time{}};
};
INSTANTIATE_TEST_CASE_P(WithGc, StorageV3, ::testing::Values(true));
INSTANTIATE_TEST_CASE_P(WithoutGc, StorageV3, ::testing::Values(false));
INSTANTIATE_TEST_SUITE_P(WithGc, StorageV3, ::testing::Values(true));
INSTANTIATE_TEST_SUITE_P(WithoutGc, StorageV3, ::testing::Values(false));
// NOLINTNEXTLINE(hicpp-special-member-functions)
TEST_P(StorageV3, Commit) {

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -60,8 +60,8 @@ class StorageEdgeTest : public ::testing::TestWithParam<bool> {
coordinator::Hlc last_hlc{0, io::Time{}};
};
INSTANTIATE_TEST_CASE_P(EdgesWithProperties, StorageEdgeTest, ::testing::Values(true));
INSTANTIATE_TEST_CASE_P(EdgesWithoutProperties, StorageEdgeTest, ::testing::Values(false));
INSTANTIATE_TEST_SUITE_P(EdgesWithProperties, StorageEdgeTest, ::testing::Values(true));
INSTANTIATE_TEST_SUITE_P(EdgesWithoutProperties, StorageEdgeTest, ::testing::Values(false));
// NOLINTNEXTLINE(hicpp-special-member-functions)
TEST_P(StorageEdgeTest, EdgeCreateFromSmallerCommit) {

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -135,6 +135,6 @@ TEST_P(StorageIsolationLevelTest, Visibility) {
}
}
INSTANTIATE_TEST_CASE_P(ParameterizedStorageIsolationLevelTests, StorageIsolationLevelTest,
::testing::ValuesIn(isolation_levels), StorageIsolationLevelTest::PrintToStringParamName());
INSTANTIATE_TEST_SUITE_P(ParameterizedStorageIsolationLevelTests, StorageIsolationLevelTest,
::testing::ValuesIn(isolation_levels), StorageIsolationLevelTest::PrintToStringParamName());
} // namespace memgraph::storage::v3::tests

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -330,4 +330,4 @@ TEST_P(CsvReaderTest, EmptyColumns) {
}
}
INSTANTIATE_TEST_CASE_P(NewlineParameterizedTest, CsvReaderTest, ::testing::Values("\n", "\r\n"));
INSTANTIATE_TEST_SUITE_P(NewlineParameterizedTest, CsvReaderTest, ::testing::Values("\n", "\r\n"));

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -190,9 +190,9 @@ TEST_P(FileLockerParameterizedTest, RemovePath) {
std::filesystem::current_path(save_path);
}
INSTANTIATE_TEST_CASE_P(FileLockerPathVariantTests, FileLockerParameterizedTest,
::testing::Values(std::make_tuple(false, false), std::make_tuple(false, true),
std::make_tuple(true, false), std::make_tuple(true, true)));
INSTANTIATE_TEST_SUITE_P(FileLockerPathVariantTests, FileLockerParameterizedTest,
::testing::Values(std::make_tuple(false, false), std::make_tuple(false, true),
std::make_tuple(true, false), std::make_tuple(true, true)));
TEST_F(FileLockerTest, MultipleLockers) {
CreateFiles(3);

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -393,7 +393,7 @@ class AllocatorTest : public ::testing::Test {};
using ContainersWithAllocators = ::testing::Types<ContainerWithAllocatorLast, ContainerWithAllocatorFirst>;
TYPED_TEST_CASE(AllocatorTest, ContainersWithAllocators);
TYPED_TEST_SUITE(AllocatorTest, ContainersWithAllocators);
TYPED_TEST(AllocatorTest, PropagatesToStdUsesAllocator) {
std::vector<TypeParam, memgraph::utils::Allocator<TypeParam>> vec(memgraph::utils::NewDeleteResource());

View File

@ -11,7 +11,7 @@
#include <filesystem>
#include <gmock/gmock-generated-matchers.h>
#include <gmock/gmock-matchers.h>
#include <gtest/gtest.h>
#include "utils/settings.hpp"