9ad7d82a54
Summary: Union query combinator implementation consists of: * adjustments to the AST and `cypher_main_visitor` * enabling `QueryStripper` to parse multiple `return` statements (not stopping after first) * symbol generation for union results * union logical operator * query plan generator adjustments Reviewers: teon.banek, mislav.bradac Reviewed By: teon.banek Subscribers: pullbot, buda Differential Revision: https://phabricator.memgraph.io/D1038
1679 lines
65 KiB
C++
1679 lines
65 KiB
C++
#include <list>
|
|
#include <tuple>
|
|
#include <unordered_set>
|
|
|
|
#include "gmock/gmock.h"
|
|
#include "gtest/gtest.h"
|
|
|
|
#include "query/frontend/ast/ast.hpp"
|
|
#include "query/frontend/semantic/symbol_generator.hpp"
|
|
#include "query/frontend/semantic/symbol_table.hpp"
|
|
#include "query/plan/operator.hpp"
|
|
#include "query/plan/planner.hpp"
|
|
|
|
#include "query_common.hpp"
|
|
|
|
using namespace query::plan;
|
|
using query::AstTreeStorage;
|
|
using query::SingleQuery;
|
|
using query::Symbol;
|
|
using query::SymbolGenerator;
|
|
using query::SymbolTable;
|
|
using Direction = query::EdgeAtom::Direction;
|
|
using Bound = ScanAllByLabelPropertyRange::Bound;
|
|
|
|
namespace {
|
|
|
|
class BaseOpChecker {
|
|
public:
|
|
virtual ~BaseOpChecker() {}
|
|
|
|
virtual void CheckOp(LogicalOperator &, const SymbolTable &) = 0;
|
|
};
|
|
|
|
class PlanChecker : public HierarchicalLogicalOperatorVisitor {
|
|
public:
|
|
using HierarchicalLogicalOperatorVisitor::PostVisit;
|
|
using HierarchicalLogicalOperatorVisitor::PreVisit;
|
|
using HierarchicalLogicalOperatorVisitor::Visit;
|
|
|
|
PlanChecker(const std::list<BaseOpChecker *> &checkers,
|
|
const SymbolTable &symbol_table)
|
|
: checkers_(checkers), symbol_table_(symbol_table) {}
|
|
|
|
#define PRE_VISIT(TOp) \
|
|
bool PreVisit(TOp &op) override { \
|
|
CheckOp(op); \
|
|
return true; \
|
|
}
|
|
|
|
PRE_VISIT(CreateNode);
|
|
PRE_VISIT(CreateExpand);
|
|
PRE_VISIT(Delete);
|
|
PRE_VISIT(ScanAll);
|
|
PRE_VISIT(ScanAllByLabel);
|
|
PRE_VISIT(ScanAllByLabelPropertyValue);
|
|
PRE_VISIT(ScanAllByLabelPropertyRange);
|
|
PRE_VISIT(Expand);
|
|
PRE_VISIT(ExpandVariable);
|
|
PRE_VISIT(Filter);
|
|
PRE_VISIT(ConstructNamedPath);
|
|
PRE_VISIT(Produce);
|
|
PRE_VISIT(SetProperty);
|
|
PRE_VISIT(SetProperties);
|
|
PRE_VISIT(SetLabels);
|
|
PRE_VISIT(RemoveProperty);
|
|
PRE_VISIT(RemoveLabels);
|
|
PRE_VISIT(ExpandUniquenessFilter<VertexAccessor>);
|
|
PRE_VISIT(ExpandUniquenessFilter<EdgeAccessor>);
|
|
PRE_VISIT(Accumulate);
|
|
PRE_VISIT(Aggregate);
|
|
PRE_VISIT(Skip);
|
|
PRE_VISIT(Limit);
|
|
PRE_VISIT(OrderBy);
|
|
bool PreVisit(Merge &op) override {
|
|
CheckOp(op);
|
|
op.input()->Accept(*this);
|
|
return false;
|
|
}
|
|
bool PreVisit(Optional &op) override {
|
|
CheckOp(op);
|
|
op.input()->Accept(*this);
|
|
return false;
|
|
}
|
|
PRE_VISIT(Unwind);
|
|
PRE_VISIT(Distinct);
|
|
|
|
bool Visit(Once &) override {
|
|
// Ignore checking Once, it is implicitly at the end.
|
|
return true;
|
|
}
|
|
|
|
bool Visit(CreateIndex &op) override {
|
|
CheckOp(op);
|
|
return true;
|
|
}
|
|
#undef PRE_VISIT
|
|
|
|
std::list<BaseOpChecker *> checkers_;
|
|
|
|
private:
|
|
void CheckOp(LogicalOperator &op) {
|
|
ASSERT_FALSE(checkers_.empty());
|
|
checkers_.back()->CheckOp(op, symbol_table_);
|
|
checkers_.pop_back();
|
|
}
|
|
|
|
const SymbolTable &symbol_table_;
|
|
};
|
|
|
|
template <class TOp>
|
|
class OpChecker : public BaseOpChecker {
|
|
public:
|
|
void CheckOp(LogicalOperator &op, const SymbolTable &symbol_table) override {
|
|
auto *expected_op = dynamic_cast<TOp *>(&op);
|
|
ASSERT_TRUE(expected_op);
|
|
ExpectOp(*expected_op, symbol_table);
|
|
}
|
|
|
|
virtual void ExpectOp(TOp &, const SymbolTable &) {}
|
|
};
|
|
|
|
using ExpectCreateNode = OpChecker<CreateNode>;
|
|
using ExpectCreateExpand = OpChecker<CreateExpand>;
|
|
using ExpectDelete = OpChecker<Delete>;
|
|
using ExpectScanAll = OpChecker<ScanAll>;
|
|
using ExpectScanAllByLabel = OpChecker<ScanAllByLabel>;
|
|
using ExpectExpand = OpChecker<Expand>;
|
|
using ExpectFilter = OpChecker<Filter>;
|
|
using ExpectConstructNamedPath = OpChecker<ConstructNamedPath>;
|
|
using ExpectProduce = OpChecker<Produce>;
|
|
using ExpectSetProperty = OpChecker<SetProperty>;
|
|
using ExpectSetProperties = OpChecker<SetProperties>;
|
|
using ExpectSetLabels = OpChecker<SetLabels>;
|
|
using ExpectRemoveProperty = OpChecker<RemoveProperty>;
|
|
using ExpectRemoveLabels = OpChecker<RemoveLabels>;
|
|
template <class TAccessor>
|
|
using ExpectExpandUniquenessFilter =
|
|
OpChecker<ExpandUniquenessFilter<TAccessor>>;
|
|
using ExpectSkip = OpChecker<Skip>;
|
|
using ExpectLimit = OpChecker<Limit>;
|
|
using ExpectOrderBy = OpChecker<OrderBy>;
|
|
using ExpectUnwind = OpChecker<Unwind>;
|
|
using ExpectDistinct = OpChecker<Distinct>;
|
|
|
|
class ExpectExpandVariable : public OpChecker<ExpandVariable> {
|
|
public:
|
|
void ExpectOp(ExpandVariable &op, const SymbolTable &) override {
|
|
EXPECT_EQ(op.type(), query::EdgeAtom::Type::DEPTH_FIRST);
|
|
}
|
|
};
|
|
|
|
class ExpectExpandBreadthFirst : public OpChecker<ExpandVariable> {
|
|
public:
|
|
void ExpectOp(ExpandVariable &op, const SymbolTable &) override {
|
|
EXPECT_EQ(op.type(), query::EdgeAtom::Type::BREADTH_FIRST);
|
|
}
|
|
};
|
|
|
|
class ExpectAccumulate : public OpChecker<Accumulate> {
|
|
public:
|
|
explicit ExpectAccumulate(const std::unordered_set<Symbol> &symbols)
|
|
: symbols_(symbols) {}
|
|
|
|
void ExpectOp(Accumulate &op, const SymbolTable &) override {
|
|
std::unordered_set<Symbol> got_symbols(op.symbols().begin(),
|
|
op.symbols().end());
|
|
EXPECT_EQ(symbols_, got_symbols);
|
|
}
|
|
|
|
private:
|
|
const std::unordered_set<Symbol> symbols_;
|
|
};
|
|
|
|
class ExpectAggregate : public OpChecker<Aggregate> {
|
|
public:
|
|
ExpectAggregate(const std::vector<query::Aggregation *> &aggregations,
|
|
const std::unordered_set<query::Expression *> &group_by)
|
|
: aggregations_(aggregations), group_by_(group_by) {}
|
|
|
|
void ExpectOp(Aggregate &op, const SymbolTable &symbol_table) override {
|
|
auto aggr_it = aggregations_.begin();
|
|
for (const auto &aggr_elem : op.aggregations()) {
|
|
ASSERT_NE(aggr_it, aggregations_.end());
|
|
auto aggr = *aggr_it++;
|
|
EXPECT_EQ(aggr_elem.value, aggr->expression1_);
|
|
EXPECT_EQ(aggr_elem.key, aggr->expression2_);
|
|
EXPECT_EQ(aggr_elem.op, aggr->op_);
|
|
EXPECT_EQ(aggr_elem.output_sym, symbol_table.at(*aggr));
|
|
}
|
|
EXPECT_EQ(aggr_it, aggregations_.end());
|
|
auto got_group_by = std::unordered_set<query::Expression *>(
|
|
op.group_by().begin(), op.group_by().end());
|
|
EXPECT_EQ(group_by_, got_group_by);
|
|
}
|
|
|
|
private:
|
|
const std::vector<query::Aggregation *> aggregations_;
|
|
const std::unordered_set<query::Expression *> group_by_;
|
|
};
|
|
|
|
class ExpectMerge : public OpChecker<Merge> {
|
|
public:
|
|
ExpectMerge(const std::list<BaseOpChecker *> &on_match,
|
|
const std::list<BaseOpChecker *> &on_create)
|
|
: on_match_(on_match), on_create_(on_create) {}
|
|
|
|
void ExpectOp(Merge &merge, const SymbolTable &symbol_table) override {
|
|
PlanChecker check_match(on_match_, symbol_table);
|
|
merge.merge_match()->Accept(check_match);
|
|
PlanChecker check_create(on_create_, symbol_table);
|
|
merge.merge_create()->Accept(check_create);
|
|
}
|
|
|
|
private:
|
|
const std::list<BaseOpChecker *> &on_match_;
|
|
const std::list<BaseOpChecker *> &on_create_;
|
|
};
|
|
|
|
class ExpectOptional : public OpChecker<Optional> {
|
|
public:
|
|
explicit ExpectOptional(const std::list<BaseOpChecker *> &optional)
|
|
: optional_(optional) {}
|
|
|
|
ExpectOptional(const std::vector<Symbol> &optional_symbols,
|
|
const std::list<BaseOpChecker *> &optional)
|
|
: optional_symbols_(optional_symbols), optional_(optional) {}
|
|
|
|
void ExpectOp(Optional &optional, const SymbolTable &symbol_table) override {
|
|
if (!optional_symbols_.empty()) {
|
|
EXPECT_THAT(optional.optional_symbols(),
|
|
testing::UnorderedElementsAreArray(optional_symbols_));
|
|
}
|
|
PlanChecker check_optional(optional_, symbol_table);
|
|
optional.optional()->Accept(check_optional);
|
|
}
|
|
|
|
private:
|
|
std::vector<Symbol> optional_symbols_;
|
|
const std::list<BaseOpChecker *> &optional_;
|
|
};
|
|
|
|
class ExpectScanAllByLabelPropertyValue
|
|
: public OpChecker<ScanAllByLabelPropertyValue> {
|
|
public:
|
|
ExpectScanAllByLabelPropertyValue(
|
|
GraphDbTypes::Label label,
|
|
const std::pair<std::string, GraphDbTypes::Property> &prop_pair,
|
|
query::Expression *expression)
|
|
: label_(label), property_(prop_pair.second), expression_(expression) {}
|
|
|
|
void ExpectOp(ScanAllByLabelPropertyValue &scan_all,
|
|
const SymbolTable &) override {
|
|
EXPECT_EQ(scan_all.label(), label_);
|
|
EXPECT_EQ(scan_all.property(), property_);
|
|
EXPECT_EQ(scan_all.expression(), expression_);
|
|
}
|
|
|
|
private:
|
|
GraphDbTypes::Label label_;
|
|
GraphDbTypes::Property property_;
|
|
query::Expression *expression_;
|
|
};
|
|
|
|
class ExpectScanAllByLabelPropertyRange
|
|
: public OpChecker<ScanAllByLabelPropertyRange> {
|
|
public:
|
|
ExpectScanAllByLabelPropertyRange(
|
|
GraphDbTypes::Label label, GraphDbTypes::Property property,
|
|
std::experimental::optional<Bound> lower_bound,
|
|
std::experimental::optional<Bound> upper_bound)
|
|
: label_(label),
|
|
property_(property),
|
|
lower_bound_(lower_bound),
|
|
upper_bound_(upper_bound) {}
|
|
|
|
void ExpectOp(ScanAllByLabelPropertyRange &scan_all,
|
|
const SymbolTable &) override {
|
|
EXPECT_EQ(scan_all.label(), label_);
|
|
EXPECT_EQ(scan_all.property(), property_);
|
|
if (lower_bound_) {
|
|
ASSERT_TRUE(scan_all.lower_bound());
|
|
EXPECT_EQ(scan_all.lower_bound()->value(), lower_bound_->value());
|
|
EXPECT_EQ(scan_all.lower_bound()->type(), lower_bound_->type());
|
|
}
|
|
if (upper_bound_) {
|
|
ASSERT_TRUE(scan_all.upper_bound());
|
|
EXPECT_EQ(scan_all.upper_bound()->value(), upper_bound_->value());
|
|
EXPECT_EQ(scan_all.upper_bound()->type(), upper_bound_->type());
|
|
}
|
|
}
|
|
|
|
private:
|
|
GraphDbTypes::Label label_;
|
|
GraphDbTypes::Property property_;
|
|
std::experimental::optional<Bound> lower_bound_;
|
|
std::experimental::optional<Bound> upper_bound_;
|
|
};
|
|
|
|
class ExpectCreateIndex : public OpChecker<CreateIndex> {
|
|
public:
|
|
ExpectCreateIndex(GraphDbTypes::Label label, GraphDbTypes::Property property)
|
|
: label_(label), property_(property) {}
|
|
|
|
void ExpectOp(CreateIndex &create_index, const SymbolTable &) override {
|
|
EXPECT_EQ(create_index.label(), label_);
|
|
EXPECT_EQ(create_index.property(), property_);
|
|
}
|
|
|
|
private:
|
|
GraphDbTypes::Label label_;
|
|
GraphDbTypes::Property property_;
|
|
};
|
|
|
|
auto MakeSymbolTable(query::Query &query) {
|
|
SymbolTable symbol_table;
|
|
SymbolGenerator symbol_generator(symbol_table);
|
|
query.Accept(symbol_generator);
|
|
return symbol_table;
|
|
}
|
|
|
|
template <class... TChecker>
|
|
auto CheckPlan(LogicalOperator &plan, const SymbolTable &symbol_table,
|
|
TChecker... checker) {
|
|
std::list<BaseOpChecker *> checkers{&checker...};
|
|
PlanChecker plan_checker(checkers, symbol_table);
|
|
plan.Accept(plan_checker);
|
|
EXPECT_TRUE(plan_checker.checkers_.empty());
|
|
}
|
|
|
|
template <class... TChecker>
|
|
auto CheckPlan(AstTreeStorage &storage, TChecker... checker) {
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, checker...);
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchNodeReturn) {
|
|
// Test MATCH (n) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateNodeReturn) {
|
|
// Test CREATE (n) RETURN n AS n
|
|
AstTreeStorage storage;
|
|
auto ident_n = IDENT("n");
|
|
auto query =
|
|
QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"))), RETURN(ident_n, AS("n"))));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), acc, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateExpand) {
|
|
// Test CREATE (n) -[r :rel1]-> (m)
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("relationship");
|
|
QUERY(SINGLE_QUERY(CREATE(PATTERN(
|
|
NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
|
|
CheckPlan(storage, ExpectCreateNode(), ExpectCreateExpand());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateMultipleNode) {
|
|
// Test CREATE (n), (m)
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n")), PATTERN(NODE("m")))));
|
|
CheckPlan(storage, ExpectCreateNode(), ExpectCreateNode());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateNodeExpandNode) {
|
|
// Test CREATE (n) -[r :rel]-> (m), (l)
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("rel");
|
|
QUERY(SINGLE_QUERY(CREATE(
|
|
PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")),
|
|
PATTERN(NODE("l")))));
|
|
CheckPlan(storage, ExpectCreateNode(), ExpectCreateExpand(),
|
|
ExpectCreateNode());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateNamedPattern) {
|
|
// Test CREATE p = (n) -[r :rel]-> (m)
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("rel");
|
|
QUERY(SINGLE_QUERY(CREATE(NAMED_PATTERN(
|
|
"p", NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
|
|
CheckPlan(storage, ExpectCreateNode(), ExpectCreateExpand(),
|
|
ExpectConstructNamedPath());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchCreateExpand) {
|
|
// Test MATCH (n) CREATE (n) -[r :rel1]-> (m)
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("relationship");
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"))),
|
|
CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}),
|
|
NODE("m")))));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectCreateExpand());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchLabeledNodes) {
|
|
// Test MATCH (n :label) RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAllByLabel(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchPathReturn) {
|
|
// Test MATCH (n) -[r :relationship]- (m) RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("relationship");
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {relationship}),
|
|
NODE("m"))),
|
|
RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchNamedPatternReturn) {
|
|
// Test MATCH p = (n) -[r :relationship]- (m) RETURN p
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("relationship");
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(NAMED_PATTERN("p", NODE("n"),
|
|
EDGE("r", Direction::BOTH, {relationship}),
|
|
NODE("m"))),
|
|
RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(),
|
|
ExpectConstructNamedPath(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchNamedPatternWithPredicateReturn) {
|
|
// Test MATCH p = (n) -[r :relationship]- (m) RETURN p
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto relationship = dba.EdgeType("relationship");
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(NAMED_PATTERN("p", NODE("n"),
|
|
EDGE("r", Direction::BOTH, {relationship}),
|
|
NODE("m"))),
|
|
WHERE(EQ(LITERAL(2), IDENT("p"))), RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(),
|
|
ExpectConstructNamedPath(), ExpectFilter(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, OptionalMatchNamedPatternReturn) {
|
|
// Test OPTIONAL MATCH p = (n) -[r]- (m) RETURN p
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto node_n = NODE("n");
|
|
auto edge = EDGE("r");
|
|
auto node_m = NODE("m");
|
|
auto pattern = NAMED_PATTERN("p", node_n, edge, node_m);
|
|
QUERY(SINGLE_QUERY(OPTIONAL_MATCH(pattern), RETURN("p")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
|
|
std::list<BaseOpChecker *> optional{new ExpectScanAll(), new ExpectExpand(),
|
|
new ExpectConstructNamedPath()};
|
|
auto get_symbol = [&symbol_table](const auto *ast_node) {
|
|
return symbol_table.at(*ast_node->identifier_);
|
|
};
|
|
std::vector<Symbol> optional_symbols{get_symbol(pattern), get_symbol(node_n),
|
|
get_symbol(edge), get_symbol(node_m)};
|
|
CheckPlan(*plan, symbol_table, ExpectOptional(optional_symbols, optional),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWhereReturn) {
|
|
// Test MATCH (n) WHERE n.property < 42 RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto property = dba.Property("property");
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
WHERE(LESS(PROPERTY_LOOKUP("n", property), LITERAL(42))),
|
|
RETURN("n")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectFilter(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchDelete) {
|
|
// Test MATCH (n) DELETE n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), DELETE(IDENT("n"))));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectDelete());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchNodeSet) {
|
|
// Test MATCH (n) SET n.prop = 42, n = n, n :label
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
auto label = dba.Label("label");
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
SET(PROPERTY_LOOKUP("n", prop), LITERAL(42)),
|
|
SET("n", IDENT("n")), SET("n", {label})));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectSetProperty(),
|
|
ExpectSetProperties(), ExpectSetLabels());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchRemove) {
|
|
// Test MATCH (n) REMOVE n.prop REMOVE n :label
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
auto label = dba.Label("label");
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
REMOVE(PROPERTY_LOOKUP("n", prop)), REMOVE("n", {label})));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectRemoveProperty(),
|
|
ExpectRemoveLabels());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchMultiPattern) {
|
|
// Test MATCH (n) -[r]- (m), (j) -[e]- (i) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m")),
|
|
PATTERN(NODE("j"), EDGE("e"), NODE("i"))),
|
|
RETURN("n")));
|
|
// We expect the expansions after the first to have a uniqueness filter in a
|
|
// single MATCH clause.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectScanAll(),
|
|
ExpectExpand(), ExpectExpandUniquenessFilter<EdgeAccessor>(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchMultiPatternSameStart) {
|
|
// Test MATCH (n), (n) -[e]- (m) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n")), PATTERN(NODE("n"), EDGE("e"), NODE("m"))),
|
|
RETURN("n")));
|
|
// We expect the second pattern to generate only an Expand, since another
|
|
// ScanAll would be redundant.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchMultiPatternSameExpandStart) {
|
|
// Test MATCH (n) -[r]- (m), (m) -[e]- (l) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m")),
|
|
PATTERN(NODE("m"), EDGE("e"), NODE("l"))),
|
|
RETURN("n")));
|
|
// We expect the second pattern to generate only an Expand. Another
|
|
// ScanAll would be redundant, as it would generate the nodes obtained from
|
|
// expansion. Additionally, a uniqueness filter is expected.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectExpand(),
|
|
ExpectExpandUniquenessFilter<EdgeAccessor>(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MultiMatch) {
|
|
// Test MATCH (n) -[r]- (m) MATCH (j) -[e]- (i) -[f]- (h) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
MATCH(PATTERN(NODE("j"), EDGE("e"), NODE("i"), EDGE("f"), NODE("h"))),
|
|
RETURN("n")));
|
|
// Multiple MATCH clauses form a Cartesian product, so the uniqueness should
|
|
// not cross MATCH boundaries.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectScanAll(),
|
|
ExpectExpand(), ExpectExpand(),
|
|
ExpectExpandUniquenessFilter<EdgeAccessor>(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MultiMatchSameStart) {
|
|
// Test MATCH (n) MATCH (n) -[r]- (m) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
RETURN("n")));
|
|
// Similar to MatchMultiPatternSameStart, we expect only Expand from second
|
|
// MATCH clause.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWithReturn) {
|
|
// Test MATCH (old) WITH old AS new RETURN new
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("old"))), WITH("old", AS("new")),
|
|
RETURN("new")));
|
|
// No accumulation since we only do reads.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWithWhereReturn) {
|
|
// Test MATCH (old) WITH old AS new WHERE new.prop < 42 RETURN new
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("old"))), WITH("old", AS("new")),
|
|
WHERE(LESS(PROPERTY_LOOKUP("new", prop), LITERAL(42))),
|
|
RETURN("new")));
|
|
// No accumulation since we only do reads.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce(), ExpectFilter(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateMultiExpand) {
|
|
// Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto r = dba.EdgeType("r");
|
|
auto p = dba.EdgeType("p");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(
|
|
CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {r}), NODE("m")),
|
|
PATTERN(NODE("n"), EDGE("p", Direction::OUT, {p}), NODE("l")))));
|
|
CheckPlan(storage, ExpectCreateNode(), ExpectCreateExpand(),
|
|
ExpectCreateExpand());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWithSumWhereReturn) {
|
|
// Test MATCH (n) WITH SUM(n.prop) + 42 AS sum WHERE sum < 42
|
|
// RETURN sum AS result
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(PROPERTY_LOOKUP("n", prop));
|
|
auto literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"))), WITH(ADD(sum, literal), AS("sum")),
|
|
WHERE(LESS(IDENT("sum"), LITERAL(42))), RETURN("sum", AS("result"))));
|
|
auto aggr = ExpectAggregate({sum}, {literal});
|
|
CheckPlan(storage, ExpectScanAll(), aggr, ExpectProduce(), ExpectFilter(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchReturnSum) {
|
|
// Test MATCH (n) RETURN SUM(n.prop1) AS sum, n.prop2 AS group
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop1 = dba.Property("prop1");
|
|
auto prop2 = dba.Property("prop2");
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(PROPERTY_LOOKUP("n", prop1));
|
|
auto n_prop2 = PROPERTY_LOOKUP("n", prop2);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
RETURN(sum, AS("sum"), n_prop2, AS("group"))));
|
|
auto aggr = ExpectAggregate({sum}, {n_prop2});
|
|
CheckPlan(storage, ExpectScanAll(), aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateWithSum) {
|
|
// Test CREATE (n) WITH SUM(n.prop) AS sum
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto n_prop = PROPERTY_LOOKUP("n", prop);
|
|
auto sum = SUM(n_prop);
|
|
auto query =
|
|
QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"))), WITH(sum, AS("sum"))));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
|
|
auto aggr = ExpectAggregate({sum}, {});
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
// We expect both the accumulation and aggregation because the part before
|
|
// WITH updates the database.
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), acc, aggr,
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWithCreate) {
|
|
// Test MATCH (n) WITH n AS a CREATE (a) -[r :r]-> (b)
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto r_type = dba.EdgeType("r");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"))), WITH("n", AS("a")),
|
|
CREATE(
|
|
PATTERN(NODE("a"), EDGE("r", Direction::OUT, {r_type}), NODE("b")))));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce(), ExpectCreateExpand());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchReturnSkipLimit) {
|
|
// Test MATCH (n) RETURN n SKIP 2 LIMIT 1
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
RETURN("n", SKIP(LITERAL(2)), LIMIT(LITERAL(1)))));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce(), ExpectSkip(),
|
|
ExpectLimit());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateWithSkipReturnLimit) {
|
|
// Test CREATE (n) WITH n AS m SKIP 2 RETURN m LIMIT 1
|
|
AstTreeStorage storage;
|
|
auto ident_n = IDENT("n");
|
|
auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"))),
|
|
WITH(ident_n, AS("m"), SKIP(LITERAL(2))),
|
|
RETURN("m", LIMIT(LITERAL(1)))));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
// Since we have a write query, we need to have Accumulate. This is a bit
|
|
// different than Neo4j 3.0, which optimizes WITH followed by RETURN as a
|
|
// single RETURN clause and then moves Skip and Limit before Accumulate. This
|
|
// causes different behaviour. A newer version of Neo4j does the same thing as
|
|
// us here (but who knows if they change it again).
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), acc, ExpectProduce(),
|
|
ExpectSkip(), ExpectProduce(), ExpectLimit());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateReturnSumSkipLimit) {
|
|
// Test CREATE (n) RETURN SUM(n.prop) AS s SKIP 2 LIMIT 1
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto n_prop = PROPERTY_LOOKUP("n", prop);
|
|
auto sum = SUM(n_prop);
|
|
auto query = QUERY(
|
|
SINGLE_QUERY(CREATE(PATTERN(NODE("n"))),
|
|
RETURN(sum, AS("s"), SKIP(LITERAL(2)), LIMIT(LITERAL(1)))));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
|
|
auto aggr = ExpectAggregate({sum}, {});
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), acc, aggr, ExpectProduce(),
|
|
ExpectSkip(), ExpectLimit());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchReturnOrderBy) {
|
|
// Test MATCH (n) RETURN n ORDER BY n.prop
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto ret = RETURN("n", ORDER_BY(PROPERTY_LOOKUP("n", prop)));
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), ret));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectProduce(), ExpectOrderBy());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateWithOrderByWhere) {
|
|
// Test CREATE (n) -[r :r]-> (m)
|
|
// WITH n AS new ORDER BY new.prop, r.prop WHERE m.prop < 42
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
auto r_type = dba.EdgeType("r");
|
|
AstTreeStorage storage;
|
|
auto ident_n = IDENT("n");
|
|
auto new_prop = PROPERTY_LOOKUP("new", prop);
|
|
auto r_prop = PROPERTY_LOOKUP("r", prop);
|
|
auto m_prop = PROPERTY_LOOKUP("m", prop);
|
|
auto query = QUERY(SINGLE_QUERY(
|
|
CREATE(
|
|
PATTERN(NODE("n"), EDGE("r", Direction::OUT, {r_type}), NODE("m"))),
|
|
WITH(ident_n, AS("new"), ORDER_BY(new_prop, r_prop)),
|
|
WHERE(LESS(m_prop, LITERAL(42)))));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
// Since this is a write query, we expect to accumulate to old used symbols.
|
|
auto acc = ExpectAccumulate({
|
|
symbol_table.at(*ident_n), // `n` in WITH
|
|
symbol_table.at(*r_prop->expression_), // `r` in ORDER BY
|
|
symbol_table.at(*m_prop->expression_), // `m` in WHERE
|
|
});
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), ExpectCreateExpand(), acc,
|
|
ExpectProduce(), ExpectOrderBy(), ExpectFilter());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ReturnAddSumCountOrderBy) {
|
|
// Test RETURN SUM(1) + COUNT(2) AS result ORDER BY result
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(1));
|
|
auto count = COUNT(LITERAL(2));
|
|
QUERY(SINGLE_QUERY(
|
|
RETURN(ADD(sum, count), AS("result"), ORDER_BY(IDENT("result")))));
|
|
auto aggr = ExpectAggregate({sum, count}, {});
|
|
CheckPlan(storage, aggr, ExpectProduce(), ExpectOrderBy());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchMerge) {
|
|
// Test MATCH (n) MERGE (n) -[r :r]- (m)
|
|
// ON MATCH SET n.prop = 42 ON CREATE SET m = n
|
|
// RETURN n AS n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto r_type = dba.EdgeType("r");
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto ident_n = IDENT("n");
|
|
auto query = QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"))),
|
|
MERGE(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {r_type}), NODE("m")),
|
|
ON_MATCH(SET(PROPERTY_LOOKUP("n", prop), LITERAL(42))),
|
|
ON_CREATE(SET("m", IDENT("n")))),
|
|
RETURN(ident_n, AS("n"))));
|
|
std::list<BaseOpChecker *> on_match{new ExpectExpand(),
|
|
new ExpectSetProperty()};
|
|
std::list<BaseOpChecker *> on_create{new ExpectCreateExpand(),
|
|
new ExpectSetProperties()};
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
// We expect Accumulate after Merge, because it is considered as a write.
|
|
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectScanAll(),
|
|
ExpectMerge(on_match, on_create), acc, ExpectProduce());
|
|
for (auto &op : on_match) delete op;
|
|
on_match.clear();
|
|
for (auto &op : on_create) delete op;
|
|
on_create.clear();
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchOptionalMatchWhereReturn) {
|
|
// Test MATCH (n) OPTIONAL MATCH (n) -[r]- (m) WHERE m.prop < 42 RETURN r
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
OPTIONAL_MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
WHERE(LESS(PROPERTY_LOOKUP("m", prop), LITERAL(42))),
|
|
RETURN("r")));
|
|
std::list<BaseOpChecker *> optional{new ExpectScanAll(), new ExpectExpand(),
|
|
new ExpectFilter()};
|
|
CheckPlan(storage, ExpectScanAll(), ExpectOptional(optional),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchUnwindReturn) {
|
|
// Test MATCH (n) UNWIND [1,2,3] AS x RETURN n, x
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
|
|
UNWIND(LIST(LITERAL(1), LITERAL(2), LITERAL(3)), AS("x")),
|
|
RETURN("n", "x")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectUnwind(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ReturnDistinctOrderBySkipLimit) {
|
|
// Test RETURN DISTINCT 1 ORDER BY 1 SKIP 1 LIMIT 1
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(RETURN_DISTINCT(LITERAL(1), AS("1"), ORDER_BY(LITERAL(1)),
|
|
SKIP(LITERAL(1)), LIMIT(LITERAL(1)))));
|
|
CheckPlan(storage, ExpectProduce(), ExpectDistinct(), ExpectOrderBy(),
|
|
ExpectSkip(), ExpectLimit());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateWithDistinctSumWhereReturn) {
|
|
// Test CREATE (n) WITH DISTINCT SUM(n.prop) AS s WHERE s < 42 RETURN s
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto node_n = NODE("n");
|
|
auto sum = SUM(PROPERTY_LOOKUP("n", prop));
|
|
auto query =
|
|
QUERY(SINGLE_QUERY(CREATE(PATTERN(node_n)), WITH_DISTINCT(sum, AS("s")),
|
|
WHERE(LESS(IDENT("s"), LITERAL(42))), RETURN("s")));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto acc = ExpectAccumulate({symbol_table.at(*node_n->identifier_)});
|
|
auto aggr = ExpectAggregate({sum}, {});
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectCreateNode(), acc, aggr, ExpectProduce(),
|
|
ExpectDistinct(), ExpectFilter(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchCrossReferenceVariable) {
|
|
// Test MATCH (n {prop: m.prop}), (m {prop: n.prop}) RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = PROPERTY_PAIR("prop");
|
|
AstTreeStorage storage;
|
|
auto node_n = NODE("n");
|
|
auto m_prop = PROPERTY_LOOKUP("m", prop.second);
|
|
node_n->properties_[prop] = m_prop;
|
|
auto node_m = NODE("m");
|
|
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
|
|
node_m->properties_[prop] = n_prop;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n), PATTERN(node_m)), RETURN("n")));
|
|
// We expect both ScanAll to come before filters (2 are joined into one),
|
|
// because they need to populate the symbol values.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectScanAll(), ExpectFilter(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWhereBeforeExpand) {
|
|
// Test MATCH (n) -[r]- (m) WHERE n.prop < 42 RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))),
|
|
RETURN("n")));
|
|
// We expect Fitler to come immediately after ScanAll, since it only uses `n`.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectFilter(), ExpectExpand(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MultiMatchWhere) {
|
|
// Test MATCH (n) -[r]- (m) MATCH (l) WHERE n.prop < 42 RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
MATCH(PATTERN(NODE("l"))),
|
|
WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))),
|
|
RETURN("n")));
|
|
// Even though WHERE is in the second MATCH clause, we expect Filter to come
|
|
// before second ScanAll, since it only uses the value from first ScanAll.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectFilter(), ExpectExpand(),
|
|
ExpectScanAll(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchOptionalMatchWhere) {
|
|
// Test MATCH (n) -[r]- (m) OPTIONAL MATCH (l) WHERE n.prop < 42 RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
OPTIONAL_MATCH(PATTERN(NODE("l"))),
|
|
WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))),
|
|
RETURN("n")));
|
|
// Even though WHERE is in the second MATCH clause, and it uses the value from
|
|
// first ScanAll, it must remain part of the Optional. It should come before
|
|
// optional ScanAll.
|
|
std::list<BaseOpChecker *> optional{new ExpectFilter(), new ExpectScanAll()};
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpand(), ExpectOptional(optional),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchReturnAsterisk) {
|
|
// Test MATCH (n) -[e]- (m) RETURN *, m.prop
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto ret = RETURN(PROPERTY_LOOKUP("m", prop), AS("m.prop"));
|
|
ret->body_.all_identifiers = true;
|
|
auto query =
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))), ret));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table, ExpectScanAll(), ExpectExpand(),
|
|
ExpectProduce());
|
|
std::vector<std::string> output_names;
|
|
for (const auto &output_symbol : plan->OutputSymbols(symbol_table)) {
|
|
output_names.emplace_back(output_symbol.name());
|
|
}
|
|
std::vector<std::string> expected_names{"e", "m", "n", "m.prop"};
|
|
EXPECT_EQ(output_names, expected_names);
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchReturnAsteriskSum) {
|
|
// Test MATCH (n) RETURN *, SUM(n.prop) AS s
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = dba.Property("prop");
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(PROPERTY_LOOKUP("n", prop));
|
|
auto ret = RETURN(sum, AS("s"));
|
|
ret->body_.all_identifiers = true;
|
|
auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), ret));
|
|
auto symbol_table = MakeSymbolTable(*query);
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
auto *produce = dynamic_cast<Produce *>(plan.get());
|
|
ASSERT_TRUE(produce);
|
|
const auto &named_expressions = produce->named_expressions();
|
|
ASSERT_EQ(named_expressions.size(), 2);
|
|
auto *expanded_ident =
|
|
dynamic_cast<query::Identifier *>(named_expressions[0]->expression_);
|
|
ASSERT_TRUE(expanded_ident);
|
|
auto aggr = ExpectAggregate({sum}, {expanded_ident});
|
|
CheckPlan(*plan, symbol_table, ExpectScanAll(), aggr, ExpectProduce());
|
|
std::vector<std::string> output_names;
|
|
for (const auto &output_symbol : plan->OutputSymbols(symbol_table)) {
|
|
output_names.emplace_back(output_symbol.name());
|
|
}
|
|
std::vector<std::string> expected_names{"n", "s"};
|
|
EXPECT_EQ(output_names, expected_names);
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, UnwindMergeNodeProperty) {
|
|
// Test UNWIND [1] AS i MERGE (n {prop: i})
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto node_n = NODE("n");
|
|
node_n->properties_[PROPERTY_PAIR("prop")] = IDENT("i");
|
|
QUERY(
|
|
SINGLE_QUERY(UNWIND(LIST(LITERAL(1)), AS("i")), MERGE(PATTERN(node_n))));
|
|
std::list<BaseOpChecker *> on_match{new ExpectScanAll(), new ExpectFilter()};
|
|
std::list<BaseOpChecker *> on_create{new ExpectCreateNode()};
|
|
CheckPlan(storage, ExpectUnwind(), ExpectMerge(on_match, on_create));
|
|
for (auto &op : on_match) delete op;
|
|
for (auto &op : on_create) delete op;
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MultipleOptionalMatchReturn) {
|
|
// Test OPTIONAL MATCH (n) OPTIONAL MATCH (m) RETURN n
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(OPTIONAL_MATCH(PATTERN(NODE("n"))),
|
|
OPTIONAL_MATCH(PATTERN(NODE("m"))), RETURN("n")));
|
|
std::list<BaseOpChecker *> optional{new ExpectScanAll()};
|
|
CheckPlan(storage, ExpectOptional(optional), ExpectOptional(optional),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, FunctionAggregationReturn) {
|
|
// Test RETURN sqrt(SUM(2)) AS result, 42 AS group_by
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(
|
|
RETURN(FN("sqrt", sum), AS("result"), group_by_literal, AS("group_by"))));
|
|
auto aggr = ExpectAggregate({sum}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, FunctionWithoutArguments) {
|
|
// Test RETURN pi() AS pi
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(RETURN(FN("pi"), AS("pi"))));
|
|
CheckPlan(storage, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ListLiteralAggregationReturn) {
|
|
// Test RETURN [SUM(2)] AS result, 42 AS group_by
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(
|
|
RETURN(LIST(sum), AS("result"), group_by_literal, AS("group_by"))));
|
|
auto aggr = ExpectAggregate({sum}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MapLiteralAggregationReturn) {
|
|
// Test RETURN {sum: SUM(2)} AS result, 42 AS group_by
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto sum = SUM(LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum}), AS("result"),
|
|
group_by_literal, AS("group_by"))));
|
|
auto aggr = ExpectAggregate({sum}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, EmptyListIndexAggregation) {
|
|
// Test RETURN [][SUM(2)] AS result, 42 AS group_by
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto empty_list = LIST();
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(
|
|
RETURN(storage.Create<query::ListMapIndexingOperator>(empty_list, sum),
|
|
AS("result"), group_by_literal, AS("group_by"))));
|
|
// We expect to group by '42' and the empty list, because it is a
|
|
// sub-expression of a binary operator which contains an aggregation. This is
|
|
// similar to grouping by '1' in `RETURN 1 + SUM(2)`.
|
|
auto aggr = ExpectAggregate({sum}, {empty_list, group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ListSliceAggregationReturn) {
|
|
// Test RETURN [1, 2][0..SUM(2)] AS result, 42 AS group_by
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto list = LIST(LITERAL(1), LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(RETURN(SLICE(list, LITERAL(0), sum), AS("result"),
|
|
group_by_literal, AS("group_by"))));
|
|
// Similarly to EmptyListIndexAggregation test, we expect grouping by list and
|
|
// '42', because slicing is an operator.
|
|
auto aggr = ExpectAggregate({sum}, {list, group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ListWithAggregationAndGroupBy) {
|
|
// Test RETURN [sum(2), 42]
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(RETURN(LIST(sum, group_by_literal), AS("result"))));
|
|
auto aggr = ExpectAggregate({sum}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, AggregatonWithListWithAggregationAndGroupBy) {
|
|
// Test RETURN sum(2), [sum(3), 42]
|
|
AstTreeStorage storage;
|
|
auto sum2 = SUM(LITERAL(2));
|
|
auto sum3 = SUM(LITERAL(3));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(
|
|
RETURN(sum2, AS("sum2"), LIST(sum3, group_by_literal), AS("list"))));
|
|
auto aggr = ExpectAggregate({sum2, sum3}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MapWithAggregationAndGroupBy) {
|
|
// Test RETURN {lit: 42, sum: sum(2)}
|
|
GraphDb db;
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LITERAL(2));
|
|
auto group_by_literal = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum},
|
|
{PROPERTY_PAIR("lit"), group_by_literal}),
|
|
AS("result"))));
|
|
auto aggr = ExpectAggregate({sum}, {group_by_literal});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, CreateIndex) {
|
|
// Test CREATE INDEX ON :Label(property)
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = dba.Property("property");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(CREATE_INDEX_ON(label, property)));
|
|
CheckPlan(storage, ExpectCreateIndex(label, property));
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, AtomIndexedLabelProperty) {
|
|
// Test MATCH (n :label {property: 42, not_indexed: 0}) RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = PROPERTY_PAIR("property");
|
|
auto not_indexed = PROPERTY_PAIR("not_indexed");
|
|
auto vertex = dba.InsertVertex();
|
|
vertex.add_label(label);
|
|
vertex.PropsSet(property.second, 42);
|
|
dba.Commit();
|
|
GraphDbAccessor(db).BuildIndex(label, property.second);
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
auto node = NODE("n", label);
|
|
auto lit_42 = LITERAL(42);
|
|
node->properties_[property] = lit_42;
|
|
node->properties_[not_indexed] = LITERAL(0);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(node)), RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, AtomPropertyWhereLabelIndexing) {
|
|
// Test MATCH (n {property: 42}) WHERE n.not_indexed AND n:label RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = PROPERTY_PAIR("property");
|
|
auto not_indexed = PROPERTY_PAIR("not_indexed");
|
|
dba.BuildIndex(label, property.second);
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
auto node = NODE("n");
|
|
auto lit_42 = LITERAL(42);
|
|
node->properties_[property] = lit_42;
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(node)),
|
|
WHERE(AND(PROPERTY_LOOKUP("n", not_indexed),
|
|
storage.Create<query::LabelsTest>(
|
|
IDENT("n"), std::vector<GraphDbTypes::Label>{label}))),
|
|
RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, WhereIndexedLabelProperty) {
|
|
// Test MATCH (n :label) WHERE n.property = 42 RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = PROPERTY_PAIR("property");
|
|
dba.BuildIndex(label, property.second);
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
auto lit_42 = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
|
|
WHERE(EQ(PROPERTY_LOOKUP("n", property), lit_42)),
|
|
RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
|
|
ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, BestPropertyIndexed) {
|
|
// Test MATCH (n :label) WHERE n.property = 1 AND n.better = 42 RETURN n
|
|
AstTreeStorage storage;
|
|
GraphDb db;
|
|
auto label = GraphDbAccessor(db).Label("label");
|
|
auto property = GraphDbAccessor(db).Property("property");
|
|
{
|
|
GraphDbAccessor(db).BuildIndex(label, property);
|
|
GraphDbAccessor dba(db);
|
|
// Add a vertex with :label+property combination, so that the best
|
|
// :label+better remains empty and thus better choice.
|
|
auto vertex = dba.InsertVertex();
|
|
vertex.add_label(label);
|
|
vertex.PropsSet(property, 1);
|
|
dba.Commit();
|
|
}
|
|
ASSERT_EQ(GraphDbAccessor(db).VerticesCount(label, property), 1);
|
|
auto better = PROPERTY_PAIR("better");
|
|
GraphDbAccessor(db).BuildIndex(label, better.second);
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
ASSERT_EQ(dba.VerticesCount(label, better.second), 0);
|
|
auto lit_42 = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
|
|
WHERE(AND(EQ(PROPERTY_LOOKUP("n", property), LITERAL(1)),
|
|
EQ(PROPERTY_LOOKUP("n", better), lit_42))),
|
|
RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyValue(label, better, lit_42),
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MultiPropertyIndexScan) {
|
|
// Test MATCH (n :label1), (m :label2) WHERE n.prop1 = 1 AND m.prop2 = 2
|
|
// RETURN n, m
|
|
GraphDb db;
|
|
auto label1 = GraphDbAccessor(db).Label("label1");
|
|
auto label2 = GraphDbAccessor(db).Label("label2");
|
|
auto prop1 = PROPERTY_PAIR("prop1");
|
|
auto prop2 = PROPERTY_PAIR("prop2");
|
|
GraphDbAccessor(db).BuildIndex(label1, prop1.second);
|
|
GraphDbAccessor(db).BuildIndex(label2, prop2.second);
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto lit_1 = LITERAL(1);
|
|
auto lit_2 = LITERAL(2);
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n", label1)), PATTERN(NODE("m", label2))),
|
|
WHERE(AND(EQ(PROPERTY_LOOKUP("n", prop1), lit_1),
|
|
EQ(PROPERTY_LOOKUP("m", prop2), lit_2))),
|
|
RETURN("n", "m")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyValue(label1, prop1, lit_1),
|
|
ExpectScanAllByLabelPropertyValue(label2, prop2, lit_2),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, WhereIndexedLabelPropertyRange) {
|
|
// Test MATCH (n :label) WHERE n.property REL_OP 42 RETURN n
|
|
// REL_OP is one of: `<`, `<=`, `>`, `>=`
|
|
GraphDb db;
|
|
auto label = GraphDbAccessor(db).Label("label");
|
|
auto property = GraphDbAccessor(db).Property("property");
|
|
GraphDbAccessor(db).BuildIndex(label, property);
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto lit_42 = LITERAL(42);
|
|
auto n_prop = PROPERTY_LOOKUP("n", property);
|
|
auto check_planned_range = [&label, &property, &dba](
|
|
const auto &rel_expr, auto lower_bound, auto upper_bound) {
|
|
// Shadow the first storage, so that the query is created in this one.
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), WHERE(rel_expr),
|
|
RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(*plan, symbol_table,
|
|
ExpectScanAllByLabelPropertyRange(label, property, lower_bound,
|
|
upper_bound),
|
|
ExpectProduce());
|
|
};
|
|
{
|
|
// Test relation operators which form an upper bound for range.
|
|
std::vector<std::pair<query::Expression *, Bound::Type>> upper_bound_rel_op{
|
|
std::make_pair(LESS(n_prop, lit_42), Bound::Type::EXCLUSIVE),
|
|
std::make_pair(LESS_EQ(n_prop, lit_42), Bound::Type::INCLUSIVE),
|
|
std::make_pair(GREATER(lit_42, n_prop), Bound::Type::EXCLUSIVE),
|
|
std::make_pair(GREATER_EQ(lit_42, n_prop), Bound::Type::INCLUSIVE)};
|
|
for (const auto &rel_op : upper_bound_rel_op) {
|
|
check_planned_range(rel_op.first, std::experimental::nullopt,
|
|
Bound(lit_42, rel_op.second));
|
|
}
|
|
}
|
|
{
|
|
// Test relation operators which form a lower bound for range.
|
|
std::vector<std::pair<query::Expression *, Bound::Type>> lower_bound_rel_op{
|
|
std::make_pair(LESS(lit_42, n_prop), Bound::Type::EXCLUSIVE),
|
|
std::make_pair(LESS_EQ(lit_42, n_prop), Bound::Type::INCLUSIVE),
|
|
std::make_pair(GREATER(n_prop, lit_42), Bound::Type::EXCLUSIVE),
|
|
std::make_pair(GREATER_EQ(n_prop, lit_42), Bound::Type::INCLUSIVE)};
|
|
for (const auto &rel_op : lower_bound_rel_op) {
|
|
check_planned_range(rel_op.first, Bound(lit_42, rel_op.second),
|
|
std::experimental::nullopt);
|
|
}
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, UnableToUsePropertyIndex) {
|
|
// Test MATCH (n: label) WHERE n.property = n.property RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = dba.Property("property");
|
|
dba.BuildIndex(label, property);
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
|
|
WHERE(EQ(PROPERTY_LOOKUP("n", property),
|
|
PROPERTY_LOOKUP("n", property))),
|
|
RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
// We can only get ScanAllByLabelIndex, because we are comparing properties
|
|
// with those on the same node.
|
|
CheckPlan(*plan, symbol_table, ExpectScanAllByLabel(), ExpectFilter(),
|
|
ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, SecondPropertyIndex) {
|
|
// Test MATCH (n :label), (m :label) WHERE m.property = n.property RETURN n
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
auto property = PROPERTY_PAIR("property");
|
|
dba.BuildIndex(label, dba.Property("property"));
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto n_prop = PROPERTY_LOOKUP("n", property);
|
|
auto m_prop = PROPERTY_LOOKUP("m", property);
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n", label)), PATTERN(NODE("m", label))),
|
|
WHERE(EQ(m_prop, n_prop)), RETURN("n")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
CheckPlan(
|
|
*plan, symbol_table, ExpectScanAllByLabel(),
|
|
// Note: We are scanning for m, therefore property should equal n_prop.
|
|
ExpectScanAllByLabelPropertyValue(label, property, n_prop),
|
|
ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ReturnSumGroupByAll) {
|
|
// Test RETURN sum([1,2,3]), all(x in [1] where x = 1)
|
|
AstTreeStorage storage;
|
|
auto sum = SUM(LIST(LITERAL(1), LITERAL(2), LITERAL(3)));
|
|
auto *all = ALL("x", LIST(LITERAL(1)), WHERE(EQ(IDENT("x"), LITERAL(1))));
|
|
QUERY(SINGLE_QUERY(RETURN(sum, AS("sum"), all, AS("all"))));
|
|
auto aggr = ExpectAggregate({sum}, {all});
|
|
CheckPlan(storage, aggr, ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchExpandVariable) {
|
|
// Test MATCH (n) -[r *..3]-> (m) RETURN r
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r");
|
|
edge->upper_bound_ = LITERAL(3);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpandVariable(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchExpandVariableNoBounds) {
|
|
// Test MATCH (n) -[r *]-> (m) RETURN r
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r");
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpandVariable(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchExpandVariableInlinedFilter) {
|
|
// Test MATCH (n) -[r :type * {prop: 42}]-> (m) RETURN r
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto type = dba.EdgeType("type");
|
|
auto prop = PROPERTY_PAIR("prop");
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r", Direction::BOTH, {type});
|
|
edge->properties_[prop] = LITERAL(42);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectScanAll(),
|
|
ExpectExpandVariable(), // Filter is both inlined and post-expand
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchExpandVariableNotInlinedFilter) {
|
|
// Test MATCH (n) -[r :type * {prop: m.prop}]-> (m) RETURN r
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto type = dba.EdgeType("type");
|
|
auto prop = PROPERTY_PAIR("prop");
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r", Direction::BOTH, {type});
|
|
edge->properties_[prop] = EQ(PROPERTY_LOOKUP("m", prop), LITERAL(42));
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpandVariable(), ExpectFilter(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, UnwindMatchVariable) {
|
|
// Test UNWIND [1,2,3] AS depth MATCH (n) -[r*d]-> (m) RETURN r
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r", Direction::OUT);
|
|
edge->lower_bound_ = IDENT("d");
|
|
edge->upper_bound_ = IDENT("d");
|
|
QUERY(SINGLE_QUERY(UNWIND(LIST(LITERAL(1), LITERAL(2), LITERAL(3)), AS("d")),
|
|
MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectUnwind(), ExpectScanAll(), ExpectExpandVariable(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchBreadthFirst) {
|
|
// Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto edge_type = dba.EdgeType("type");
|
|
AstTreeStorage storage;
|
|
auto *bfs = storage.Create<query::EdgeAtom>(
|
|
IDENT("r"), query::EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
|
|
std::vector<GraphDbTypes::EdgeType>{edge_type});
|
|
bfs->inner_edge_ = IDENT("r");
|
|
bfs->inner_node_ = IDENT("n");
|
|
bfs->filter_expression_ = IDENT("n");
|
|
bfs->upper_bound_ = LITERAL(10);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), bfs, NODE("m"))), RETURN("r")));
|
|
CheckPlan(storage, ExpectScanAll(), ExpectExpandBreadthFirst(),
|
|
ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchDoubleScanToExpandExisting) {
|
|
// Test MATCH (n) -[r]- (m :label) RETURN r
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto label = dba.Label("label");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m", label))),
|
|
RETURN("r")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
// We expect 2x ScanAll and then Expand, since we are guessing that is
|
|
// faster (due to low label index vertex count).
|
|
CheckPlan(*plan, symbol_table, ExpectScanAll(), ExpectScanAllByLabel(),
|
|
ExpectExpand(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchScanToExpand) {
|
|
// Test MATCH (n) -[r]- (m :label {property: 1}) RETURN r
|
|
GraphDb db;
|
|
auto label = GraphDbAccessor(db).Label("label");
|
|
auto property = GraphDbAccessor(db).Property("property");
|
|
GraphDbAccessor(db).BuildIndex(label, property);
|
|
GraphDbAccessor dba(db);
|
|
// Fill vertices to the max.
|
|
for (int64_t i = 0; i < FLAGS_query_vertex_count_to_expand_existing; ++i) {
|
|
auto vertex = dba.InsertVertex();
|
|
vertex.PropsSet(property, 1);
|
|
vertex.add_label(label);
|
|
}
|
|
// Add one more above the max.
|
|
auto vertex = dba.InsertVertex();
|
|
vertex.add_label(label);
|
|
vertex.PropsSet(property, 1);
|
|
dba.Commit();
|
|
{
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto node_m = NODE("m", label);
|
|
node_m->properties_[std::make_pair("property", property)] = LITERAL(1);
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), node_m)),
|
|
RETURN("r")));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
// We expect 1x ScanAllByLabel and then Expand, since we are guessing that
|
|
// is faster (due to high label index vertex count).
|
|
CheckPlan(*plan, symbol_table, ExpectScanAll(), ExpectExpand(),
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, MatchWhereAndSplit) {
|
|
// Test MATCH (n) -[r]- (m) WHERE n.prop AND r.prop RETURN m
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
auto prop = PROPERTY_PAIR("prop");
|
|
AstTreeStorage storage;
|
|
QUERY(SINGLE_QUERY(
|
|
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
|
|
WHERE(AND(PROPERTY_LOOKUP("n", prop), PROPERTY_LOOKUP("r", prop))),
|
|
RETURN("m")));
|
|
// We expect `n.prop` filter right after scanning `n`.
|
|
CheckPlan(storage, ExpectScanAll(), ExpectFilter(), ExpectExpand(),
|
|
ExpectFilter(), ExpectProduce());
|
|
}
|
|
|
|
TEST(TestLogicalPlanner, ReturnAsteriskOmitsLambdaSymbols) {
|
|
// Test MATCH (n) -[r* (ie, in | true)]- (m) RETURN *
|
|
GraphDb db;
|
|
GraphDbAccessor dba(db);
|
|
AstTreeStorage storage;
|
|
auto edge = EDGE_VARIABLE("r", Direction::BOTH);
|
|
edge->inner_edge_ = IDENT("ie");
|
|
edge->inner_node_ = IDENT("in");
|
|
edge->filter_expression_ = LITERAL(true);
|
|
auto ret = storage.Create<query::Return>();
|
|
ret->body_.all_identifiers = true;
|
|
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), ret));
|
|
auto symbol_table = MakeSymbolTable(*storage.query());
|
|
auto planning_context = MakePlanningContext(storage, symbol_table, dba);
|
|
auto query_parts = CollectQueryParts(symbol_table, storage);
|
|
ASSERT_TRUE(query_parts.query_parts.size() > 0);
|
|
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
|
|
auto plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
|
|
single_query_parts, planning_context);
|
|
auto *produce = dynamic_cast<Produce *>(plan.get());
|
|
ASSERT_TRUE(produce);
|
|
std::vector<std::string> outputs;
|
|
for (const auto &output_symbol : produce->OutputSymbols(symbol_table)) {
|
|
outputs.emplace_back(output_symbol.name());
|
|
}
|
|
// We expect `*` expanded to `n`, `r` and `m`.
|
|
EXPECT_EQ(outputs.size(), 3);
|
|
for (const auto &name : {"n", "r", "m"}) {
|
|
EXPECT_TRUE(utils::Contains(outputs, name));
|
|
}
|
|
}
|
|
|
|
} // namespace
|