Use make_unique instead of new when planning operators

Reviewers: florijan, msantl

Reviewed By: florijan

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1160
This commit is contained in:
Teon Banek 2018-02-01 09:48:28 +01:00
parent 27a90a5c25
commit c2d0348333
2 changed files with 178 additions and 174 deletions

View File

@ -57,18 +57,18 @@ auto ReducePattern(
auto prev_node = current_node;
current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
DCHECK(current_node) << "Expected a node atom in pattern.";
last_res = collect(last_res, prev_node, edge, current_node);
last_res = collect(std::move(last_res), prev_node, edge, current_node);
}
return last_res;
}
auto GenCreate(Create &create, LogicalOperator *input_op,
auto GenCreate(Create &create, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto last_op = input_op;
auto last_op = std::move(input_op);
for (auto pattern : create.patterns_) {
last_op = impl::GenCreateForPattern(*pattern, last_op, symbol_table,
bound_symbols);
last_op = impl::GenCreateForPattern(*pattern, std::move(last_op),
symbol_table, bound_symbols);
}
return last_op;
}
@ -437,52 +437,52 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
std::vector<NamedExpression *> named_expressions_;
};
auto GenReturnBody(LogicalOperator *input_op, bool advance_command,
const ReturnBodyContext &body, bool accumulate = false) {
std::unique_ptr<LogicalOperator> GenReturnBody(
std::unique_ptr<LogicalOperator> input_op, bool advance_command,
const ReturnBodyContext &body, bool accumulate = false) {
std::vector<Symbol> used_symbols(body.used_symbols().begin(),
body.used_symbols().end());
auto last_op = input_op;
auto last_op = std::move(input_op);
if (accumulate) {
// We only advance the command in Accumulate. This is done for WITH clause,
// when the first part updated the database. RETURN clause may only need an
// accumulation after updates, without advancing the command.
last_op = new Accumulate(std::shared_ptr<LogicalOperator>(last_op),
used_symbols, advance_command);
last_op = std::make_unique<Accumulate>(std::move(last_op), used_symbols,
advance_command);
}
if (!body.aggregations().empty()) {
// When we have aggregation, SKIP/LIMIT should always come after it.
std::vector<Symbol> remember(body.group_by_used_symbols().begin(),
body.group_by_used_symbols().end());
last_op = new Aggregate(std::shared_ptr<LogicalOperator>(last_op),
body.aggregations(), body.group_by(), remember);
last_op = std::make_unique<Aggregate>(
std::move(last_op), body.aggregations(), body.group_by(), remember);
}
last_op = new Produce(std::shared_ptr<LogicalOperator>(last_op),
body.named_expressions());
last_op =
std::make_unique<Produce>(std::move(last_op), body.named_expressions());
// Distinct in ReturnBody only makes Produce values unique, so plan after it.
if (body.distinct()) {
last_op = new Distinct(std::shared_ptr<LogicalOperator>(last_op),
body.output_symbols());
last_op =
std::make_unique<Distinct>(std::move(last_op), body.output_symbols());
}
// Like Where, OrderBy can read from symbols established by named expressions
// in Produce, so it must come after it.
if (!body.order_by().empty()) {
last_op = new OrderBy(std::shared_ptr<LogicalOperator>(last_op),
body.order_by(), body.output_symbols());
last_op = std::make_unique<OrderBy>(std::move(last_op), body.order_by(),
body.output_symbols());
}
// Finally, Skip and Limit must come after OrderBy.
if (body.skip()) {
last_op = new Skip(std::shared_ptr<LogicalOperator>(last_op), body.skip());
last_op = std::make_unique<Skip>(std::move(last_op), body.skip());
}
// Limit is always after Skip.
if (body.limit()) {
last_op =
new Limit(std::shared_ptr<LogicalOperator>(last_op), body.limit());
last_op = std::make_unique<Limit>(std::move(last_op), body.limit());
}
// Where may see new symbols so it comes after we generate Produce and in
// general, comes after any OrderBy, Skip or Limit.
if (body.where()) {
last_op = new Filter(std::shared_ptr<LogicalOperator>(last_op),
body.where()->expression_);
last_op =
std::make_unique<Filter>(std::move(last_op), body.where()->expression_);
}
return last_op;
}
@ -506,19 +506,20 @@ Expression *ExtractFilters(const std::unordered_set<Symbol> &bound_symbols,
return filter_expr;
}
LogicalOperator *GenFilters(LogicalOperator *last_op,
const std::unordered_set<Symbol> &bound_symbols,
Filters &filters, AstTreeStorage &storage) {
std::unique_ptr<LogicalOperator> GenFilters(
std::unique_ptr<LogicalOperator> last_op,
const std::unordered_set<Symbol> &bound_symbols, Filters &filters,
AstTreeStorage &storage) {
auto *filter_expr = ExtractFilters(bound_symbols, filters, storage);
if (filter_expr) {
last_op =
new Filter(std::shared_ptr<LogicalOperator>(last_op), filter_expr);
last_op = std::make_unique<Filter>(std::move(last_op), filter_expr);
}
return last_op;
}
LogicalOperator *GenNamedPaths(
LogicalOperator *last_op, std::unordered_set<Symbol> &bound_symbols,
std::unique_ptr<LogicalOperator> GenNamedPaths(
std::unique_ptr<LogicalOperator> last_op,
std::unordered_set<Symbol> &bound_symbols,
std::unordered_map<Symbol, std::vector<Symbol>> &named_paths) {
auto all_are_bound = [&bound_symbols](const std::vector<Symbol> &syms) {
for (const auto &sym : syms)
@ -528,8 +529,8 @@ LogicalOperator *GenNamedPaths(
for (auto named_path_it = named_paths.begin();
named_path_it != named_paths.end();) {
if (all_are_bound(named_path_it->second)) {
last_op = new ConstructNamedPath(
std::shared_ptr<LogicalOperator>(last_op), named_path_it->first,
last_op = std::make_unique<ConstructNamedPath>(
std::move(last_op), named_path_it->first,
std::move(named_path_it->second));
bound_symbols.insert(named_path_it->first);
named_path_it = named_paths.erase(named_path_it);
@ -541,10 +542,10 @@ LogicalOperator *GenNamedPaths(
return last_op;
}
LogicalOperator *GenReturn(Return &ret, LogicalOperator *input_op,
SymbolTable &symbol_table, bool is_write,
const std::unordered_set<Symbol> &bound_symbols,
AstTreeStorage &storage) {
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, AstTreeStorage &storage) {
// Similar to WITH clause, but we want to accumulate and advance command 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,
@ -554,22 +555,22 @@ LogicalOperator *GenReturn(Return &ret, LogicalOperator *input_op,
bool accumulate = is_write;
bool advance_command = false;
ReturnBodyContext body(ret.body_, symbol_table, bound_symbols, storage);
return GenReturnBody(input_op, advance_command, body, accumulate);
return GenReturnBody(std::move(input_op), advance_command, body, accumulate);
}
LogicalOperator *GenCreateForPattern(
Pattern &pattern, LogicalOperator *input_op,
std::unique_ptr<LogicalOperator> GenCreateForPattern(
Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto base = [&](NodeAtom *node) -> LogicalOperator * {
auto base = [&](NodeAtom *node) -> std::unique_ptr<LogicalOperator> {
if (bound_symbols.insert(symbol_table.at(*node->identifier_)).second)
return new CreateNode(node, std::shared_ptr<LogicalOperator>(input_op));
return std::make_unique<CreateNode>(node, std::move(input_op));
else
return input_op;
return std::move(input_op);
};
auto collect = [&](LogicalOperator *last_op, NodeAtom *prev_node,
EdgeAtom *edge, NodeAtom *node) {
auto collect = [&](std::unique_ptr<LogicalOperator> last_op,
NodeAtom *prev_node, EdgeAtom *edge, NodeAtom *node) {
// Store the symbol from the first node as the input to CreateExpand.
const auto &input_symbol = symbol_table.at(*prev_node->identifier_);
// If the expand node was already bound, then we need to indicate this,
@ -581,65 +582,66 @@ LogicalOperator *GenCreateForPattern(
if (!bound_symbols.insert(symbol_table.at(*edge->identifier_)).second) {
LOG(FATAL) << "Symbols used for created edges cannot be redeclared.";
}
return new CreateExpand(node, edge,
std::shared_ptr<LogicalOperator>(last_op),
input_symbol, node_existing);
return std::make_unique<CreateExpand>(node, edge, std::move(last_op),
input_symbol, node_existing);
};
LogicalOperator *last_op =
ReducePattern<LogicalOperator *>(pattern, base, collect);
auto last_op =
ReducePattern<std::unique_ptr<LogicalOperator>>(pattern, base, collect);
// If the pattern is named, append the path constructing logical operator.
if (pattern.identifier_->user_declared_) {
std::vector<Symbol> path_elements;
for (const PatternAtom *atom : pattern.atoms_)
path_elements.emplace_back(symbol_table.at(*atom->identifier_));
last_op = new ConstructNamedPath(std::shared_ptr<LogicalOperator>(last_op),
symbol_table.at(*pattern.identifier_),
path_elements);
last_op = std::make_unique<ConstructNamedPath>(
std::move(last_op), symbol_table.at(*pattern.identifier_),
path_elements);
}
return last_op;
}
// Generate an operator for a clause which writes to the database. If the clause
// isn't handled, returns nullptr.
LogicalOperator *HandleWriteClause(Clause *clause, LogicalOperator *input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
// Generate an operator for a clause which writes to the database. Ownership of
// input_op is transferred to the newly created operator. If the clause isn't
// handled, returns nullptr and input_op is left as is.
std::unique_ptr<LogicalOperator> HandleWriteClause(
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
if (auto *create = dynamic_cast<Create *>(clause)) {
return GenCreate(*create, input_op, symbol_table, bound_symbols);
return GenCreate(*create, std::move(input_op), symbol_table, bound_symbols);
} else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
return new plan::Delete(std::shared_ptr<LogicalOperator>(input_op),
del->expressions_, del->detach_);
return std::make_unique<plan::Delete>(std::move(input_op),
del->expressions_, del->detach_);
} else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
return new plan::SetProperty(std::shared_ptr<LogicalOperator>(input_op),
set->property_lookup_, set->expression_);
return std::make_unique<plan::SetProperty>(
std::move(input_op), set->property_lookup_, set->expression_);
} else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
auto op = set->update_ ? plan::SetProperties::Op::UPDATE
: plan::SetProperties::Op::REPLACE;
const auto &input_symbol = symbol_table.at(*set->identifier_);
return new plan::SetProperties(std::shared_ptr<LogicalOperator>(input_op),
input_symbol, set->expression_, op);
return std::make_unique<plan::SetProperties>(
std::move(input_op), input_symbol, set->expression_, op);
} else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*set->identifier_);
return new plan::SetLabels(std::shared_ptr<LogicalOperator>(input_op),
input_symbol, set->labels_);
return std::make_unique<plan::SetLabels>(std::move(input_op), input_symbol,
set->labels_);
} else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
return new plan::RemoveProperty(std::shared_ptr<LogicalOperator>(input_op),
rem->property_lookup_);
return std::make_unique<plan::RemoveProperty>(std::move(input_op),
rem->property_lookup_);
} else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*rem->identifier_);
return new plan::RemoveLabels(std::shared_ptr<LogicalOperator>(input_op),
input_symbol, rem->labels_);
return std::make_unique<plan::RemoveLabels>(std::move(input_op),
input_symbol, rem->labels_);
}
return nullptr;
}
LogicalOperator *GenWith(With &with, LogicalOperator *input_op,
SymbolTable &symbol_table, bool is_write,
std::unordered_set<Symbol> &bound_symbols,
AstTreeStorage &storage) {
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, AstTreeStorage &storage) {
// 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
@ -649,8 +651,8 @@ LogicalOperator *GenWith(With &with, LogicalOperator *input_op,
bool advance_command = is_write;
ReturnBodyContext body(with.body_, symbol_table, bound_symbols, storage,
with.where_);
LogicalOperator *last_op =
GenReturnBody(input_op, advance_command, body, accumulate);
auto last_op =
GenReturnBody(std::move(input_op), advance_command, body, accumulate);
// Reset bound symbols, so that only those in WITH are exposed.
bound_symbols.clear();
for (const auto &symbol : body.output_symbols()) {
@ -659,13 +661,12 @@ LogicalOperator *GenWith(With &with, LogicalOperator *input_op,
return last_op;
}
LogicalOperator *GenUnion(CypherUnion &cypher_union,
std::shared_ptr<LogicalOperator> left_op,
std::shared_ptr<LogicalOperator> right_op,
SymbolTable &symbol_table) {
return new Union(left_op, right_op, cypher_union.union_symbols_,
left_op->OutputSymbols(symbol_table),
right_op->OutputSymbols(symbol_table));
std::unique_ptr<LogicalOperator> GenUnion(
CypherUnion &cypher_union, std::shared_ptr<LogicalOperator> left_op,
std::shared_ptr<LogicalOperator> right_op, SymbolTable &symbol_table) {
return std::make_unique<Union>(left_op, right_op, cypher_union.union_symbols_,
left_op->OutputSymbols(symbol_table),
right_op->OutputSymbols(symbol_table));
}
} // namespace impl

View File

@ -69,41 +69,40 @@ namespace impl {
Expression *ExtractFilters(const std::unordered_set<Symbol> &, Filters &,
AstTreeStorage &);
LogicalOperator *GenFilters(LogicalOperator *,
const std::unordered_set<Symbol> &, Filters &,
AstTreeStorage &);
std::unique_ptr<LogicalOperator> GenFilters(std::unique_ptr<LogicalOperator>,
const std::unordered_set<Symbol> &,
Filters &, AstTreeStorage &);
// For all given `named_paths` checks if all its symbols have been bound.
// If so, it creates a logical operator for named path generation, binds its
// symbol, removes that path from the collection of unhandled ones and returns
// the new op. Otherwise, returns `nullptr`.
LogicalOperator *GenNamedPaths(
LogicalOperator *last_op, std::unordered_set<Symbol> &bound_symbols,
// the new op. Otherwise, returns `last_op`.
std::unique_ptr<LogicalOperator> GenNamedPaths(
std::unique_ptr<LogicalOperator> last_op,
std::unordered_set<Symbol> &bound_symbols,
std::unordered_map<Symbol, std::vector<Symbol>> &named_paths);
LogicalOperator *GenReturn(Return &ret, LogicalOperator *input_op,
SymbolTable &symbol_table, bool is_write,
const std::unordered_set<Symbol> &bound_symbols,
AstTreeStorage &storage);
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, AstTreeStorage &storage);
LogicalOperator *GenCreateForPattern(Pattern &pattern,
LogicalOperator *input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols);
std::unique_ptr<LogicalOperator> GenCreateForPattern(
Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
LogicalOperator *HandleWriteClause(Clause *clause, LogicalOperator *input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols);
std::unique_ptr<LogicalOperator> HandleWriteClause(
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
LogicalOperator *GenWith(With &with, LogicalOperator *input_op,
SymbolTable &symbol_table, bool is_write,
std::unordered_set<Symbol> &bound_symbols,
AstTreeStorage &storage);
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, AstTreeStorage &storage);
LogicalOperator *GenUnion(CypherUnion &cypher_union,
std::shared_ptr<LogicalOperator> left_op,
std::shared_ptr<LogicalOperator> right_op,
SymbolTable &symbol_table);
std::unique_ptr<LogicalOperator> GenUnion(
CypherUnion &cypher_union, std::shared_ptr<LogicalOperator> left_op,
std::shared_ptr<LogicalOperator> right_op, SymbolTable &symbol_table);
template <class TBoolOperator>
Expression *BoolJoin(AstTreeStorage &storage, Expression *expr1,
@ -130,21 +129,20 @@ class RuleBasedPlanner {
/// @brief Generates the operator tree based on explicitly set rules.
PlanResult Plan(const std::vector<SingleQueryPart> &query_parts) {
auto &context = context_;
LogicalOperator *input_op = nullptr;
std::unique_ptr<LogicalOperator> input_op;
// Set to true if a query command writes to the database.
bool is_write = false;
for (const auto &query_part : query_parts) {
MatchContext match_ctx{query_part.matching, context.symbol_table,
context.bound_symbols};
input_op = PlanMatching(match_ctx, input_op);
input_op = PlanMatching(match_ctx, std::move(input_op));
for (const auto &matching : query_part.optional_matching) {
MatchContext opt_ctx{matching, context.symbol_table,
context.bound_symbols};
auto *match_op = PlanMatching(opt_ctx, nullptr);
auto match_op = PlanMatching(opt_ctx, nullptr);
if (match_op) {
input_op = new Optional(std::shared_ptr<LogicalOperator>(input_op),
std::shared_ptr<LogicalOperator>(match_op),
opt_ctx.new_symbols);
input_op = std::make_unique<Optional>(
std::move(input_op), std::move(match_op), opt_ctx.new_symbols);
}
}
int merge_id = 0;
@ -152,44 +150,44 @@ class RuleBasedPlanner {
DCHECK(!dynamic_cast<Match *>(clause))
<< "Unexpected Match in remaining clauses";
if (auto *ret = dynamic_cast<Return *>(clause)) {
input_op =
impl::GenReturn(*ret, input_op, context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
input_op = impl::GenReturn(
*ret, std::move(input_op), context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
} else if (auto *merge = dynamic_cast<query::Merge *>(clause)) {
input_op =
GenMerge(*merge, input_op, query_part.merge_matching[merge_id++]);
input_op = GenMerge(*merge, std::move(input_op),
query_part.merge_matching[merge_id++]);
// Treat MERGE clause as write, because we do not know if it will
// create anything.
is_write = true;
} else if (auto *with = dynamic_cast<query::With *>(clause)) {
input_op =
impl::GenWith(*with, input_op, context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
input_op = impl::GenWith(*with, std::move(input_op),
context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
// WITH clause advances the command, so reset the flag.
is_write = false;
} else if (auto *op = impl::HandleWriteClause(clause, input_op,
context.symbol_table,
context.bound_symbols)) {
} else if (auto op = impl::HandleWriteClause(clause, input_op,
context.symbol_table,
context.bound_symbols)) {
is_write = true;
input_op = op;
input_op = std::move(op);
} else if (auto *unwind = dynamic_cast<query::Unwind *>(clause)) {
const auto &symbol =
context.symbol_table.at(*unwind->named_expression_);
context.bound_symbols.insert(symbol);
input_op =
new plan::Unwind(std::shared_ptr<LogicalOperator>(input_op),
unwind->named_expression_->expression_, symbol);
input_op = std::make_unique<plan::Unwind>(
std::move(input_op), unwind->named_expression_->expression_,
symbol);
} else if (auto *create_index =
dynamic_cast<query::CreateIndex *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before CreateIndex";
input_op = new plan::CreateIndex(create_index->label_,
create_index->property_);
input_op = std::make_unique<plan::CreateIndex>(
create_index->label_, create_index->property_);
} else {
throw utils::NotYetImplemented("clause conversion to operator(s)");
}
}
}
return std::unique_ptr<LogicalOperator>(input_op);
return input_op;
}
private:
@ -258,10 +256,11 @@ class RuleBasedPlanner {
// `max_vertex_count` controls, whether no operator should be created if the
// vertex count in the best index exceeds this number. In such a case,
// `nullptr` is returned and `last_op` is not chained.
ScanAll *GenScanByIndex(LogicalOperator *last_op, const Symbol &node_symbol,
const MatchContext &match_ctx, Filters &filters,
const std::experimental::optional<int64_t>
&max_vertex_count = std::experimental::nullopt) {
std::unique_ptr<ScanAll> GenScanByIndex(
std::unique_ptr<LogicalOperator> &last_op, const Symbol &node_symbol,
const MatchContext &match_ctx, Filters &filters,
const std::experimental::optional<int64_t> &max_vertex_count =
std::experimental::nullopt) {
const auto labels = filters.FilteredLabels(node_symbol);
if (labels.empty()) {
// Without labels, we cannot generated any indexed ScanAll.
@ -279,17 +278,16 @@ class RuleBasedPlanner {
filters.EraseFilter(found_index->filter);
filters.EraseLabelFilter(node_symbol, found_index->label);
if (prop_filter.lower_bound_ || prop_filter.upper_bound_) {
return new ScanAllByLabelPropertyRange(
std::shared_ptr<LogicalOperator>(last_op), node_symbol,
found_index->label, prop_filter.property_, prop_filter.lower_bound_,
return std::make_unique<ScanAllByLabelPropertyRange>(
std::move(last_op), node_symbol, found_index->label,
prop_filter.property_, prop_filter.lower_bound_,
prop_filter.upper_bound_, match_ctx.graph_view);
} else {
DCHECK(prop_filter.value_) << "Property filter should either have "
"bounds or a value expression.";
return new ScanAllByLabelPropertyValue(
std::shared_ptr<LogicalOperator>(last_op), node_symbol,
found_index->label, prop_filter.property_, prop_filter.value_,
match_ctx.graph_view);
return std::make_unique<ScanAllByLabelPropertyValue>(
std::move(last_op), node_symbol, found_index->label,
prop_filter.property_, prop_filter.value_, match_ctx.graph_view);
}
}
auto label = FindBestLabelIndex(labels);
@ -300,12 +298,12 @@ class RuleBasedPlanner {
return nullptr;
}
filters.EraseLabelFilter(node_symbol, label);
return new ScanAllByLabel(std::shared_ptr<LogicalOperator>(last_op),
node_symbol, label, match_ctx.graph_view);
return std::make_unique<ScanAllByLabel>(std::move(last_op), node_symbol,
label, match_ctx.graph_view);
}
LogicalOperator *PlanMatching(MatchContext &match_context,
LogicalOperator *input_op) {
std::unique_ptr<LogicalOperator> PlanMatching(
MatchContext &match_context, std::unique_ptr<LogicalOperator> input_op) {
auto &bound_symbols = match_context.bound_symbols;
auto &storage = context_.ast_storage;
const auto &symbol_table = match_context.symbol_table;
@ -317,25 +315,29 @@ class RuleBasedPlanner {
// Try to generate any filters even before the 1st match operator. This
// optimizes the optional match which filters only on symbols bound in
// regular match.
auto *last_op = impl::GenFilters(input_op, bound_symbols, filters, storage);
auto last_op =
impl::GenFilters(std::move(input_op), bound_symbols, filters, storage);
for (const auto &expansion : matching.expansions) {
const auto &node1_symbol = symbol_table.at(*expansion.node1->identifier_);
if (bound_symbols.insert(node1_symbol).second) {
// We have just bound this symbol, so generate ScanAll which fills it.
if (auto *indexed_scan =
if (auto indexed_scan =
GenScanByIndex(last_op, node1_symbol, match_context, filters)) {
// First, try to get an indexed scan.
last_op = indexed_scan;
last_op = std::move(indexed_scan);
} else {
// If indexed scan is not possible, we can only generate ScanAll of
// everything.
last_op = new ScanAll(std::shared_ptr<LogicalOperator>(last_op),
node1_symbol, match_context.graph_view);
last_op = std::make_unique<ScanAll>(std::move(last_op), node1_symbol,
match_context.graph_view);
}
match_context.new_symbols.emplace_back(node1_symbol);
last_op = impl::GenFilters(last_op, bound_symbols, filters, storage);
last_op = impl::GenNamedPaths(last_op, bound_symbols, named_paths);
last_op = impl::GenFilters(last_op, bound_symbols, filters, storage);
last_op = impl::GenFilters(std::move(last_op), bound_symbols, filters,
storage);
last_op =
impl::GenNamedPaths(std::move(last_op), bound_symbols, named_paths);
last_op = impl::GenFilters(std::move(last_op), bound_symbols, filters,
storage);
}
// We have an edge, so generate Expand.
if (expansion.edge) {
@ -384,12 +386,12 @@ class RuleBasedPlanner {
bound_symbols.erase(inner_edge_symbol);
bound_symbols.erase(inner_node_symbol);
last_op = new ExpandVariable(
last_op = std::make_unique<ExpandVariable>(
node_symbol, edge_symbol, edge->type_, expansion.direction,
edge->edge_types_, expansion.is_flipped, edge->lower_bound_,
edge->upper_bound_, std::shared_ptr<LogicalOperator>(last_op),
node1_symbol, existing_node, inner_edge_symbol, inner_node_symbol,
filter_expr, match_context.graph_view);
edge->upper_bound_, std::move(last_op), node1_symbol,
existing_node, inner_edge_symbol, inner_node_symbol, filter_expr,
match_context.graph_view);
} else {
if (!existing_node) {
// Try to get better behaviour by creating an indexed scan and then
@ -399,18 +401,18 @@ class RuleBasedPlanner {
// want to replace Expand with index ScanAll + Expand into existing.
// It would be better to somehow test whether the input vertex
// degree is larger than the destination vertex index count.
auto *indexed_scan =
auto indexed_scan =
GenScanByIndex(last_op, node_symbol, match_context, filters,
FLAGS_query_vertex_count_to_expand_existing);
if (indexed_scan) {
last_op = indexed_scan;
last_op = std::move(indexed_scan);
existing_node = true;
}
}
last_op = new Expand(
last_op = std::make_unique<Expand>(
node_symbol, edge_symbol, expansion.direction, edge->edge_types_,
std::shared_ptr<LogicalOperator>(last_op), node1_symbol,
existing_node, match_context.graph_view);
std::move(last_op), node1_symbol, existing_node,
match_context.graph_view);
}
// Bind the expanded edge and node.
@ -435,14 +437,16 @@ class RuleBasedPlanner {
other_symbols.push_back(symbol);
}
if (!other_symbols.empty()) {
last_op = new ExpandUniquenessFilter<EdgeAccessor>(
std::shared_ptr<LogicalOperator>(last_op), edge_symbol,
other_symbols);
last_op = std::make_unique<ExpandUniquenessFilter<EdgeAccessor>>(
std::move(last_op), edge_symbol, other_symbols);
}
}
last_op = impl::GenFilters(last_op, bound_symbols, filters, storage);
last_op = impl::GenNamedPaths(last_op, bound_symbols, named_paths);
last_op = impl::GenFilters(last_op, bound_symbols, filters, storage);
last_op = impl::GenFilters(std::move(last_op), bound_symbols, filters,
storage);
last_op =
impl::GenNamedPaths(std::move(last_op), bound_symbols, named_paths);
last_op = impl::GenFilters(std::move(last_op), bound_symbols, filters,
storage);
}
}
DCHECK(named_paths.empty()) << "Expected to generate all named paths";
@ -456,7 +460,7 @@ class RuleBasedPlanner {
return last_op;
}
auto GenMerge(query::Merge &merge, LogicalOperator *input_op,
auto GenMerge(query::Merge &merge, std::unique_ptr<LogicalOperator> input_op,
const Matching &matching) {
// Copy the bound symbol set, because we don't want to use the updated
// version when generating the create part.
@ -478,9 +482,8 @@ class RuleBasedPlanner {
context_.bound_symbols);
DCHECK(on_match) << "Expected SET in MERGE ... ON MATCH";
}
return new plan::Merge(std::shared_ptr<LogicalOperator>(input_op),
std::shared_ptr<LogicalOperator>(on_match),
std::shared_ptr<LogicalOperator>(on_create));
return std::make_unique<plan::Merge>(
std::move(input_op), std::move(on_match), std::move(on_create));
}
};