diff --git a/src/query/frontend/stripped.hpp b/src/query/frontend/stripped.hpp
index 3e8482d8b..fc1f8d644 100644
--- a/src/query/frontend/stripped.hpp
+++ b/src/query/frontend/stripped.hpp
@@ -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_; }
diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp
index 5be604d14..533677f24 100644
--- a/src/query/interpreter.cpp
+++ b/src/query/interpreter.cpp
@@ -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
diff --git a/src/query/interpreter.hpp b/src/query/interpreter.hpp
index d78e2c677..e38f76962 100644
--- a/src/query/interpreter.hpp
+++ b/src/query/interpreter.hpp
@@ -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
diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp
index e6648be9e..d549631d3 100644
--- a/src/query/plan/operator.cpp
+++ b/src/query/plan/operator.cpp
@@ -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);
 }
 
diff --git a/src/query/plan/operator.hpp b/src/query/plan/operator.hpp
index 53ea1ef3f..cef02651e 100644
--- a/src/query/plan/operator.hpp
+++ b/src/query/plan/operator.hpp
@@ -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_; }
diff --git a/src/utils/timer.hpp b/src/utils/timer.hpp
index 6f7693320..f653c5236 100644
--- a/src/utils/timer.hpp
+++ b/src/utils/timer.hpp
@@ -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_;
   }