Compare commits

...

7 Commits

Author SHA1 Message Date
Aidar Samerkhanov
0d8cdf5f6d
Merge branch 'master' into 1310-pattern-comprehension-executor 2024-03-25 11:44:05 +04:00
Aidar Samerkhanov
2895f604c3
Merge branch 'master' into 1310-pattern-comprehension-executor 2024-03-22 20:46:33 +04:00
Aidar Samerkhanov
ee575fd114 Add implementation for binary operators and function in PatternVisitor class. 2024-03-22 20:46:10 +04:00
Aidar Samerkhanov
beba77a42f
Merge branch 'master' into 1310-pattern-comprehension-executor 2024-03-21 01:42:55 +04:00
Aidar Samerkhanov
b480d7556e Add Reset() for list_aggregation_branch cursor in RollUpApply.
This resets state of Once and allow to collect data for new input value.
2024-03-21 01:42:30 +04:00
Aidar Samerkhanov
9bb2050373 Fix returning pattern comprehension and a property value. 2024-03-18 20:52:30 +04:00
Aidar Samerkhanov
c6cd2134a0 Implement execution of pattern comprehension.
Simple query works with one result works.
When multiple rows in return - doesn't work.
When after pattern comprehension, returing property - doesn't work.
2024-03-17 02:00:18 +04:00
9 changed files with 244 additions and 69 deletions

View File

