Add caching logical plans

Summary: Add const to MakeCursor method

Reviewers: florijan, mislav.bradac

Reviewed By: mislav.bradac

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D791
This commit is contained in:
Teon Banek 2017-09-19 16:58:22 +02:00
parent cf7190ecc6
commit 08eaaba77f
6 changed files with 304 additions and 184 deletions

View File

@ -48,6 +48,7 @@ class StrippedQuery {
StrippedQuery &operator=(StrippedQuery &&other) = default;
const std::string &query() const { return query_; }
const auto &original_query() const { return original_; }
const auto &literals() const { return literals_; }
const auto &named_expressions() const { return named_exprs_; }
const auto &parameters() const { return parameters_; }

View File

@ -1,5 +1,10 @@
#include "query/interpreter.hpp"
#include "query/plan/cost_estimator.hpp"
#include "query/plan/planner.hpp"
#include "query/plan/vertex_count_cache.hpp"
#include "utils/flag_validation.hpp"
// TODO: Remove this flag. Ast caching can be disabled by setting this flag to
// false, this is useful for recerating antlr crashes in highly concurrent test.
// Once antlr bugs are fixed, or real test is written this flag can be removed.
@ -7,3 +12,81 @@ DEFINE_bool(ast_cache, true, "Use ast caching.");
DEFINE_bool(query_cost_planner, true,
"Use the cost estimator to generate plans for queries.");
DEFINE_bool(query_plan_cache, true, "Cache generated query plans");
DEFINE_VALIDATED_int32(
query_cache_expire_seconds, 60,
"Expire cached queries after this amount of seconds since caching",
FLAG_IN_RANGE(0, std::numeric_limits<int32_t>::max()));
namespace query {
AstTreeStorage Interpreter::QueryToAst(const StrippedQuery &stripped,
Context &ctx) {
if (!ctx.is_query_cached_) {
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(
stripped.original_query());
}();
auto low_level_tree = parser->tree();
// AST -> high level tree
frontend::CypherMainVisitor visitor(ctx);
visitor.visit(low_level_tree);
return std::move(visitor.storage());
}
auto ast_cache_accessor = ast_cache_.access();
auto ast_it = ast_cache_accessor.find(stripped.hash());
if (ast_it == ast_cache_accessor.end()) {
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(stripped.query());
}();
auto low_level_tree = parser->tree();
// AST -> high level tree
frontend::CypherMainVisitor visitor(ctx);
visitor.visit(low_level_tree);
// Cache it.
ast_it =
ast_cache_accessor.insert(stripped.hash(), std::move(visitor.storage()))
.first;
}
AstTreeStorage new_ast;
ast_it->second.query()->Clone(new_ast);
return new_ast;
}
std::pair<std::unique_ptr<plan::LogicalOperator>, double>
Interpreter::MakeLogicalPlan(AstTreeStorage &ast_storage,
const GraphDbAccessor &db_accessor,
Context &context) {
std::unique_ptr<plan::LogicalOperator> logical_plan;
double min_cost = std::numeric_limits<double>::max();
auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
if (FLAGS_query_cost_planner) {
auto plans = plan::MakeLogicalPlan<plan::VariableStartPlanner>(
ast_storage, context.symbol_table_, vertex_counts);
for (auto &plan : plans) {
auto cost = EstimatePlanCost(vertex_counts, context.parameters_, *plan);
if (!logical_plan || cost < min_cost) {
// We won't be iterating over plans anymore, so it's ok to invalidate
// unique_ptrs inside.
logical_plan = std::move(plan);
min_cost = cost;
}
}
} else {
logical_plan = plan::MakeLogicalPlan<plan::RuleBasedPlanner>(
ast_storage, context.symbol_table_, vertex_counts);
min_cost =
EstimatePlanCost(vertex_counts, context.parameters_, *logical_plan);
}
return {std::move(logical_plan), min_cost};
};
} // namespace query

View File