@ -1105,8 +1105,15 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
}
}
TypedValue Visit(PatternComprehension & /*pattern_comprehension*/) override {
throw utils::NotYetImplemented("Expression evaluator can not handle pattern comprehension.");
TypedValue Visit(PatternComprehension &pattern_comprehension) override {
TypedValue &frame_pattern_comprehension_value = frame_->at(symbol_table_->at(pattern_comprehension));
if (!frame_pattern_comprehension_value.IsList()) [[unlikely]] {
throw QueryRuntimeException(
"Unexpected behavior: Pattern Comprehension expected a list, got {}. Please report the problem on GitHub "
"issues",
frame_pattern_comprehension_value.type());
}
return frame_pattern_comprehension_value;
}
private:

View File

@ -140,6 +140,7 @@ extern const Event EvaluatePatternFilterOperator;
extern const Event ApplyOperator;
extern const Event IndexedJoinOperator;
extern const Event HashJoinOperator;
extern const Event RollUpApplyOperator;
} // namespace memgraph::metrics
namespace memgraph::query::plan {
@ -5770,17 +5771,20 @@ UniqueCursorPtr HashJoin::MakeCursor(utils::MemoryResource *mem) const {
return MakeUniqueCursorPtr<HashJoinCursor>(mem, *this, mem);
}
RollUpApply::RollUpApply(const std::shared_ptr<LogicalOperator> &input,
std::shared_ptr<LogicalOperator> &&second_branch)
: input_(input), list_collection_branch_(second_branch) {}
RollUpApply::RollUpApply(const std::shared_ptr<LogicalOperator> &&input,
std::shared_ptr<LogicalOperator> &&list_collection_branch,
const std::vector<Symbol> &list_collection_symbols, Symbol result_symbol)
: input_(input),
list_collection_branch_(list_collection_branch),
list_collection_symbols_(list_collection_symbols),
result_symbol_(result_symbol) {}
std::vector<Symbol> RollUpApply::OutputSymbols(const SymbolTable & /*symbol_table*/) const {
std::vector<Symbol> symbols;
std::vector<Symbol> RollUpApply::ModifiedSymbols(const SymbolTable &table) const {
auto symbols = input_->ModifiedSymbols(table);
symbols.push_back(result_symbol_);
return symbols;
}
std::vector<Symbol> RollUpApply::ModifiedSymbols(const SymbolTable &table) const { return OutputSymbols(table); }
bool RollUpApply::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
if (visitor.PreVisit(*this)) {
if (!input_ || !list_collection_branch_) {
@ -5791,4 +5795,71 @@ bool RollUpApply::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.PostVisit(*this);
}
namespace {
class RollUpApplyCursor : public Cursor {
public:
RollUpApplyCursor(const RollUpApply &self, utils::MemoryResource *mem)
: self_(self),
input_cursor_(self.input_->MakeCursor(mem)),
list_collection_cursor_(self_.list_collection_branch_->MakeCursor(mem)) {
MG_ASSERT(input_cursor_ != nullptr, "RollUpApplyCursor: Missing left operator cursor.");
MG_ASSERT(list_collection_cursor_ != nullptr, "RollUpApplyCursor: Missing right operator cursor.");
MG_ASSERT(self_.list_collection_symbols_.size() == 1U, "Expected a single list collection symbol.");
}
bool Pull(Frame &frame, ExecutionContext &context) override {
OOMExceptionEnabler oom_exception;
SCOPED_PROFILE_OP_BY_REF(self_);
auto clear_frame_change_collector = [&context](const auto &symbols) {
for (const auto &symbol : symbols) {
if (context.frame_change_collector && context.frame_change_collector->IsKeyTracked(symbol.name())) {
context.frame_change_collector->ResetTrackingValue(symbol.name());
}
}
};
TypedValue result(std::vector<TypedValue>(), context.evaluation_context.memory);
if (input_cursor_->Pull(frame, context)) {
while (list_collection_cursor_->Pull(frame, context)) {
// collect values from the list collection branch
for (const auto &output_symbol : self_.list_collection_symbols_) {
result.ValueList().emplace_back(frame[output_symbol]);
}
}
clear_frame_change_collector(self_.list_collection_symbols_);
frame[self_.result_symbol_] = result;
// After a successful input from the list_collection_cursor_
// reset state of cursor because it has to a Once at the beginning
list_collection_cursor_->Reset();
} else {
return false;
}
return true;
}
void Shutdown() override {
input_cursor_->Shutdown();
list_collection_cursor_->Shutdown();
}
void Reset() override {
input_cursor_->Reset();
list_collection_cursor_->Reset();
}
private:
const RollUpApply &self_;
const UniqueCursorPtr input_cursor_;
const UniqueCursorPtr list_collection_cursor_;
};
} // namespace
UniqueCursorPtr RollUpApply::MakeCursor(utils::MemoryResource *mem) const {
memgraph::metrics::IncrementCounter(memgraph::metrics::RollUpApplyOperator);
return MakeUniqueCursorPtr<RollUpApplyCursor>(mem, *this, mem);
}
} // namespace memgraph::query::plan

View File

@ -2677,28 +2677,30 @@ class RollUpApply : public memgraph::query::plan::LogicalOperator {
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
RollUpApply() = default;
RollUpApply(const std::shared_ptr<LogicalOperator> &input, std::shared_ptr<LogicalOperator> &&second_branch);
RollUpApply(const std::shared_ptr<LogicalOperator> &&input, std::shared_ptr<LogicalOperator> &&list_collection_branch,
const std::vector<Symbol> &list_collection_symbols, Symbol result_symbol);
bool HasSingleInput() const override { return false; }
std::shared_ptr<LogicalOperator> input() const override { return input_; }
void set_input(std::shared_ptr<LogicalOperator> input) override { input_ = input; }
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override {
throw utils::NotYetImplemented("Execution of Pattern comprehension is currently unsupported.");
}
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override;
std::unique_ptr<LogicalOperator> Clone(AstStorage *storage) const override {
auto object = std::make_unique<RollUpApply>();
object->input_ = input_ ? input_->Clone(storage) : nullptr;
object->list_collection_branch_ = list_collection_branch_ ? list_collection_branch_->Clone(storage) : nullptr;
object->list_collection_symbols_ = list_collection_symbols_;
object->result_symbol_ = result_symbol_;
return object;
}
std::shared_ptr<memgraph::query::plan::LogicalOperator> input_;
std::shared_ptr<memgraph::query::plan::LogicalOperator> list_collection_branch_;
std::vector<Symbol> list_collection_symbols_;
Symbol result_symbol_;
};
} // namespace plan

View File

@ -684,9 +684,8 @@ static void ParseForeach(query::Foreach &foreach, SingleQueryPart &query_part, A
static void ParseReturn(query::Return &ret, AstStorage &storage, SymbolTable &symbol_table,
std::unordered_map<std::string, PatternComprehensionMatching> &matchings) {
PatternVisitor visitor(symbol_table, storage);
for (auto *expr : ret.body_.named_expressions) {
PatternVisitor visitor(symbol_table, storage);
expr->Accept(visitor);
auto pattern_comprehension_matchings = visitor.getPatternComprehensionMatchings();
for (auto &matching : pattern_comprehension_matchings) {
@ -702,6 +701,7 @@ void PatternVisitor::Visit(PatternComprehension &op) {
AddMatching({op.pattern_}, op.filter_, symbol_table_, storage_, matching);
matching.result_expr = storage_.Create<NamedExpression>(symbol_table_.at(op).name(), op.resultExpr_);
matching.result_expr->MapTo(symbol_table_.at(op));
matching.result_symbol = symbol_table_.at(op);
pattern_comprehension_matchings_.push_back(std::move(matching));
}

View File

@ -181,36 +181,81 @@ class PatternVisitor : public ExpressionVisitor<void> {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(XorOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(AndOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(NotEqualOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
};
}
void Visit(EqualOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
};
}
void Visit(InListOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
};
void Visit(AdditionOperator &op) override{};
void Visit(SubtractionOperator &op) override{};
void Visit(MultiplicationOperator &op) override{};
void Visit(DivisionOperator &op) override{};
void Visit(ModOperator &op) override{};
void Visit(LessOperator &op) override{};
void Visit(GreaterOperator &op) override{};
void Visit(LessEqualOperator &op) override{};
void Visit(GreaterEqualOperator &op) override{};
void Visit(SubscriptOperator &op) override{};
}
void Visit(AdditionOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(SubtractionOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(MultiplicationOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(DivisionOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(ModOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(LessOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(GreaterOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(LessEqualOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(GreaterEqualOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
void Visit(SubscriptOperator &op) override {
op.expression1_->Accept(*this);
op.expression2_->Accept(*this);
}
// Other
void Visit(ListSlicingOperator &op) override{};
@ -220,7 +265,13 @@ class PatternVisitor : public ExpressionVisitor<void> {
void Visit(MapProjectionLiteral &op) override{};
void Visit(LabelsTest &op) override{};
void Visit(Aggregation &op) override{};
void Visit(Function &op) override{};
void Visit(Function &op) override {
for (auto *argument : op.arguments_) {
argument->Accept(*this);
}
}
void Visit(Reduce &op) override{};
void Visit(Coalesce &op) override{};
void Visit(Extract &op) override{};
@ -503,6 +554,7 @@ inline auto Filters::IdFilters(const Symbol &symbol) const -> std::vector<Filter
struct PatternComprehensionMatching : Matching {
/// Pattern comprehension result named expression
NamedExpression *result_expr = nullptr;
Symbol result_symbol;
};
/// @brief Represents a read (+ write) part of a query. Parts are split on

View File

@ -19,6 +19,7 @@
#include <unordered_set>
#include "query/frontend/ast/ast.hpp"
#include "query/frontend/semantic/symbol_table.hpp"
#include "query/plan/operator.hpp"
#include "query/plan/preprocess.hpp"
#include "utils/algorithm.hpp"
@ -43,8 +44,7 @@ namespace {
class ReturnBodyContext : public HierarchicalTreeVisitor {
public:
ReturnBodyContext(const ReturnBody &body, SymbolTable &symbol_table, const std::unordered_set<Symbol> &bound_symbols,
AstStorage &storage, std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops,
Where *where = nullptr)
AstStorage &storage, PatternComprehensionDataMap &pc_ops, Where *where = nullptr)
: body_(body), symbol_table_(symbol_table), bound_symbols_(bound_symbols), storage_(storage), where_(where) {
// Collect symbols from named expressions.
output_symbols_.reserve(body_.named_expressions.size());
@ -57,13 +57,12 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
output_symbols_.emplace_back(symbol_table_.at(*named_expr));
named_expr->Accept(*this);
named_expressions_.emplace_back(named_expr);
if (pattern_comprehension_) {
if (auto it = pc_ops.find(named_expr->name_); it != pc_ops.end()) {
pattern_comprehension_op_ = std::move(it->second);
pc_ops.erase(it);
} else {
throw utils::NotYetImplemented("Operation on top of pattern comprehension");
}
// Pattern comprehension can be filled during named expression traversion
if (auto it = pc_ops.find(named_expr->name_); it != pc_ops.end()) {
MG_ASSERT(pattern_comprehension_data_.pattern_comprehension != nullptr);
pattern_comprehension_data_.op = std::move(it->second.op);
pc_ops.erase(it);
}
}
// Collect symbols used in group by expressions.
@ -399,18 +398,19 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
}
bool PreVisit(PatternComprehension & /*unused*/) override {
pattern_compression_aggregations_start_index_ = has_aggregation_.size();
aggregations_start_index_ = has_aggregation_.size();
return true;
}
bool PostVisit(PatternComprehension &pattern_comprehension) override {
bool has_aggr = false;
for (auto i = has_aggregation_.size(); i > pattern_compression_aggregations_start_index_; --i) {
for (auto i = has_aggregation_.size(); i > aggregations_start_index_; --i) {
has_aggr |= has_aggregation_.back();
has_aggregation_.pop_back();
}
has_aggregation_.emplace_back(has_aggr);
pattern_comprehension_ = &pattern_comprehension;
pattern_comprehension_data_.pattern_comprehension = &pattern_comprehension;
pattern_comprehension_data_.result_symbol = symbol_table_.at(pattern_comprehension);
return true;
}
@ -468,9 +468,11 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
// named_expressions.
const auto &output_symbols() const { return output_symbols_; }
const auto *pattern_comprehension() const { return pattern_comprehension_; }
const bool has_pattern_comprehension() const { return pattern_comprehension_data_.pattern_comprehension != nullptr; }
std::shared_ptr<LogicalOperator> pattern_comprehension_op() const { return pattern_comprehension_op_; }
const PatternComprehensionData pattern_comprehension_data() const { return pattern_comprehension_data_; }
const SymbolTable &symbol_table() const { return symbol_table_; }
private:
const ReturnBody &body_;
@ -493,9 +495,8 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
// group by it.
std::list<bool> has_aggregation_;
std::vector<NamedExpression *> named_expressions_;
PatternComprehension *pattern_comprehension_ = nullptr;
std::shared_ptr<LogicalOperator> pattern_comprehension_op_;
size_t pattern_compression_aggregations_start_index_ = 0;
PatternComprehensionData pattern_comprehension_data_;
size_t aggregations_start_index_ = 0;
};
std::unique_ptr<LogicalOperator> GenReturnBody(std::unique_ptr<LogicalOperator> input_op, bool advance_command,
@ -514,8 +515,11 @@ std::unique_ptr<LogicalOperator> GenReturnBody(std::unique_ptr<LogicalOperator>
last_op = std::make_unique<Aggregate>(std::move(last_op), body.aggregations(), body.group_by(), remember);
}
if (body.pattern_comprehension()) {
last_op = std::make_unique<RollUpApply>(std::move(last_op), body.pattern_comprehension_op());
if (body.has_pattern_comprehension()) {
auto list_collection_data = body.pattern_comprehension_data();
auto list_collection_symbols = list_collection_data.op->ModifiedSymbols(body.symbol_table());
last_op = std::make_unique<RollUpApply>(std::move(last_op), std::move(list_collection_data.op),
list_collection_symbols, list_collection_data.result_symbol);
}
last_op = std::make_unique<Produce>(std::move(last_op), body.named_expressions());
@ -580,9 +584,9 @@ Expression *ExtractFilters(const std::unordered_set<Symbol> &bound_symbols, Filt
return filter_expr;
}
std::unordered_set<Symbol> GetSubqueryBoundSymbols(
const std::vector<SingleQueryPart> &single_query_parts, SymbolTable &symbol_table, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) {
std::unordered_set<Symbol> GetSubqueryBoundSymbols(const std::vector<SingleQueryPart> &single_query_parts,
SymbolTable &symbol_table, AstStorage &storage,
PatternComprehensionDataMap &pc_ops) {
const auto &query = single_query_parts[0];
if (!query.matching.expansions.empty() || query.remaining_clauses.empty()) {
@ -622,7 +626,7 @@ std::unique_ptr<LogicalOperator> GenNamedPaths(std::unique_ptr<LogicalOperator>
std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,
const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) {
PatternComprehensionDataMap &pc_ops) {
// Similar to WITH clause, but we want to accumulate when the query writes to
// the database. This way we handle the case when we want to return
// expressions with the latest updated results. For example, `MATCH (n) -- ()
@ -638,7 +642,7 @@ std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalO
std::unique_ptr<LogicalOperator> GenWith(With &with, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,
std::unordered_set<Symbol> &bound_symbols, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) {
PatternComprehensionDataMap &pc_ops) {
// WITH clause is Accumulate/Aggregate (advance_command) + Produce and
// optional Filter. In case of update and aggregation, we want to accumulate
// first, so that when aggregating, we get the latest results. Similar to

View File

@ -27,6 +27,18 @@
namespace memgraph::query::plan {
struct PatternComprehensionData {
PatternComprehensionData() = default;
PatternComprehensionData(std::shared_ptr<LogicalOperator> lop, Symbol res_symbol)
: op(std::move(lop)), result_symbol(res_symbol) {}
PatternComprehension *pattern_comprehension = nullptr;
std::shared_ptr<LogicalOperator> op;
Symbol result_symbol;
};
using PatternComprehensionDataMap = std::unordered_map<std::string, PatternComprehensionData>;
/// @brief Context which contains variables commonly used during planning.
template <class TDbAccessor>
struct PlanningContext {
@ -88,9 +100,9 @@ bool HasBoundFilterSymbols(const std::unordered_set<Symbol> &bound_symbols, cons
// Returns the set of symbols for the subquery that are actually referenced from the outer scope and
// used in the subquery.
std::unordered_set<Symbol> GetSubqueryBoundSymbols(
const std::vector<SingleQueryPart> &single_query_parts, SymbolTable &symbol_table, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops);
std::unordered_set<Symbol> GetSubqueryBoundSymbols(const std::vector<SingleQueryPart> &single_query_parts,
SymbolTable &symbol_table, AstStorage &storage,
PatternComprehensionDataMap &pc_ops);
Symbol GetSymbol(NodeAtom *atom, const SymbolTable &symbol_table);
Symbol GetSymbol(EdgeAtom *atom, const SymbolTable &symbol_table);
@ -145,12 +157,12 @@ std::unique_ptr<LogicalOperator> GenNamedPaths(std::unique_ptr<LogicalOperator>
std::unique_ptr<LogicalOperator> GenReturn(Return &ret, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,
const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops);
PatternComprehensionDataMap &pc_ops);
std::unique_ptr<LogicalOperator> GenWith(With &with, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,
std::unordered_set<Symbol> &bound_symbols, AstStorage &storage,
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops);
PatternComprehensionDataMap &pc_ops);
std::unique_ptr<LogicalOperator> GenUnion(const CypherUnion &cypher_union, std::shared_ptr<LogicalOperator> left_op,
std::shared_ptr<LogicalOperator> right_op, SymbolTable &symbol_table);
@ -194,7 +206,7 @@ class RuleBasedPlanner {
uint64_t merge_id = 0;
uint64_t subquery_id = 0;
std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pattern_comprehension_ops;
PatternComprehensionDataMap pattern_comprehension_ops;
if (single_query_part.pattern_comprehension_matchings.size() > 1) {
throw utils::NotYetImplemented("Multiple pattern comprehensions.");
@ -204,7 +216,8 @@ class RuleBasedPlanner {
MatchContext match_ctx{matching.second, *context.symbol_table, context.bound_symbols};
new_input = PlanMatching(match_ctx, std::move(new_input));
new_input = std::make_unique<Produce>(std::move(new_input), std::vector{matching.second.result_expr});
pattern_comprehension_ops.emplace(matching.first, std::move(new_input));
pattern_comprehension_ops.emplace(
matching.first, PatternComprehensionData(std::move(new_input), matching.second.result_symbol));
}
for (const auto &clause : single_query_part.remaining_clauses) {
@ -875,9 +888,9 @@ class RuleBasedPlanner {
symbol);
}
std::unique_ptr<LogicalOperator> HandleSubquery(
std::unique_ptr<LogicalOperator> last_op, std::shared_ptr<QueryParts> subquery, SymbolTable &symbol_table,
AstStorage &storage, std::unordered_map<std::string, std::shared_ptr<LogicalOperator>> pc_ops) {
std::unique_ptr<LogicalOperator> HandleSubquery(std::unique_ptr<LogicalOperator> last_op,
std::shared_ptr<QueryParts> subquery, SymbolTable &symbol_table,
AstStorage &storage, PatternComprehensionDataMap &pc_ops) {
std::unordered_set<Symbol> outer_scope_bound_symbols;
outer_scope_bound_symbols.insert(std::make_move_iterator(context_->bound_symbols.begin()),
std::make_move_iterator(context_->bound_symbols.end()));

View File

@ -57,6 +57,7 @@
M(ApplyOperator, Operator, "Number of times ApplyOperator operator was used.") \
M(IndexedJoinOperator, Operator, "Number of times IndexedJoin operator was used.") \
M(HashJoinOperator, Operator, "Number of times HashJoin operator was used.") \
M(RollUpApplyOperator, Operator, "Number of times RollUpApply operator was used.") \
\
M(ActiveLabelIndices, Index, "Number of active label indices in the system.") \
M(ActiveLabelPropertyIndices, Index, "Number of active label property indices in the system.") \

View File

@ -287,10 +287,31 @@ Feature: List operators
MATCH (keanu:Person {name: 'Keanu Reeves'})
RETURN [(keanu)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.released] AS years
"""
Then an error should be raised
# Then the result should be:
# | years |
# | [2021,2003,2003,1999] |
Then the result should be:
| years |
| [2003, 2003, 1999, 2021] |
Scenario: List pattern comprehension and property
Given graph "graph_keanu"
When executing query:
"""
MATCH (keanu:Person {name: 'Keanu Reeves'})
RETURN [(keanu)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.released] AS years, keanu.name
"""
Then the result should be:
| years | keanu.name |
| [2003, 2003, 1999, 2021] | 'Keanu Reeves' |
Scenario: List pattern comprehension with function on selected property
Given graph "graph_keanu"
When executing query:
"""
MATCH (keanu:Person {name: 'Keanu Reeves'})
RETURN [(keanu)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | size(b.title)] AS movie_lens;
"""
Then the result should be:
| movie_lens |
| [22, 19, 10, 24] |
Scenario: Multiple entries with list pattern comprehension
Given graph "graph_keanu"
@ -299,7 +320,11 @@ Feature: List operators
MATCH (n:Person)
RETURN n.name, [(n)-->(b:Movie) WHERE b.title CONTAINS 'Matrix' | b.released] AS years
"""
Then an error should be raised
Then the result should be:
| n.name | years |
| 'Keanu Reeves' | [2003, 2003, 1999, 2021] |
| 'Carrie-Anne Moss' | [1999,2003] |
| 'Laurence Fishburne' | [1999] |
Scenario: Multiple list pattern comprehensions in Return
Given graph "graph_keanu"