@ -14,25 +14,60 @@
#include "query/frontend/semantic/symbol_generator.hpp"
#include "query/frontend/stripped.hpp"
#include "query/interpret/frame.hpp"
#include "query/plan/cost_estimator.hpp"
#include "query/plan/planner.hpp"
#include "query/plan/vertex_count_cache.hpp"
#include "query/plan/operator.hpp"
#include "threading/sync/spinlock.hpp"
#include "utils/timer.hpp"
// TODO: Remove ast_cache flag and add flag that limits cache size.
DECLARE_bool(ast_cache);
DECLARE_bool(query_cost_planner);
DECLARE_bool(query_plan_cache);
DECLARE_int32(query_cache_expire_seconds);
namespace query {
class Interpreter {
private:
class CachedPlan {
public:
CachedPlan(std::unique_ptr<plan::LogicalOperator> plan, double cost,
SymbolTable symbol_table, AstTreeStorage storage)
: plan_(std::move(plan)),
cost_(cost),
symbol_table_(symbol_table),
ast_storage_(std::move(storage)) {}
const auto &plan() const { return *plan_; }
double cost() const { return cost_; }
const auto &symbol_table() const { return symbol_table_; }
bool IsExpired() const {
auto elapsed = cache_timer_.Elapsed();
return std::chrono::duration_cast<std::chrono::seconds>(elapsed) >
std::chrono::seconds(FLAGS_query_cache_expire_seconds);
};
private:
std::unique_ptr<plan::LogicalOperator> plan_;
double cost_;
SymbolTable symbol_table_;
AstTreeStorage ast_storage_;
utils::Timer cache_timer_;
};
public:
Interpreter() {}
template <typename Stream>
void Interpret(const std::string &query, GraphDbAccessor &db_accessor,
Stream &stream,
const std::map<std::string, TypedValue> &params) {
if (!FLAGS_ast_cache && !params.empty()) {
// This is totally fine, since we don't really expect anyone to turn off
// the cache.
throw utils::NotYetImplemented(
"Params not implemented if ast cache is turned off");
}
utils::Timer frontend_timer;
Context ctx(db_accessor);
ctx.is_query_cached_ = FLAGS_ast_cache;
@ -40,98 +75,83 @@ class Interpreter {
// query -> stripped query
StrippedQuery stripped(query);
// stripped query -> high level tree
AstTreeStorage ast_storage = [&]() {
if (!ctx.is_query_cached_) {
// This is totally fine, since we don't really expect anyone to turn off
// the cache.
if (!params.empty()) {
throw utils::NotYetImplemented(
"Params not implemented if ast cache is turned off");
}
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(query);
}();
auto low_level_tree = parser->tree();
// AST -> high level tree
frontend::CypherMainVisitor visitor(ctx);
visitor.visit(low_level_tree);
return std::move(visitor.storage());
// Update context with provided parameters.
ctx.parameters_ = stripped.literals();
for (const auto &param_pair : stripped.parameters()) {
auto param_it = params.find(param_pair.second);
if (param_it == params.end()) {
throw query::UnprovidedParameterError(
fmt::format("Parameter$ {} not provided", param_pair.second));
}
ctx.parameters_.Add(param_pair.first, param_it->second);
}
auto ast_cache_accessor = ast_cache_.access();
auto ast_it = ast_cache_accessor.find(stripped.hash());
if (ast_it == ast_cache_accessor.end()) {
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(
stripped.query());
}();
auto low_level_tree = parser->tree();
std::shared_ptr<CachedPlan> cached_plan;
std::experimental::optional<AstTreeStorage> ast_storage;
// Check if we have a cached logical plan ready, so that we can skip the
// whole query -> AST -> logical_plan process.
auto plan_cache_accessor = plan_cache_.access();
auto plan_cache_it = plan_cache_accessor.find(stripped.hash());
if (plan_cache_it != plan_cache_accessor.end() &&
plan_cache_it->second->IsExpired()) {
// Remove the expired plan.
plan_cache_accessor.remove(stripped.hash());
plan_cache_it = plan_cache_accessor.end();
}
if (plan_cache_it == plan_cache_accessor.end()) {
// We didn't find a cached plan or it was expired.
// stripped query -> high level tree
ast_storage = QueryToAst(stripped, ctx);
} else {
cached_plan = plan_cache_it->second;
}
// AST -> high level tree
frontend::CypherMainVisitor visitor(ctx);
visitor.visit(low_level_tree);
// Cache it.
ast_it = ast_cache_accessor
.insert(stripped.hash(), std::move(visitor.storage()))
.first;
}
// Update context with provided parameters.
ctx.parameters_ = stripped.literals();
for (const auto &param_pair : stripped.parameters()) {
auto param_it = params.find(param_pair.second);
if (param_it == params.end()) {
throw query::UnprovidedParameterError(
fmt::format("Parameter$ {} not provided", param_pair.second));
}
ctx.parameters_.Add(param_pair.first, param_it->second);
}
AstTreeStorage new_ast;
ast_it->second.query()->Clone(new_ast);
return new_ast;
}();
auto frontend_time = frontend_timer.Elapsed();
utils::Timer planning_timer;
// symbol table fill
SymbolGenerator symbol_generator(ctx.symbol_table_);
ast_storage.query()->Accept(symbol_generator);
// high level tree -> logical plan
std::unique_ptr<plan::LogicalOperator> logical_plan;
auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
auto fill_symbol_table = [](auto &ast_storage, auto &symbol_table) {
SymbolGenerator symbol_generator(symbol_table);
ast_storage.query()->Accept(symbol_generator);
};
// If the plan is not stored in the cache, `tmp_logical_plan` owns the newly
// generated plan. Otherwise, it is empty and `cached_plan` owns the plan.
// In all cases, `logical_plan` references the plan to be used.
std::unique_ptr<plan::LogicalOperator> tmp_logical_plan;
const plan::LogicalOperator *logical_plan = nullptr;
double query_plan_cost_estimation = 0.0;
if (FLAGS_query_cost_planner) {
auto plans = plan::MakeLogicalPlan<plan::VariableStartPlanner>(
ast_storage, ctx.symbol_table_, vertex_counts);
double min_cost = std::numeric_limits<double>::max();
for (auto &plan : plans) {
auto cost = EstimatePlanCost(vertex_counts, ctx.parameters_, *plan);
if (!logical_plan || cost < min_cost) {
// We won't be iterating over plans anymore, so it's ok to invalidate
// unique_ptrs inside.
logical_plan = std::move(plan);
min_cost = cost;
}
if (FLAGS_query_plan_cache) {
if (!cached_plan) {
debug_assert(ast_storage, "AST is required to generate a plan");
fill_symbol_table(*ast_storage, ctx.symbol_table_);
std::tie(tmp_logical_plan, query_plan_cost_estimation) =
MakeLogicalPlan(*ast_storage, db_accessor, ctx);
// Cache the generated plan.
auto plan_cache_accessor = plan_cache_.access();
auto plan_cache_it =
plan_cache_accessor
.insert(
stripped.hash(),
std::make_shared<CachedPlan>(
std::move(tmp_logical_plan), query_plan_cost_estimation,
ctx.symbol_table_, std::move(*ast_storage)))
.first;
cached_plan = plan_cache_it->second;
}
query_plan_cost_estimation = min_cost;
query_plan_cost_estimation = cached_plan->cost();
ctx.symbol_table_ = cached_plan->symbol_table();
logical_plan = &cached_plan->plan();
} else {
logical_plan = plan::MakeLogicalPlan<plan::RuleBasedPlanner>(
ast_storage, ctx.symbol_table_, vertex_counts);
query_plan_cost_estimation =
EstimatePlanCost(vertex_counts, ctx.parameters_, *logical_plan);
debug_assert(ast_storage, "Without plan caching, AST must be generated.");
fill_symbol_table(*ast_storage, ctx.symbol_table_);
std::tie(tmp_logical_plan, query_plan_cost_estimation) =
MakeLogicalPlan(*ast_storage, db_accessor, ctx);
logical_plan = tmp_logical_plan.get();
}
// Below this point, ast_storage should not be used. Other than not allowing
// modifications, the ast_storage may have moved to a cache.
// generate frame based on symbol table max_position
Frame frame(ctx.symbol_table_.max_position());
@ -166,16 +186,16 @@ class Interpreter {
values.emplace_back(frame[symbol]);
stream.Result(values);
}
} else if (dynamic_cast<plan::CreateNode *>(logical_plan.get()) ||
dynamic_cast<plan::CreateExpand *>(logical_plan.get()) ||
dynamic_cast<plan::SetProperty *>(logical_plan.get()) ||
dynamic_cast<plan::SetProperties *>(logical_plan.get()) ||
dynamic_cast<plan::SetLabels *>(logical_plan.get()) ||
dynamic_cast<plan::RemoveProperty *>(logical_plan.get()) ||
dynamic_cast<plan::RemoveLabels *>(logical_plan.get()) ||
dynamic_cast<plan::Delete *>(logical_plan.get()) ||
dynamic_cast<plan::Merge *>(logical_plan.get()) ||
dynamic_cast<plan::CreateIndex *>(logical_plan.get())) {
} else if (dynamic_cast<const plan::CreateNode *>(logical_plan) ||
dynamic_cast<const plan::CreateExpand *>(logical_plan) ||
dynamic_cast<const plan::SetProperty *>(logical_plan) ||
dynamic_cast<const plan::SetProperties *>(logical_plan) ||
dynamic_cast<const plan::SetLabels *>(logical_plan) ||
dynamic_cast<const plan::RemoveProperty *>(logical_plan) ||
dynamic_cast<const plan::RemoveLabels *>(logical_plan) ||
dynamic_cast<const plan::Delete *>(logical_plan) ||
dynamic_cast<const plan::Merge *>(logical_plan) ||
dynamic_cast<const plan::CreateIndex *>(logical_plan)) {
stream.Header(header);
auto cursor = logical_plan->MakeCursor(db_accessor);
while (cursor->Pull(frame, ctx)) continue;
@ -201,7 +221,16 @@ class Interpreter {
}
private:
// stripped query -> high level tree
AstTreeStorage QueryToAst(const StrippedQuery &stripped, Context &ctx);
// high level tree -> (logical plan, plan cost)
// AstTreeStorage and SymbolTable may be modified during planning.
std::pair<std::unique_ptr<plan::LogicalOperator>, double> MakeLogicalPlan(
AstTreeStorage &, const GraphDbAccessor &, Context &);
ConcurrentMap<HashType, AstTreeStorage> ast_cache_;
ConcurrentMap<HashType, std::shared_ptr<CachedPlan>> plan_cache_;
// Antlr has singleton instance that is shared between threads. It is
// protected by locks inside of antlr. Unfortunately, they are not protected
// in a very good way. Once we have antlr version without race conditions we

View File

@ -68,7 +68,7 @@ bool Once::OnceCursor::Pull(Frame &, Context &) {
return false;
}
std::unique_ptr<Cursor> Once::MakeCursor(GraphDbAccessor &) {
std::unique_ptr<Cursor> Once::MakeCursor(GraphDbAccessor &) const {
return std::make_unique<OnceCursor>();
}
@ -80,7 +80,7 @@ CreateNode::CreateNode(const NodeAtom *node_atom,
ACCEPT_WITH_INPUT(CreateNode)
std::unique_ptr<Cursor> CreateNode::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> CreateNode::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<CreateNodeCursor>(*this, db);
}
@ -122,7 +122,7 @@ CreateExpand::CreateExpand(const NodeAtom *node_atom, const EdgeAtom *edge_atom,
ACCEPT_WITH_INPUT(CreateExpand)
std::unique_ptr<Cursor> CreateExpand::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> CreateExpand::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<CreateExpandCursor>(*this, db);
}
@ -257,7 +257,7 @@ ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(ScanAll)
std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) const {
auto vertices = [this, &db](Frame &, Context &) {
return db.Vertices(graph_view_ == GraphView::NEW);
};
@ -272,7 +272,7 @@ ScanAllByLabel::ScanAllByLabel(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(ScanAllByLabel)
std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(GraphDbAccessor &db) const {
auto vertices = [this, &db](Frame &, Context &) {
return db.Vertices(label_, graph_view_ == GraphView::NEW);
};
@ -296,7 +296,7 @@ ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
ACCEPT_WITH_INPUT(ScanAllByLabelPropertyRange)
std::unique_ptr<Cursor> ScanAllByLabelPropertyRange::MakeCursor(
GraphDbAccessor &db) {
GraphDbAccessor &db) const {
auto is_less = [](const TypedValue &a, const TypedValue &b,
Bound::Type bound_type) {
try {
@ -387,7 +387,7 @@ class ScanAllByLabelPropertyValueCursor : public Cursor {
};
std::unique_ptr<Cursor> ScanAllByLabelPropertyValue::MakeCursor(
GraphDbAccessor &db) {
GraphDbAccessor &db) const {
return std::make_unique<ScanAllByLabelPropertyValueCursor>(*this, db);
}
@ -423,7 +423,7 @@ bool ExpandCommon::HandleExistingNode(const VertexAccessor &new_node,
ACCEPT_WITH_INPUT(Expand)
std::unique_ptr<Cursor> Expand::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Expand::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<ExpandCursor>(*this, db);
}
@ -960,7 +960,7 @@ class ExpandVariableCursor : public Cursor {
}
};
std::unique_ptr<Cursor> ExpandVariable::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> ExpandVariable::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<ExpandVariableCursor>(*this, db);
}
@ -985,7 +985,8 @@ ExpandBreadthFirst::ExpandBreadthFirst(
ACCEPT_WITH_INPUT(ExpandBreadthFirst)
std::unique_ptr<Cursor> ExpandBreadthFirst::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> ExpandBreadthFirst::MakeCursor(
GraphDbAccessor &db) const {
return std::make_unique<ExpandBreadthFirst::Cursor>(*this, db);
}
@ -1203,7 +1204,7 @@ class ConstructNamedPathCursor : public Cursor {
ACCEPT_WITH_INPUT(ConstructNamedPath)
std::unique_ptr<Cursor> ConstructNamedPath::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> ConstructNamedPath::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<ConstructNamedPathCursor>(*this, db);
}
@ -1214,7 +1215,7 @@ Filter::Filter(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Filter)
std::unique_ptr<Cursor> Filter::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Filter::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<FilterCursor>(*this, db);
}
@ -1241,11 +1242,12 @@ Produce::Produce(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Produce)
std::unique_ptr<Cursor> Produce::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Produce::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<ProduceCursor>(*this, db);
}
std::vector<Symbol> Produce::OutputSymbols(const SymbolTable &symbol_table) {
std::vector<Symbol> Produce::OutputSymbols(
const SymbolTable &symbol_table) const {
std::vector<Symbol> symbols;
for (const auto &named_expr : named_expressions_) {
symbols.emplace_back(symbol_table.at(*named_expr));
@ -1280,7 +1282,7 @@ Delete::Delete(const std::shared_ptr<LogicalOperator> &input_,
ACCEPT_WITH_INPUT(Delete)
std::unique_ptr<Cursor> Delete::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Delete::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<DeleteCursor>(*this, db);
}
@ -1344,7 +1346,7 @@ SetProperty::SetProperty(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(SetProperty)
std::unique_ptr<Cursor> SetProperty::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> SetProperty::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<SetPropertyCursor>(*this, db);
}
@ -1392,7 +1394,7 @@ SetProperties::SetProperties(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(SetProperties)
std::unique_ptr<Cursor> SetProperties::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> SetProperties::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<SetPropertiesCursor>(*this, db);
}
@ -1470,7 +1472,7 @@ SetLabels::SetLabels(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(SetLabels)
std::unique_ptr<Cursor> SetLabels::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> SetLabels::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<SetLabelsCursor>(*this, db);
}
@ -1500,7 +1502,7 @@ RemoveProperty::RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(RemoveProperty)
std::unique_ptr<Cursor> RemoveProperty::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> RemoveProperty::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<RemovePropertyCursor>(*this, db);
}
@ -1543,7 +1545,7 @@ RemoveLabels::RemoveLabels(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(RemoveLabels)
std::unique_ptr<Cursor> RemoveLabels::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> RemoveLabels::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<RemoveLabelsCursor>(*this, db);
}
@ -1580,7 +1582,7 @@ ACCEPT_WITH_INPUT(ExpandUniquenessFilter<TAccessor>)
template <typename TAccessor>
std::unique_ptr<Cursor> ExpandUniquenessFilter<TAccessor>::MakeCursor(
GraphDbAccessor &db) {
GraphDbAccessor &db) const {
return std::make_unique<ExpandUniquenessFilterCursor>(*this, db);
}
@ -1697,7 +1699,7 @@ Accumulate::Accumulate(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Accumulate)
std::unique_ptr<Cursor> Accumulate::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Accumulate::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<Accumulate::AccumulateCursor>(*this, db);
}
@ -1749,11 +1751,11 @@ Aggregate::Aggregate(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Aggregate)
std::unique_ptr<Cursor> Aggregate::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Aggregate::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<AggregateCursor>(*this, db);
}
Aggregate::AggregateCursor::AggregateCursor(Aggregate &self,
Aggregate::AggregateCursor::AggregateCursor(const Aggregate &self,
GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
@ -2021,16 +2023,16 @@ Skip::Skip(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Skip)
std::unique_ptr<Cursor> Skip::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Skip::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<SkipCursor>(*this, db);
}
std::vector<Symbol> Skip::OutputSymbols(const SymbolTable &symbol_table) {
std::vector<Symbol> Skip::OutputSymbols(const SymbolTable &symbol_table) const {
// Propagate this to potential Produce.
return input_->OutputSymbols(symbol_table);
}
Skip::SkipCursor::SkipCursor(Skip &self, GraphDbAccessor &db)
Skip::SkipCursor::SkipCursor(const Skip &self, GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Skip::SkipCursor::Pull(Frame &frame, Context &context) {
@ -2068,16 +2070,17 @@ Limit::Limit(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Limit)
std::unique_ptr<Cursor> Limit::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Limit::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<LimitCursor>(*this, db);
}
std::vector<Symbol> Limit::OutputSymbols(const SymbolTable &symbol_table) {
std::vector<Symbol> Limit::OutputSymbols(
const SymbolTable &symbol_table) const {
// Propagate this to potential Produce.
return input_->OutputSymbols(symbol_table);
}
Limit::LimitCursor::LimitCursor(Limit &self, GraphDbAccessor &db)
Limit::LimitCursor::LimitCursor(const Limit &self, GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool Limit::LimitCursor::Pull(Frame &frame, Context &context) {
@ -2127,16 +2130,17 @@ OrderBy::OrderBy(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(OrderBy)
std::unique_ptr<Cursor> OrderBy::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> OrderBy::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<OrderByCursor>(*this, db);
}
std::vector<Symbol> OrderBy::OutputSymbols(const SymbolTable &symbol_table) {
std::vector<Symbol> OrderBy::OutputSymbols(
const SymbolTable &symbol_table) const {
// Propagate this to potential Produce.
return input_->OutputSymbols(symbol_table);
}
OrderBy::OrderByCursor::OrderByCursor(OrderBy &self, GraphDbAccessor &db)
OrderBy::OrderByCursor::OrderByCursor(const OrderBy &self, GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool OrderBy::OrderByCursor::Pull(Frame &frame, Context &context) {
@ -2271,11 +2275,11 @@ bool Merge::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.PostVisit(*this);
}
std::unique_ptr<Cursor> Merge::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Merge::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<MergeCursor>(*this, db);
}
Merge::MergeCursor::MergeCursor(Merge &self, GraphDbAccessor &db)
Merge::MergeCursor::MergeCursor(const Merge &self, GraphDbAccessor &db)
: input_cursor_(self.input_->MakeCursor(db)),
merge_match_cursor_(self.merge_match_->MakeCursor(db)),
merge_create_cursor_(self.merge_create_->MakeCursor(db)) {}
@ -2336,11 +2340,12 @@ bool Optional::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.PostVisit(*this);
}
std::unique_ptr<Cursor> Optional::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Optional::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<OptionalCursor>(*this, db);
}
Optional::OptionalCursor::OptionalCursor(Optional &self, GraphDbAccessor &db)
Optional::OptionalCursor::OptionalCursor(const Optional &self,
GraphDbAccessor &db)
: self_(self),
input_cursor_(self.input_->MakeCursor(db)),
optional_cursor_(self.optional_->MakeCursor(db)) {}
@ -2394,11 +2399,11 @@ Unwind::Unwind(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Unwind)
std::unique_ptr<Cursor> Unwind::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Unwind::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<UnwindCursor>(*this, db);
}
Unwind::UnwindCursor::UnwindCursor(Unwind &self, GraphDbAccessor &db)
Unwind::UnwindCursor::UnwindCursor(const Unwind &self, GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self.input_->MakeCursor(db)) {}
bool Unwind::UnwindCursor::Pull(Frame &frame, Context &context) {
@ -2439,16 +2444,18 @@ Distinct::Distinct(const std::shared_ptr<LogicalOperator> &input,
ACCEPT_WITH_INPUT(Distinct)
std::unique_ptr<Cursor> Distinct::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> Distinct::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<DistinctCursor>(*this, db);
}
std::vector<Symbol> Distinct::OutputSymbols(const SymbolTable &symbol_table) {
std::vector<Symbol> Distinct::OutputSymbols(
const SymbolTable &symbol_table) const {
// Propagate this to potential Produce.
return input_->OutputSymbols(symbol_table);
}
Distinct::DistinctCursor::DistinctCursor(Distinct &self, GraphDbAccessor &db)
Distinct::DistinctCursor::DistinctCursor(const Distinct &self,
GraphDbAccessor &db)
: self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
bool Distinct::DistinctCursor::Pull(Frame &frame, Context &context) {
@ -2478,7 +2485,7 @@ bool CreateIndex::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
class CreateIndexCursor : public Cursor {
public:
CreateIndexCursor(CreateIndex &self, GraphDbAccessor &db)
CreateIndexCursor(const CreateIndex &self, GraphDbAccessor &db)
: self_(self), db_(db) {}
bool Pull(Frame &, Context &) override {
@ -2506,7 +2513,7 @@ class CreateIndexCursor : public Cursor {
bool did_create_ = false;
};
std::unique_ptr<Cursor> CreateIndex::MakeCursor(GraphDbAccessor &db) {
std::unique_ptr<Cursor> CreateIndex::MakeCursor(GraphDbAccessor &db) const {
return std::make_unique<CreateIndexCursor>(*this, db);
}

View File

@ -130,7 +130,7 @@ class LogicalOperator
*
* @param GraphDbAccessor Used to perform operations on the database.
*/
virtual std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) = 0;
virtual std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const = 0;
/** @brief Return @c Symbol vector where the results will be stored.
*
@ -142,7 +142,7 @@ class LogicalOperator
* @param SymbolTable used to find symbols for expressions.
* @return std::vector<Symbol> used for results.
*/
virtual std::vector<Symbol> OutputSymbols(const SymbolTable &) {
virtual std::vector<Symbol> OutputSymbols(const SymbolTable &) const {
return std::vector<Symbol>();
}
@ -156,7 +156,7 @@ class LogicalOperator
class Once : public LogicalOperator {
public:
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
class OnceCursor : public Cursor {
@ -190,7 +190,7 @@ class CreateNode : public LogicalOperator {
CreateNode(const NodeAtom *node_atom,
const std::shared_ptr<LogicalOperator> &input);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const NodeAtom *node_atom_ = nullptr;
@ -245,7 +245,7 @@ class CreateExpand : public LogicalOperator {
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
// info on what's getting expanded
@ -314,7 +314,7 @@ class ScanAll : public LogicalOperator {
ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto output_symbol() const { return output_symbol_; }
@ -348,7 +348,7 @@ class ScanAllByLabel : public ScanAll {
Symbol output_symbol, GraphDbTypes::Label label,
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
GraphDbTypes::Label label() const { return label_; }
@ -390,7 +390,7 @@ class ScanAllByLabelPropertyRange : public ScanAll {
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }
@ -431,7 +431,7 @@ class ScanAllByLabelPropertyValue : public ScanAll {
GraphView graph_view = GraphView::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }
@ -553,7 +553,7 @@ class Expand : public LogicalOperator, public ExpandCommon {
using ExpandCommon::ExpandCommon;
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
class ExpandCursor : public Cursor {
public:
@ -628,7 +628,7 @@ class ExpandVariable : public LogicalOperator, public ExpandCommon {
Expression *filter = nullptr);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
// lower and upper bounds of the variable length expansion
@ -677,7 +677,7 @@ class ExpandBreadthFirst : public LogicalOperator {
GraphView graph_view = GraphView::AS_IS);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
class Cursor : public query::plan::Cursor {
@ -746,7 +746,7 @@ class ConstructNamedPath : public LogicalOperator {
path_symbol_(path_symbol),
path_elements_(path_elements) {}
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
const auto &input() const { return input_; }
const auto &path_symbol() const { return path_symbol_; }
@ -771,7 +771,7 @@ class Filter : public LogicalOperator {
Filter(const std::shared_ptr<LogicalOperator> &input_,
Expression *expression_);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -806,8 +806,8 @@ class Produce : public LogicalOperator {
Produce(const std::shared_ptr<LogicalOperator> &input,
const std::vector<NamedExpression *> named_expressions);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
const std::vector<NamedExpression *> &named_expressions();
private:
@ -838,7 +838,7 @@ class Delete : public LogicalOperator {
Delete(const std::shared_ptr<LogicalOperator> &input_,
const std::vector<Expression *> &expressions, bool detach_);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -872,7 +872,7 @@ class SetProperty : public LogicalOperator {
SetProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs, Expression *rhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -919,7 +919,7 @@ class SetProperties : public LogicalOperator {
SetProperties(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, Expression *rhs, Op op);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -959,7 +959,7 @@ class SetLabels : public LogicalOperator {
SetLabels(const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -987,7 +987,7 @@ class RemoveProperty : public LogicalOperator {
RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1018,7 +1018,7 @@ class RemoveLabels : public LogicalOperator {
Symbol input_symbol,
const std::vector<GraphDbTypes::Label> &labels);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1067,7 +1067,7 @@ class ExpandUniquenessFilter : public LogicalOperator {
Symbol expand_symbol,
const std::vector<Symbol> &previous_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1119,7 +1119,7 @@ class Accumulate : public LogicalOperator {
Accumulate(const std::shared_ptr<LogicalOperator> &input,
const std::vector<Symbol> &symbols, bool advance_command = false);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
const auto &symbols() const { return symbols_; };
@ -1185,7 +1185,7 @@ class Aggregate : public LogicalOperator {
const std::vector<Expression *> &group_by,
const std::vector<Symbol> &remember);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
const auto &aggregations() const { return aggregations_; }
const auto &group_by() const { return group_by_; }
@ -1198,7 +1198,7 @@ class Aggregate : public LogicalOperator {
class AggregateCursor : public Cursor {
public:
AggregateCursor(Aggregate &self, GraphDbAccessor &db);
AggregateCursor(const Aggregate &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1292,8 +1292,8 @@ class Skip : public LogicalOperator {
public:
Skip(const std::shared_ptr<LogicalOperator> &input, Expression *expression);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1301,7 +1301,7 @@ class Skip : public LogicalOperator {
class SkipCursor : public Cursor {
public:
SkipCursor(Skip &self, GraphDbAccessor &db);
SkipCursor(const Skip &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1335,8 +1335,8 @@ class Limit : public LogicalOperator {
public:
Limit(const std::shared_ptr<LogicalOperator> &input, Expression *expression);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1344,12 +1344,12 @@ class Limit : public LogicalOperator {
class LimitCursor : public Cursor {
public:
LimitCursor(Limit &self, GraphDbAccessor &db);
LimitCursor(const Limit &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
private:
Limit &self_;
const Limit &self_;
GraphDbAccessor &db_;
std::unique_ptr<Cursor> input_cursor_;
// init limit_ to -1, indicating
@ -1376,8 +1376,8 @@ class OrderBy : public LogicalOperator {
const std::vector<std::pair<Ordering, Expression *>> &order_by,
const std::vector<Symbol> &output_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
const auto &output_symbols() const { return output_symbols_; }
@ -1412,7 +1412,7 @@ class OrderBy : public LogicalOperator {
class OrderByCursor : public Cursor {
public:
OrderByCursor(OrderBy &self, GraphDbAccessor &db);
OrderByCursor(const OrderBy &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1451,7 +1451,7 @@ class Merge : public LogicalOperator {
const std::shared_ptr<LogicalOperator> merge_match,
const std::shared_ptr<LogicalOperator> merge_create);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto merge_match() const { return merge_match_; }
@ -1464,7 +1464,7 @@ class Merge : public LogicalOperator {
class MergeCursor : public Cursor {
public:
MergeCursor(Merge &self, GraphDbAccessor &db);
MergeCursor(const Merge &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1496,7 +1496,7 @@ class Optional : public LogicalOperator {
const std::shared_ptr<LogicalOperator> &optional,
const std::vector<Symbol> &optional_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto input() const { return input_; }
auto optional() const { return optional_; }
@ -1509,7 +1509,7 @@ class Optional : public LogicalOperator {
class OptionalCursor : public Cursor {
public:
OptionalCursor(Optional &self, GraphDbAccessor &db);
OptionalCursor(const Optional &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1537,7 +1537,7 @@ class Unwind : public LogicalOperator {
Unwind(const std::shared_ptr<LogicalOperator> &input,
Expression *input_expression_, Symbol output_symbol);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
Expression *input_expression() const { return input_expression_; }
@ -1548,7 +1548,7 @@ class Unwind : public LogicalOperator {
class UnwindCursor : public Cursor {
public:
UnwindCursor(Unwind &self, GraphDbAccessor &db);
UnwindCursor(const Unwind &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1576,8 +1576,8 @@ class Distinct : public LogicalOperator {
Distinct(const std::shared_ptr<LogicalOperator> &input,
const std::vector<Symbol> &value_symbols);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
private:
const std::shared_ptr<LogicalOperator> input_;
@ -1585,7 +1585,7 @@ class Distinct : public LogicalOperator {
class DistinctCursor : public Cursor {
public:
DistinctCursor(Distinct &self, GraphDbAccessor &db);
DistinctCursor(const Distinct &self, GraphDbAccessor &db);
bool Pull(Frame &, Context &) override;
void Reset() override;
@ -1615,7 +1615,7 @@ class CreateIndex : public LogicalOperator {
public:
CreateIndex(GraphDbTypes::Label label, GraphDbTypes::Property property);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) const override;
auto label() const { return label_; }
auto property() const { return property_; }

View File

@ -8,7 +8,7 @@ namespace utils {
class Timer {
public:
/** Time elapsed since creation. */
std::chrono::duration<double> Elapsed() {
std::chrono::duration<double> Elapsed() const {
return std::chrono::steady_clock::now() - start_time_;
}