From b90375c3ae174230b99196fcf4963e3e60d5e4c5 Mon Sep 17 00:00:00 2001
From: Teon Banek <teon.banek@memgraph.io>
Date: Mon, 14 Jan 2019 14:41:37 +0100
Subject: [PATCH] Remove GraphDbAccessor and storage types from Ast

Summary:
This diff removes the need for a database when parsing a query and
creating an Ast. Instead of storing storage::{Label,Property,EdgeType}
in Ast nodes, we store the name and an index into all of the names. This
allows for easy creation of a map from {Label,Property,EdgeType} index
into the concrete storage type. Obviously, this comes with a performance
penalty during execution, but it should be minor. The upside is that the
query/frontend minimally depends on storage (PropertyValue), which makes
writing tests easier as well as running them a lot faster (there is no
database setup). This is most noticeable in the ast_serialization test
which took a long time due to start up of a distributed database.

Reviewers: mtomic, llugovic

Reviewed By: mtomic

Subscribers: mferencevic, pullbot

Differential Revision: https://phabricator.memgraph.io/D1774
---
 src/distributed/bfs_rpc_clients.cpp           |   3 +-
 src/distributed/bfs_rpc_messages.lcp          |  13 +-
 src/distributed/bfs_rpc_server.hpp            |  10 +-
 src/distributed/produce_rpc_server.cpp        |  23 +-
 src/distributed/produce_rpc_server.hpp        |   5 +-
 src/distributed/pull_produce_rpc_messages.lcp |  13 +-
 src/distributed/pull_rpc_clients.cpp          |  31 +-
 src/query/context.hpp                         |  30 +-
 src/query/distributed_interpreter.cpp         |  23 +-
 src/query/frontend/ast/ast.lcp                | 403 ++++++++++---
 .../frontend/ast/cypher_main_visitor.cpp      |  87 ++-
 .../frontend/ast/cypher_main_visitor.hpp      |  19 +-
 src/query/frontend/ast/pretty_print.cpp       | 159 +++--
 src/query/frontend/ast/pretty_print.hpp       |   6 +-
 src/query/interpret/eval.hpp                  |  18 +-
 src/query/interpreter.cpp                     | 151 +++--
 src/query/interpreter.hpp                     |  32 +-
 src/query/plan/cost_estimator.hpp             |  24 +-
 src/query/plan/distributed.cpp                |  24 +-
 src/query/plan/distributed.hpp                |   7 +-
 src/query/plan/operator.cpp                   |  26 +-
 src/query/plan/operator.lcp                   |  11 +-
 src/query/plan/planner.hpp                    |  23 +-
 src/query/plan/preprocess.cpp                 |   8 +-
 src/query/plan/preprocess.hpp                 |  13 +-
 src/query/plan/rule_based_planner.cpp         | 156 -----
 src/query/plan/rule_based_planner.hpp         | 309 ++++++++--
 src/query/plan/variable_start_planner.hpp     |  10 +-
 src/query/plan/vertex_count_cache.hpp         |  46 +-
 src/query/serialization.capnp                 |   5 -
 src/query/serialization.cpp                   |  35 +-
 src/query/serialization.hpp                   |  23 +-
 tests/benchmark/query/planner.cpp             |  53 +-
 tests/manual/distributed_query_planner.cpp    |   5 +-
 tests/manual/interactive_planning.cpp         |  78 +--
 tests/manual/interactive_planning.hpp         |  11 +-
 tests/unit/ast_serialization.cpp              | 179 +++---
 tests/unit/bfs_common.hpp                     |   4 +
 tests/unit/cypher_main_visitor.cpp            | 183 +++---
 tests/unit/distributed_query_plan.cpp         | 570 ++++++++++++------
 tests/unit/query_common.hpp                   |  74 ++-
 tests/unit/query_cost_estimator.cpp           |  15 +-
 tests/unit/query_expression_evaluator.cpp     | 254 ++++----
 tests/unit/query_plan.cpp                     | 180 +++---
 .../unit/query_plan_accumulate_aggregate.cpp  |  34 +-
 tests/unit/query_plan_bag_semantics.cpp       |  34 +-
 tests/unit/query_plan_checker.hpp             |   4 +-
 tests/unit/query_plan_common.hpp              |  47 +-
 .../query_plan_create_set_remove_delete.cpp   | 154 +++--
 tests/unit/query_plan_match_filter_return.cpp | 195 +++---
 tests/unit/query_pretty_print.cpp             |  98 +--
 tests/unit/query_required_privileges.cpp      |  20 +-
 tests/unit/query_semantic.cpp                 |  50 +-
 tests/unit/query_variable_start_planner.cpp   |  39 +-
 54 files changed, 2339 insertions(+), 1688 deletions(-)

diff --git a/src/distributed/bfs_rpc_clients.cpp b/src/distributed/bfs_rpc_clients.cpp
index e267f0717..451dd401c 100644
--- a/src/distributed/bfs_rpc_clients.cpp
+++ b/src/distributed/bfs_rpc_clients.cpp
@@ -25,7 +25,8 @@ std::unordered_map<int16_t, int64_t> BfsRpcClients::CreateBfsSubcursors(
       db_->WorkerId(), [&](int worker_id, auto &client) {
         auto res = client.template Call<CreateBfsSubcursorRpc>(
             dba->transaction_id(), direction, edge_types, filter_lambda,
-            symbol_table, evaluation_context);
+            symbol_table, evaluation_context.timestamp,
+            evaluation_context.parameters);
         return std::make_pair(worker_id, res.member);
       });
   std::unordered_map<int16_t, int64_t> subcursor_ids;
diff --git a/src/distributed/bfs_rpc_messages.lcp b/src/distributed/bfs_rpc_messages.lcp
index d0275afac..539aba1d1 100644
--- a/src/distributed/bfs_rpc_messages.lcp
+++ b/src/distributed/bfs_rpc_messages.lcp
@@ -67,16 +67,9 @@ cpp<#
                                     Load(&${member}, ${reader}, ast_storage, &loaded_ast_uids);
                                     cpp<#))
       (symbol-table "query::SymbolTable" :capnp-type "Query.SymbolTable")
-      (evaluation-context "query::EvaluationContext"
-                          :capnp-type "Query.EvaluationContext"
-                          :capnp-save (lambda (builder member capnp-name)
-                                        #>cpp
-                                        query::SaveEvaluationContext(${member}, &${builder});
-                                        cpp<#)
-                          :capnp-load (lambda (reader member capnp-name)
-                                        #>cpp
-                                        query::LoadEvaluationContext(${reader}, &${member});
-                                        cpp<#)))
+      (timestamp :int64_t)
+      (parameters "query::Parameters"
+                  :capnp-type "Utils.Map(Utils.BoxInt64, Storage.PropertyValue)"))
      (:serialize (:slk :load-args '((ast-storage "query::AstStorage *")))
                  (:capnp :load-args '((ast-storage "query::AstStorage *")))))
   (:response ((member :int64_t))))
diff --git a/src/distributed/bfs_rpc_server.hpp b/src/distributed/bfs_rpc_server.hpp
index 5fb6e77dd..eb85ca2f2 100644
--- a/src/distributed/bfs_rpc_server.hpp
+++ b/src/distributed/bfs_rpc_server.hpp
@@ -35,10 +35,16 @@ class BfsRpcServer {
         }
         dba = it->second.get();
       }
+      query::EvaluationContext evaluation_context;
+      evaluation_context.timestamp = req.timestamp;
+      evaluation_context.parameters = req.parameters;
+      evaluation_context.properties =
+          query::NamesToProperties(ast_storage->properties_, dba);
+      evaluation_context.labels =
+          query::NamesToLabels(ast_storage->labels_, dba);
       auto id = subcursor_storage_->Create(
           dba, req.direction, req.edge_types, std::move(req.symbol_table),
-          std::move(ast_storage), req.filter_lambda,
-          std::move(req.evaluation_context));
+          std::move(ast_storage), req.filter_lambda, evaluation_context);
       CreateBfsSubcursorRes res(id);
       Save(res, res_builder);
     });
diff --git a/src/distributed/produce_rpc_server.cpp b/src/distributed/produce_rpc_server.cpp
index 24ea5487a..4fba05cd0 100644
--- a/src/distributed/produce_rpc_server.cpp
+++ b/src/distributed/produce_rpc_server.cpp
@@ -11,17 +11,21 @@ namespace distributed {
 
 ProduceRpcServer::OngoingProduce::OngoingProduce(
     database::Worker *db, tx::TransactionId tx_id,
-    std::shared_ptr<query::plan::LogicalOperator> op,
-    query::SymbolTable symbol_table,
-    query::EvaluationContext evaluation_context,
+    const PlanConsumer::PlanPack &plan_pack, int64_t timestamp,
+    const query::Parameters &parameters,
     std::vector<query::Symbol> pull_symbols)
     : dba_(db->Access(tx_id)),
       context_(*dba_),
       pull_symbols_(std::move(pull_symbols)),
-      frame_(symbol_table.max_position()),
-      cursor_(op->MakeCursor(*dba_)) {
-  context_.symbol_table_ = std::move(symbol_table);
-  context_.evaluation_context_ = std::move(evaluation_context);
+      frame_(plan_pack.symbol_table.max_position()),
+      cursor_(plan_pack.plan->MakeCursor(*dba_)) {
+  context_.symbol_table_ = plan_pack.symbol_table;
+  context_.evaluation_context_.timestamp = timestamp;
+  context_.evaluation_context_.parameters = parameters;
+  context_.evaluation_context_.properties =
+      query::NamesToProperties(plan_pack.storage.properties_, dba_.get());
+  context_.evaluation_context_.labels =
+      query::NamesToLabels(plan_pack.storage.labels_, dba_.get());
 }
 
 std::pair<std::vector<query::TypedValue>, PullState>
@@ -161,9 +165,8 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce(
   auto &plan_pack = plan_consumer_.PlanForId(req.plan_id);
   return ongoing_produces_
       .emplace(std::piecewise_construct, std::forward_as_tuple(key_tuple),
-               std::forward_as_tuple(db_, req.tx_id, plan_pack.plan,
-                                     plan_pack.symbol_table,
-                                     req.evaluation_context, req.symbols))
+               std::forward_as_tuple(db_, req.tx_id, plan_pack, req.timestamp,
+                                     req.parameters, req.symbols))
       .first->second;
 }
 
diff --git a/src/distributed/produce_rpc_server.hpp b/src/distributed/produce_rpc_server.hpp
index db50f0c40..208602777 100644
--- a/src/distributed/produce_rpc_server.hpp
+++ b/src/distributed/produce_rpc_server.hpp
@@ -43,9 +43,8 @@ class ProduceRpcServer {
   class OngoingProduce {
    public:
     OngoingProduce(database::Worker *db, tx::TransactionId tx_id,
-                   std::shared_ptr<query::plan::LogicalOperator> op,
-                   query::SymbolTable symbol_table,
-                   query::EvaluationContext evaluation_context,
+                   const PlanConsumer::PlanPack &plan_pack, int64_t timestamp,
+                   const query::Parameters &parameters,
                    std::vector<query::Symbol> pull_symbols);
 
     /// Returns a vector of typed values (one for each `pull_symbol`), and an
diff --git a/src/distributed/pull_produce_rpc_messages.lcp b/src/distributed/pull_produce_rpc_messages.lcp
index 5f343327a..a71961658 100644
--- a/src/distributed/pull_produce_rpc_messages.lcp
+++ b/src/distributed/pull_produce_rpc_messages.lcp
@@ -222,16 +222,9 @@ to the appropriate value. Not used on side that generates the response.")
                    :capnp-load #'load-snapshot)
       (plan-id :int64_t)
       (command-id "tx::CommandId")
-      (evaluation-context "query::EvaluationContext"
-                          :capnp-type "Query.EvaluationContext"
-                          :capnp-save (lambda (builder member capnp-name)
-                                        #>cpp
-                                        query::SaveEvaluationContext(${member}, &${builder});
-                                        cpp<#)
-                          :capnp-load (lambda (reader member capnp-name)
-                                        #>cpp
-                                        query::LoadEvaluationContext(${reader}, &${member});
-                                        cpp<#))
+      (timestamp :int64_t)
+      (parameters "query::Parameters"
+                  :capnp-type "Utils.Map(Utils.BoxInt64, Storage.PropertyValue)")
       (symbols "std::vector<query::Symbol>" :capnp-type "List(Sem.Symbol)")
       (accumulate :bool)
       (batch-size :int64_t)
diff --git a/src/distributed/pull_rpc_clients.cpp b/src/distributed/pull_rpc_clients.cpp
index 1bf9a5f1f..fba7837f8 100644
--- a/src/distributed/pull_rpc_clients.cpp
+++ b/src/distributed/pull_rpc_clients.cpp
@@ -13,21 +13,22 @@ utils::Future<PullData> PullRpcClients::Pull(
     const query::EvaluationContext &evaluation_context,
     const std::vector<query::Symbol> &symbols, bool accumulate,
     int batch_size) {
-  return coordination_->ExecuteOnWorker<PullData>(worker_id, [
-    data_manager = data_manager_, dba, plan_id, command_id, evaluation_context,
-    symbols, accumulate, batch_size
-  ](int worker_id, ClientPool &client_pool) {
-    auto load_pull_res = [data_manager, dba](const auto &res_reader) {
-      PullRes res;
-      Load(&res, res_reader, dba, data_manager);
-      return res;
-    };
-    auto result = client_pool.CallWithLoad<PullRpc>(
-        load_pull_res, dba->transaction_id(), dba->transaction().snapshot(),
-        plan_id, command_id, evaluation_context, symbols, accumulate,
-        batch_size, storage::SendVersions::BOTH);
-    return PullData{result.data.pull_state, std::move(result.data.frames)};
-  });
+  return coordination_->ExecuteOnWorker<PullData>(
+      worker_id, [data_manager = data_manager_, dba, plan_id, command_id,
+                  evaluation_context, symbols, accumulate,
+                  batch_size](int worker_id, ClientPool &client_pool) {
+        auto load_pull_res = [data_manager, dba](const auto &res_reader) {
+          PullRes res;
+          Load(&res, res_reader, dba, data_manager);
+          return res;
+        };
+        auto result = client_pool.CallWithLoad<PullRpc>(
+            load_pull_res, dba->transaction_id(), dba->transaction().snapshot(),
+            plan_id, command_id, evaluation_context.timestamp,
+            evaluation_context.parameters, symbols, accumulate, batch_size,
+            storage::SendVersions::BOTH);
+        return PullData{result.data.pull_state, std::move(result.data.frames)};
+      });
 }
 
 utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba,
diff --git a/src/query/context.hpp b/src/query/context.hpp
index 34eb53260..b545e64c9 100644
--- a/src/query/context.hpp
+++ b/src/query/context.hpp
@@ -1,6 +1,5 @@
 #pragma once
 
-#include "antlr4-runtime.h"
 #include "database/graph_db_accessor.hpp"
 #include "query/frontend/semantic/symbol_table.hpp"
 #include "query/parameters.hpp"
@@ -11,8 +10,34 @@ namespace query {
 struct EvaluationContext {
   int64_t timestamp{-1};
   Parameters parameters;
+  /// All properties indexable via PropertyIx
+  std::vector<storage::Property> properties;
+  /// All labels indexable via LabelIx
+  std::vector<storage::Label> labels;
 };
 
+inline std::vector<storage::Property> NamesToProperties(
+    const std::vector<std::string> &property_names,
+    database::GraphDbAccessor *dba) {
+  std::vector<storage::Property> properties;
+  properties.reserve(property_names.size());
+  for (const auto &name : property_names) {
+    properties.push_back(dba->Property(name));
+  }
+  return properties;
+}
+
+inline std::vector<storage::Label> NamesToLabels(
+    const std::vector<std::string> &label_names,
+    database::GraphDbAccessor *dba) {
+  std::vector<storage::Label> labels;
+  labels.reserve(label_names.size());
+  for (const auto &name : label_names) {
+    labels.push_back(dba->Label(name));
+  }
+  return labels;
+}
+
 class Context {
  public:
   Context(const Context &) = delete;
@@ -31,6 +56,9 @@ class Context {
   plan::ProfilingStats *stats_root_{nullptr};
 };
 
+// TODO: Move this to somewhere in query/frontend. Currently, frontend includes
+// this and therefore implicitly includes the whole database because of the
+// includes at the top of this file.
 struct ParsingContext {
   bool is_query_cached = false;
 };
diff --git a/src/query/distributed_interpreter.cpp b/src/query/distributed_interpreter.cpp
index 8fd795d2f..c6cdec8d7 100644
--- a/src/query/distributed_interpreter.cpp
+++ b/src/query/distributed_interpreter.cpp
@@ -46,6 +46,9 @@ class DistributedLogicalPlan final : public LogicalPlan {
   const SymbolTable &GetSymbolTable() const override {
     return plan_.symbol_table;
   }
+  const AstStorage &GetAstStorage() const override {
+    return plan_.ast_storage;
+  }
 
  private:
   plan::DistributedPlan plan_;
@@ -61,19 +64,21 @@ DistributedInterpreter::DistributedInterpreter(database::Master *db)
 std::unique_ptr<LogicalPlan> DistributedInterpreter::MakeLogicalPlan(
     CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
     database::GraphDbAccessor *db_accessor) {
-  auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
+  auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
 
   auto symbol_table = MakeSymbolTable(query);
 
-  auto planning_context = plan::MakePlanningContext(ast_storage, symbol_table,
-                                                    query, vertex_counts);
+  auto planning_context = plan::MakePlanningContext(&ast_storage, &symbol_table,
+                                                    query, &vertex_counts);
 
   std::unique_ptr<plan::LogicalOperator> tmp_logical_plan;
   double cost;
   std::tie(tmp_logical_plan, cost) = plan::MakeLogicalPlan(
-      planning_context, parameters, FLAGS_query_cost_planner);
-  auto plan =
-      MakeDistributedPlan(*tmp_logical_plan, symbol_table, next_plan_id_);
+      &planning_context, parameters, FLAGS_query_cost_planner);
+  std::vector<storage::Property> properties_by_ix =
+      NamesToProperties(ast_storage.properties_, db_accessor);
+  auto plan = MakeDistributedPlan(ast_storage, *tmp_logical_plan, symbol_table,
+                                  next_plan_id_, properties_by_ix);
   VLOG(10) << "[Interpreter] Created plan for distributed execution "
            << next_plan_id_ - 1;
   return std::make_unique<DistributedLogicalPlan>(std::move(plan), cost,
@@ -85,10 +90,10 @@ Interpreter::Results DistributedInterpreter::operator()(
     const std::map<std::string, PropertyValue> &params,
     bool in_explicit_transaction) {
   AstStorage ast_storage;
-  Context execution_context(db_accessor);
+  Parameters parameters;
 
-  auto queries = StripAndParseQuery(query_string, &execution_context,
-                                    &ast_storage, &db_accessor, params);
+  auto queries = StripAndParseQuery(query_string, &parameters, &ast_storage,
+                                    &db_accessor, params);
   ParsedQuery &parsed_query = queries.second;
 
   if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {
diff --git a/src/query/frontend/ast/ast.lcp b/src/query/frontend/ast/ast.lcp
index 1edb586ca..aeb9226de 100644
--- a/src/query/frontend/ast/ast.lcp
+++ b/src/query/frontend/ast/ast.lcp
@@ -10,24 +10,8 @@
 #include "query/interpret/awesome_memgraph_functions.hpp"
 #include "query/typed_value.hpp"
 #include "storage/common/types/property_value.hpp"
-#include "storage/common/types/types.hpp"
 #include "utils/typeinfo.hpp"
 
-// Hash function for the key in pattern atom property maps.
-namespace std {
-template <>
-struct hash<std::pair<std::string, storage::Property>> {
-  size_t operator()(
-      const std::pair<std::string, storage::Property> &pair) const {
-    return string_hash(pair.first) ^ property_hash(pair.second);
-  };
-
- private:
-  std::hash<std::string> string_hash{};
-  std::hash<storage::Property> property_hash{};
-};
-}  // namespace std
-
 namespace database {
 class GraphDbAccessor;
 }
@@ -42,9 +26,6 @@ cpp<#
 (lcp:capnp-import 'utils "/utils/serialization.capnp")
 
 (lcp:capnp-type-conversion "PropertyValue" "Storage.PropertyValue")
-(lcp:capnp-type-conversion "storage::Label" "Storage.Label")
-(lcp:capnp-type-conversion "storage::EdgeType" "Storage.EdgeType")
-(lcp:capnp-type-conversion "storage::Property" "Storage.Property")
 (lcp:capnp-type-conversion "Symbol" "Symbol.Symbol")
 
 (defun slk-save-ast-pointer (member)
@@ -131,8 +112,8 @@ cpp<#
   for (size_t i = 0;
        i < size;
        ++i) {
-    std::pair<std::string, storage::Property> key;
-    slk::Load(&key, reader);
+    query::PropertyIx key;
+    slk::Load(&key, reader, storage);
     auto *value = query::LoadAstPointer<query::Expression>(storage, reader, loaded_uids);
     self->${member}.emplace(key, value);
   }
@@ -144,9 +125,7 @@ cpp<#
   size_t i = 0;
   for (const auto &entry : ${member}) {
     auto key_builder = entries_builder[i].initKey();
-    key_builder.setFirst(entry.first.first);
-    auto prop_id_builder = key_builder.initSecond();
-    storage::Save(entry.first.second, &prop_id_builder);
+    Save(entry.first, &key_builder);
     auto value_builder = entries_builder[i].initValue();
     Save(*entry.second, &value_builder, saved_uids);
     ++i;
@@ -156,14 +135,119 @@ cpp<#
 (defun load-property-map (reader member capnp-name)
   #>cpp
   for (const auto &entry : ${reader}.getEntries()) {
-    std::string prop_name = entry.getKey().getFirst();
-    storage::Property prop_id;
-    storage::Load(&prop_id, entry.getKey().getSecond());
-    ${member}.emplace(std::make_pair(prop_name, prop_id),
-                      static_cast<Expression *>(Load(storage, entry.getValue(), loaded_uids)));
+    PropertyIx prop;
+    Load(&prop, entry.getKey(), storage);
+    ${member}.emplace(prop, static_cast<Expression *>(
+                                Load(storage, entry.getValue(), loaded_uids)));
   }
   cpp<#)
 
+(defun capnp-load-name-ix (name-type)
+  (lambda (reader member capnp-name)
+    (declare (ignore capnp-name))
+    #>cpp
+    ${member} = storage->Get${name-type}Ix(self->name).ix;
+    cpp<#))
+
+(defun slk-load-name-ix (name-type)
+  (lambda (member)
+    #>cpp
+    self->${member} = storage->Get${name-type}Ix(self->name).ix;
+    cpp<#))
+
+;; The following index structs serve as a decoupling point of AST from
+;; concrete database types. All the names are collected in AstStorage, and can
+;; be indexed through these instances. This means that we can create a vector
+;; of concrete database types in the same order as all of the names and use the
+;; same index to get the correct behaviour. Additionally, each index is
+;; accompanied with the duplicated name found at the same index. The primary
+;; reason for this duplication is simplifying the Clone and serialization API.
+;; When an old index is being cloned or deserialized into a new AstStorage, we
+;; request the new `ix` from the new AstStorage for the same `name`. If we
+;; didn't do this, we would have to duplicate the old storage, which would
+;; require having access to that storage. This in turn would complicate the
+;; client code.
+(lcp:define-struct label-ix ()
+  ((name "std::string")
+   (ix :int64_t
+       :dont-save t
+       :slk-load (slk-load-name-ix "Label")
+       :capnp-init nil
+       :capnp-load (capnp-load-name-ix "Label")))
+  (:serialize (:slk :load-args '((storage "query::AstStorage *")))
+              (:capnp :load-args '((storage "AstStorage *")))))
+
+(lcp:define-struct property-ix ()
+  ((name "std::string")
+   (ix :int64_t
+       :dont-save t
+       :slk-load (slk-load-name-ix "Property")
+       :capnp-init nil
+       :capnp-load (capnp-load-name-ix "Property")))
+  (:serialize (:slk :load-args '((storage "query::AstStorage *")))
+              (:capnp :load-args '((storage "AstStorage *")))))
+
+(lcp:define-struct edge-type-ix ()
+  ((name "std::string")
+   (ix :int64_t
+       :dont-save t
+       :slk-load (slk-load-name-ix "EdgeType")
+       :capnp-init nil
+       :capnp-load (capnp-load-name-ix "EdgeType")))
+  (:serialize (:slk :load-args '((storage "query::AstStorage *")))
+              (:capnp :load-args '((storage "AstStorage *")))))
+
+#>cpp
+inline bool operator==(const LabelIx &a, const LabelIx &b) {
+  return a.ix == b.ix && a.name == b.name;
+}
+
+inline bool operator!=(const LabelIx &a, const LabelIx &b) { return !(a == b); }
+
+inline bool operator==(const PropertyIx &a, const PropertyIx &b) {
+  return a.ix == b.ix && a.name == b.name;
+}
+
+inline bool operator!=(const PropertyIx &a, const PropertyIx &b) {
+  return !(a == b);
+}
+
+inline bool operator==(const EdgeTypeIx &a, const EdgeTypeIx &b) {
+  return a.ix == b.ix && a.name == b.name;
+}
+
+inline bool operator!=(const EdgeTypeIx &a, const EdgeTypeIx &b) {
+  return !(a == b);
+}
+cpp<#
+
+(lcp:pop-namespace) ;; namespace query
+
+#>cpp
+namespace std {
+
+template <>
+struct hash<query::LabelIx> {
+  size_t operator()(const query::LabelIx &label) const { return label.ix; }
+};
+
+template <>
+struct hash<query::PropertyIx> {
+  size_t operator()(const query::PropertyIx &prop) const { return prop.ix; }
+};
+
+template <>
+struct hash<query::EdgeTypeIx> {
+  size_t operator()(const query::EdgeTypeIx &edge_type) const {
+    return edge_type.ix;
+  }
+};
+
+}  // namespace std
+cpp<#
+
+(lcp:namespace query)
+
 #>cpp
 #define CLONE_BINARY_EXPRESSION                                              \
   auto Clone(AstStorage &storage) const->std::remove_const<                  \
@@ -202,9 +286,37 @@ class AstStorage {
     return ptr;
   }
 
+  LabelIx GetLabelIx(const std::string &name) {
+    return LabelIx{name, FindOrAddName(name, &labels_)};
+  }
+
+  PropertyIx GetPropertyIx(const std::string &name) {
+    return PropertyIx{name, FindOrAddName(name, &properties_)};
+  }
+
+  EdgeTypeIx GetEdgeTypeIx(const std::string &name) {
+    return EdgeTypeIx{name, FindOrAddName(name, &edge_types_)};
+  }
+
+  std::vector<std::string> labels_;
+  std::vector<std::string> edge_types_;
+  std::vector<std::string> properties_;
+
   // Public only for serialization access
   std::vector<std::unique_ptr<Tree>> storage_;
   int max_existing_uid_ = -1;
+
+ private:
+  int64_t FindOrAddName(const std::string &name,
+                        std::vector<std::string> *names) {
+    for (int64_t i = 0; i < names->size(); ++i) {
+      if ((*names)[i] == name) {
+        return i;
+      }
+    }
+    names->push_back(name);
+    return names->size() - 1;
+  }
 };
 cpp<#
 
@@ -713,10 +825,10 @@ cpp<#
   (:serialize (:slk) (:capnp)))
 
 (lcp:define-class map-literal (base-literal)
-  ((elements "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
+  ((elements "std::unordered_map<PropertyIx, Expression *>"
              :slk-save #'slk-save-property-map
              :slk-load #'slk-load-property-map
-             :capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
+             :capnp-type "Utils.Map(PropertyIx, Tree)"
              :capnp-save #'save-property-map
              :capnp-load #'load-property-map
              :scope :public))
@@ -736,8 +848,10 @@ cpp<#
 
     MapLiteral *Clone(AstStorage &storage) const override {
       auto *map = storage.Create<MapLiteral>();
-      for (auto pair : elements_)
-        map->elements_.emplace(pair.first, pair.second->Clone(storage));
+      for (auto pair : elements_) {
+        auto prop = storage.GetPropertyIx(pair.first.name);
+        map->elements_.emplace(prop, pair.second->Clone(storage));
+      }
       return map;
     }
     cpp<#)
@@ -745,8 +859,7 @@ cpp<#
     #>cpp
     explicit MapLiteral(int uid) : BaseLiteral(uid) {}
     MapLiteral(int uid,
-               const std::unordered_map<std::pair<std::string, storage::Property>,
-                                        Expression *> &elements)
+               const std::unordered_map<PropertyIx, Expression *> &elements)
         : BaseLiteral(uid), elements_(elements) {}
     cpp<#)
   (:private
@@ -789,8 +902,11 @@ cpp<#
                :capnp-type "Tree" :capnp-init nil
                :capnp-save #'save-ast-pointer
                :capnp-load (load-ast-pointer "Expression *"))
-   (property-name "std::string" :scope :public)
-   (property "storage::Property" :scope :public))
+   (property "PropertyIx" :scope :public
+             :slk-load (lambda (member)
+                        #>cpp
+                        slk::Load(&self->${member}, reader, storage);
+                        cpp<#)))
   (:public
     #>cpp
     PropertyLookup() = default;
@@ -806,23 +922,15 @@ cpp<#
 
     PropertyLookup *Clone(AstStorage &storage) const override {
       return storage.Create<PropertyLookup>(expression_->Clone(storage),
-                                            property_name_, property_);
+                                            storage.GetPropertyIx(property_.name));
     }
     cpp<#)
   (:protected
     #>cpp
-    PropertyLookup(int uid, Expression *expression,
-                   const std::string &property_name, storage::Property property)
+    PropertyLookup(int uid, Expression *expression, PropertyIx property)
         : Expression(uid),
           expression_(expression),
-          property_name_(property_name),
           property_(property) {}
-    PropertyLookup(int uid, Expression *expression,
-                   const std::pair<std::string, storage::Property> &property)
-        : Expression(uid),
-          expression_(expression),
-          property_name_(property.first),
-          property_(property.second) {}
     cpp<#)
   (:private
     #>cpp
@@ -837,13 +945,23 @@ cpp<#
                :capnp-type "Tree" :capnp-init nil
                :capnp-save #'save-ast-pointer
                :capnp-load (load-ast-pointer "Expression *"))
-   (labels "std::vector<storage::Label>" :scope :public
-           :capnp-save (lcp:capnp-save-vector
-                         "storage::capnp::Label"
-                         "storage::Label")
+   (labels "std::vector<LabelIx>" :scope :public
+           :slk-load (lambda (member)
+                      #>cpp
+                      size_t size = 0;
+                      slk::Load(&size, reader);
+                      self->${member}.resize(size);
+                      for (size_t i = 0; i < size; ++i) {
+                        slk::Load(&self->${member}[i], reader, storage);
+                      }
+                      cpp<#)
            :capnp-load (lcp:capnp-load-vector
-                         "storage::capnp::Label"
-                         "storage::Label")))
+                        "capnp::LabelIx" "LabelIx"
+                        "[storage](const auto &reader) {
+                           LabelIx ix;
+                           Load(&ix, reader, storage);
+                           return ix;
+                         }")))
   (:public
     #>cpp
     LabelsTest() = default;
@@ -858,13 +976,18 @@ cpp<#
     }
 
     LabelsTest *Clone(AstStorage &storage) const override {
-      return storage.Create<LabelsTest>(expression_->Clone(storage), labels_);
+      std::vector<LabelIx> new_labels;
+      new_labels.reserve(labels_.size());
+      for (const auto &label : labels_) {
+        new_labels.push_back(storage.GetLabelIx(label.name));
+      }
+      return storage.Create<LabelsTest>(expression_->Clone(storage), new_labels);
     }
     cpp<#)
   (:protected
     #>cpp
     LabelsTest(int uid, Expression *expression,
-               const std::vector<storage::Label> &labels)
+               const std::vector<LabelIx> &labels)
         : Expression(uid), expression_(expression), labels_(labels) {}
     cpp<#)
   (:private
@@ -1337,17 +1460,27 @@ cpp<#
               (:capnp :ignore-other-base-classes t)))
 
 (lcp:define-class node-atom (pattern-atom)
-  ((labels "std::vector<storage::Label>" :scope :public
-           :capnp-save (lcp:capnp-save-vector
-                         "storage::capnp::Label"
-                         "storage::Label")
+  ((labels "std::vector<LabelIx>" :scope :public
+           :slk-load (lambda (member)
+                      #>cpp
+                      size_t size = 0;
+                      slk::Load(&size, reader);
+                      self->${member}.resize(size);
+                      for (size_t i = 0; i < size; ++i) {
+                        slk::Load(&self->${member}[i], reader, storage);
+                      }
+                      cpp<#)
            :capnp-load (lcp:capnp-load-vector
-                         "storage::capnp::Label"
-                         "storage::Label"))
-   (properties "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
+                        "capnp::LabelIx" "LabelIx"
+                        "[storage](const auto &reader) {
+                           LabelIx ix;
+                           Load(&ix, reader, storage);
+                           return ix;
+                         }"))
+   (properties "std::unordered_map<PropertyIx, Expression *>"
                :slk-save #'slk-save-property-map
                :slk-load #'slk-load-property-map
-               :capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
+               :capnp-type "Utils.Map(PropertyIx, Tree)"
                :capnp-save #'save-property-map
                :capnp-load #'load-property-map
                :scope :public))
@@ -1367,9 +1500,13 @@ cpp<#
 
     NodeAtom *Clone(AstStorage &storage) const override {
       auto *node_atom = storage.Create<NodeAtom>(identifier_->Clone(storage));
-      node_atom->labels_ = labels_;
+      node_atom->labels_.reserve(labels_.size());
+      for (const auto &label : labels_) {
+        node_atom->labels_.push_back(storage.GetLabelIx(label.name));
+      }
       for (auto property : properties_) {
-        node_atom->properties_[property.first] = property.second->Clone(storage);
+        auto prop = storage.GetPropertyIx(property.first.name);
+        node_atom->properties_[prop] = property.second->Clone(storage);
       }
       return node_atom;
     }
@@ -1387,18 +1524,28 @@ cpp<#
 (lcp:define-class edge-atom (pattern-atom)
   ((type "Type" :initval "Type::SINGLE" :scope :public)
    (direction "Direction" :initval "Direction::BOTH" :scope :public)
-   (edge-types "std::vector<storage::EdgeType>" :scope :public
-               :capnp-save (lcp:capnp-save-vector
-                             "storage::capnp::EdgeType"
-                             "storage::EdgeType")
+   (edge-types "std::vector<EdgeTypeIx>" :scope :public
+               :slk-load (lambda (member)
+                          #>cpp
+                          size_t size = 0;
+                          slk::Load(&size, reader);
+                          self->${member}.resize(size);
+                          for (size_t i = 0; i < size; ++i) {
+                            slk::Load(&self->${member}[i], reader, storage);
+                          }
+                          cpp<#)
                :capnp-load (lcp:capnp-load-vector
-                             "storage::capnp::EdgeType"
-                             "storage::EdgeType"))
-   (properties "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
+                            "capnp::EdgeTypeIx" "EdgeTypeIx"
+                            "[storage](const auto &reader) {
+                               EdgeTypeIx ix;
+                               Load(&ix, reader, storage);
+                               return ix;
+                             }"))
+   (properties "std::unordered_map<PropertyIx, Expression *>"
                :scope :public
                :slk-save #'slk-save-property-map
                :slk-load #'slk-load-property-map
-               :capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
+               :capnp-type "Utils.Map(PropertyIx, Tree)"
                :capnp-save #'save-property-map
                :capnp-load #'load-property-map)
    (lower-bound "Expression *" :initval "nullptr" :scope :public
@@ -1505,9 +1652,13 @@ cpp<#
       auto *edge_atom = storage.Create<EdgeAtom>(identifier_->Clone(storage));
       edge_atom->direction_ = direction_;
       edge_atom->type_ = type_;
-      edge_atom->edge_types_ = edge_types_;
+      edge_atom->edge_types_.reserve(edge_types_.size());
+      for (const auto &edge_type : edge_types_) {
+        edge_atom->edge_types_.push_back(storage.GetEdgeTypeIx(edge_type.name));
+      }
       for (auto property : properties_) {
-        edge_atom->properties_[property.first] = property.second->Clone(storage);
+        auto prop = storage.GetPropertyIx(property.first.name);
+        edge_atom->properties_[prop] = property.second->Clone(storage);
       }
       edge_atom->lower_bound_ = CloneOpt(lower_bound_, storage);
       edge_atom->upper_bound_ = CloneOpt(upper_bound_, storage);
@@ -1541,7 +1692,7 @@ cpp<#
 
     // Creates an edge atom for a SINGLE expansion with the given .
     EdgeAtom(int uid, Identifier *identifier, Type type, Direction direction,
-             const std::vector<storage::EdgeType> &edge_types)
+             const std::vector<EdgeTypeIx> &edge_types)
         : PatternAtom(uid, identifier),
           type_(type),
           direction_(direction),
@@ -1847,10 +1998,28 @@ cpp<#
 
 (lcp:define-class index-query (query)
   ((action "Action" :scope :public)
-   (label "storage::Label" :scope :public)
-   (properties "std::vector<storage::Property>" :scope :public
-               :capnp-save (lcp:capnp-save-vector "storage::capnp::Property" "storage::Property")
-               :capnp-load (lcp:capnp-load-vector "storage::capnp::Property" "storage::Property")))
+   (label "LabelIx" :scope :public
+          :slk-load (lambda (member)
+                     #>cpp
+                     slk::Load(&self->${member}, reader, storage);
+                     cpp<#))
+   (properties "std::vector<PropertyIx>" :scope :public
+               :slk-load (lambda (member)
+                          #>cpp
+                          size_t size = 0;
+                          slk::Load(&size, reader);
+                          self->${member}.resize(size);
+                          for (size_t i = 0; i < size; ++i) {
+                            slk::Load(&self->${member}[i], reader, storage);
+                          }
+                          cpp<#)
+               :capnp-load (lcp:capnp-load-vector
+                            "capnp::PropertyIx" "PropertyIx"
+                            "[storage](const auto &reader) {
+                               PropertyIx ix;
+                               Load(&ix, reader, storage);
+                               return ix;
+                             }")))
   (:public
    (lcp:define-enum action
        (create create-unique drop)
@@ -1862,14 +2031,20 @@ cpp<#
     DEFVISITABLE(QueryVisitor<void>);
 
     IndexQuery *Clone(AstStorage &storage) const override {
-      return storage.Create<IndexQuery>(action_, label_, properties_);
+      std::vector<PropertyIx> new_properties;
+      new_properties.reserve(properties_.size());
+      for (const auto &prop : properties_) {
+        new_properties.push_back(storage.GetPropertyIx(prop.name));
+      }
+      return storage.Create<IndexQuery>(action_, storage.GetLabelIx(label_.name),
+                                        new_properties);
     }
   cpp<#)
   (:protected
     #>cpp
     explicit IndexQuery(int uid) : Query(uid) {}
-    IndexQuery(int uid, Action action, storage::Label label,
-               std::vector<storage::Property> properties)
+    IndexQuery(int uid, Action action, LabelIx label,
+               std::vector<PropertyIx> properties)
         : Query(uid), action_(action), label_(label), properties_(properties) {}
     cpp<#)
   (:private
@@ -2330,13 +2505,23 @@ cpp<#
                :capnp-type "Tree" :capnp-init nil
                :capnp-save #'save-ast-pointer
                :capnp-load (load-ast-pointer "Identifier *"))
-   (labels "std::vector<storage::Label>" :scope :public
-           :capnp-save (lcp:capnp-save-vector
-                         "storage::capnp::Label"
-                         "storage::Label")
+   (labels "std::vector<LabelIx>" :scope :public
+           :slk-load (lambda (member)
+                      #>cpp
+                      size_t size = 0;
+                      slk::Load(&size, reader);
+                      self->${member}.resize(size);
+                      for (size_t i = 0; i < size; ++i) {
+                        slk::Load(&self->${member}[i], reader, storage);
+                      }
+                      cpp<#)
            :capnp-load (lcp:capnp-load-vector
-                         "storage::capnp::Label"
-                         "storage::Label")))
+                        "capnp::LabelIx" "LabelIx"
+                        "[storage](const auto &reader) {
+                           LabelIx ix;
+                           Load(&ix, reader, storage);
+                           return ix;
+                         }")))
   (:public
     #>cpp
     SetLabels() = default;
@@ -2349,14 +2534,19 @@ cpp<#
     }
 
     SetLabels *Clone(AstStorage &storage) const override {
-      return storage.Create<SetLabels>(identifier_->Clone(storage), labels_);
+      std::vector<LabelIx> new_labels;
+      new_labels.reserve(labels_.size());
+      for (const auto &label : labels_) {
+        new_labels.push_back(storage.GetLabelIx(label.name));
+      }
+      return storage.Create<SetLabels>(identifier_->Clone(storage), new_labels);
     }
     cpp<#)
   (:protected
     #>cpp
     explicit SetLabels(int uid) : Clause(uid) {}
     SetLabels(int uid, Identifier *identifier,
-              const std::vector<storage::Label> &labels)
+              const std::vector<LabelIx> &labels)
         : Clause(uid), identifier_(identifier), labels_(labels) {}
     cpp<#)
   (:private
@@ -2406,13 +2596,23 @@ cpp<#
                :capnp-type "Tree" :capnp-init nil
                :capnp-save #'save-ast-pointer
                :capnp-load (load-ast-pointer "Identifier *"))
-   (labels "std::vector<storage::Label>" :scope :public
-           :capnp-save (lcp:capnp-save-vector
-                         "storage::capnp::Label"
-                         "storage::Label")
+   (labels "std::vector<LabelIx>" :scope :public
+           :slk-load (lambda (member)
+                      #>cpp
+                      size_t size = 0;
+                      slk::Load(&size, reader);
+                      self->${member}.resize(size);
+                      for (size_t i = 0; i < size; ++i) {
+                        slk::Load(&self->${member}[i], reader, storage);
+                      }
+                      cpp<#)
            :capnp-load (lcp:capnp-load-vector
-                         "storage::capnp::Label"
-                         "storage::Label")))
+                        "capnp::LabelIx" "LabelIx"
+                        "[storage](const auto &reader) {
+                           LabelIx ix;
+                           Load(&ix, reader, storage);
+                           return ix;
+                         }")))
   (:public
     #>cpp
     RemoveLabels() = default;
@@ -2425,14 +2625,19 @@ cpp<#
     }
 
     RemoveLabels *Clone(AstStorage &storage) const override {
-      return storage.Create<RemoveLabels>(identifier_->Clone(storage), labels_);
+      std::vector<LabelIx> new_labels;
+      new_labels.reserve(labels_.size());
+      for (const auto &label : labels_) {
+        new_labels.push_back(storage.GetLabelIx(label.name));
+      }
+      return storage.Create<RemoveLabels>(identifier_->Clone(storage), new_labels);
     }
     cpp<#)
   (:protected
     #>cpp
     explicit RemoveLabels(int uid) : Clause(uid) {}
     RemoveLabels(int uid, Identifier *identifier,
-                 const std::vector<storage::Label> &labels)
+                 const std::vector<LabelIx> &labels)
         : Clause(uid), identifier_(identifier), labels_(labels) {}
     cpp<#)
   (:private
diff --git a/src/query/frontend/ast/cypher_main_visitor.cpp b/src/query/frontend/ast/cypher_main_visitor.cpp
index 9e7f128d8..07bf15bc9 100644
--- a/src/query/frontend/ast/cypher_main_visitor.cpp
+++ b/src/query/frontend/ast/cypher_main_visitor.cpp
@@ -12,9 +12,8 @@
 #include <utility>
 #include <vector>
 
-#include "glog/logging.h"
+#include <glog/logging.h>
 
-#include "database/graph_db.hpp"
 #include "query/common.hpp"
 #include "query/exceptions.hpp"
 #include "query/interpret/awesome_memgraph_functions.hpp"
@@ -87,10 +86,9 @@ antlrcpp::Any CypherMainVisitor::visitCreateIndex(
     MemgraphCypher::CreateIndexContext *ctx) {
   auto *index_query = storage_->Create<IndexQuery>();
   index_query->action_ = IndexQuery::Action::CREATE;
-  index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
-  std::pair<std::string, storage::Property> name_key =
-      ctx->propertyKeyName()->accept(this);
-  index_query->properties_ = {name_key.second};
+  index_query->label_ = AddLabel(ctx->labelName()->accept(this));
+  PropertyIx name_key = ctx->propertyKeyName()->accept(this);
+  index_query->properties_ = {name_key};
   return index_query;
 }
 
@@ -98,12 +96,11 @@ antlrcpp::Any CypherMainVisitor::visitCreateUniqueIndex(
     MemgraphCypher::CreateUniqueIndexContext *ctx) {
   auto *index_query = storage_->Create<IndexQuery>();
   index_query->action_ = IndexQuery::Action::CREATE_UNIQUE;
-  index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
+  index_query->label_ = AddLabel(ctx->labelName()->accept(this));
   index_query->properties_.reserve(ctx->propertyKeyName().size());
   for (const auto &prop_name : ctx->propertyKeyName()) {
-    std::pair<std::string, storage::Property> name_key =
-        prop_name->accept(this);
-    index_query->properties_.push_back(name_key.second);
+    PropertyIx name_key = prop_name->accept(this);
+    index_query->properties_.push_back(name_key);
   }
   return index_query;
 }
@@ -112,10 +109,9 @@ antlrcpp::Any CypherMainVisitor::visitDropIndex(
     MemgraphCypher::DropIndexContext *ctx) {
   auto *index_query = storage_->Create<IndexQuery>();
   index_query->action_ = IndexQuery::Action::DROP;
-  std::pair<std::string, storage::Property> key =
-      ctx->propertyKeyName()->accept(this);
-  index_query->properties_ = {key.second};
-  index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
+  PropertyIx key = ctx->propertyKeyName()->accept(this);
+  index_query->properties_ = {key};
+  index_query->label_ = AddLabel(ctx->labelName()->accept(this));
   return index_query;
 }
 
@@ -746,24 +742,21 @@ antlrcpp::Any CypherMainVisitor::visitNodePattern(
     anonymous_identifiers.push_back(&node->identifier_);
   }
   if (ctx->nodeLabels()) {
-    node->labels_ =
-        ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
+    node->labels_ = ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
   }
   if (ctx->properties()) {
-    node->properties_ =
-        ctx->properties()
-            ->accept(this)
-            .as<std::unordered_map<std::pair<std::string, storage::Property>,
-                                   Expression *>>();
+    node->properties_ = ctx->properties()
+                            ->accept(this)
+                            .as<std::unordered_map<PropertyIx, Expression *>>();
   }
   return node;
 }
 
 antlrcpp::Any CypherMainVisitor::visitNodeLabels(
     MemgraphCypher::NodeLabelsContext *ctx) {
-  std::vector<storage::Label> labels;
+  std::vector<LabelIx> labels;
   for (auto *node_label : ctx->nodeLabel()) {
-    labels.push_back(dba_->Label(node_label->accept(this)));
+    labels.push_back(AddLabel(node_label->accept(this)));
   }
   return labels;
 }
@@ -783,11 +776,9 @@ antlrcpp::Any CypherMainVisitor::visitProperties(
 
 antlrcpp::Any CypherMainVisitor::visitMapLiteral(
     MemgraphCypher::MapLiteralContext *ctx) {
-  std::unordered_map<std::pair<std::string, storage::Property>, Expression *>
-      map;
+  std::unordered_map<PropertyIx, Expression *> map;
   for (int i = 0; i < static_cast<int>(ctx->propertyKeyName().size()); ++i) {
-    std::pair<std::string, storage::Property> key =
-        ctx->propertyKeyName()[i]->accept(this);
+    PropertyIx key = ctx->propertyKeyName()[i]->accept(this);
     Expression *value = ctx->expression()[i]->accept(this);
     if (!map.insert({key, value}).second) {
       throw SemanticException("Same key can't appear twice in a map literal.");
@@ -806,8 +797,7 @@ antlrcpp::Any CypherMainVisitor::visitListLiteral(
 
 antlrcpp::Any CypherMainVisitor::visitPropertyKeyName(
     MemgraphCypher::PropertyKeyNameContext *ctx) {
-  const std::string key_name = visitChildren(ctx);
-  return std::make_pair(key_name, dba_->Property(key_name));
+  return AddProperty(visitChildren(ctx));
 }
 
 antlrcpp::Any CypherMainVisitor::visitSymbolicName(
@@ -930,7 +920,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
     edge->edge_types_ = ctx->relationshipDetail()
                             ->relationshipTypes()
                             ->accept(this)
-                            .as<std::vector<storage::EdgeType>>();
+                            .as<std::vector<EdgeTypeIx>>();
   }
 
   auto relationshipLambdas = relationshipDetail->relationshipLambda();
@@ -1009,8 +999,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
       edge->properties_ =
           properties[0]
               ->accept(this)
-              .as<std::unordered_map<std::pair<std::string, storage::Property>,
-                                     Expression *>>();
+              .as<std::unordered_map<PropertyIx, Expression *>>();
       break;
     }
     default:
@@ -1035,9 +1024,9 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipLambda(
 
 antlrcpp::Any CypherMainVisitor::visitRelationshipTypes(
     MemgraphCypher::RelationshipTypesContext *ctx) {
-  std::vector<storage::EdgeType> types;
+  std::vector<EdgeTypeIx> types;
   for (auto *edge_type : ctx->relTypeName()) {
-    types.push_back(dba_->EdgeType(edge_type->accept(this)));
+    types.push_back(AddEdgeType(edge_type->accept(this)));
   }
   return types;
 }
@@ -1296,7 +1285,7 @@ antlrcpp::Any CypherMainVisitor::visitExpression2a(
   Expression *expression = ctx->expression2b()->accept(this);
   if (ctx->nodeLabels()) {
     auto labels =
-        ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
+        ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
     expression = storage_->Create<LabelsTest>(expression, labels);
   }
   return expression;
@@ -1306,9 +1295,8 @@ antlrcpp::Any CypherMainVisitor::visitExpression2b(
     MemgraphCypher::Expression2bContext *ctx) {
   Expression *expression = ctx->atom()->accept(this);
   for (auto *lookup : ctx->propertyLookup()) {
-    std::pair<std::string, storage::Property> key = lookup->accept(this);
-    auto property_lookup =
-        storage_->Create<PropertyLookup>(expression, key.first, key.second);
+    PropertyIx key = lookup->accept(this);
+    auto property_lookup = storage_->Create<PropertyLookup>(expression, key);
     expression = property_lookup;
   }
   return expression;
@@ -1444,8 +1432,7 @@ antlrcpp::Any CypherMainVisitor::visitLiteral(
     return static_cast<Expression *>(storage_->Create<MapLiteral>(
         ctx->mapLiteral()
             ->accept(this)
-            .as<std::unordered_map<std::pair<std::string, storage::Property>,
-                                   Expression *>>()));
+            .as<std::unordered_map<PropertyIx, Expression *>>()));
   }
   return visitChildren(ctx);
 }
@@ -1604,7 +1591,7 @@ antlrcpp::Any CypherMainVisitor::visitSetItem(
   set_labels->identifier_ = storage_->Create<Identifier>(
       ctx->variable()->accept(this).as<std::string>());
   set_labels->labels_ =
-      ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
+      ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
   return static_cast<Clause *>(set_labels);
 }
 
@@ -1631,7 +1618,7 @@ antlrcpp::Any CypherMainVisitor::visitRemoveItem(
   remove_labels->identifier_ = storage_->Create<Identifier>(
       ctx->variable()->accept(this).as<std::string>());
   remove_labels->labels_ =
-      ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
+      ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
   return static_cast<Clause *>(remove_labels);
 }
 
@@ -1639,9 +1626,9 @@ antlrcpp::Any CypherMainVisitor::visitPropertyExpression(
     MemgraphCypher::PropertyExpressionContext *ctx) {
   Expression *expression = ctx->atom()->accept(this);
   for (auto *lookup : ctx->propertyLookup()) {
-    std::pair<std::string, storage::Property> key = lookup->accept(this);
+    PropertyIx key = lookup->accept(this);
     auto property_lookup =
-        storage_->Create<PropertyLookup>(expression, key.first, key.second);
+        storage_->Create<PropertyLookup>(expression, key);
     expression = property_lookup;
   }
   // It is guaranteed by grammar that there is at least one propertyLookup.
@@ -1722,4 +1709,16 @@ antlrcpp::Any CypherMainVisitor::visitFilterExpression(
   return 0;
 }
 
+LabelIx CypherMainVisitor::AddLabel(const std::string &name) {
+  return storage_->GetLabelIx(name);
+}
+
+PropertyIx CypherMainVisitor::AddProperty(const std::string &name) {
+  return storage_->GetPropertyIx(name);
+}
+
+EdgeTypeIx CypherMainVisitor::AddEdgeType(const std::string &name) {
+  return storage_->GetEdgeTypeIx(name);
+}
+
 }  // namespace query::frontend
diff --git a/src/query/frontend/ast/cypher_main_visitor.hpp b/src/query/frontend/ast/cypher_main_visitor.hpp
index b0a7501eb..88a7f2dbd 100644
--- a/src/query/frontend/ast/cypher_main_visitor.hpp
+++ b/src/query/frontend/ast/cypher_main_visitor.hpp
@@ -16,13 +16,11 @@ namespace query {
 namespace frontend {
 
 using antlropencypher::MemgraphCypher;
-using query::Context;
 
 class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
  public:
-  explicit CypherMainVisitor(ParsingContext context, AstStorage *storage,
-                             database::GraphDbAccessor *dba)
-      : context_(context), storage_(storage), dba_(dba) {}
+  explicit CypherMainVisitor(ParsingContext context, AstStorage *storage)
+      : context_(context), storage_(storage) {}
 
  private:
   Expression *CreateBinaryOperatorByToken(size_t token, Expression *e1,
@@ -409,13 +407,13 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
       MemgraphCypher::NodePatternContext *ctx) override;
 
   /**
-   * @return vector<storage::Label>
+   * @return vector<LabelIx>
    */
   antlrcpp::Any visitNodeLabels(
       MemgraphCypher::NodeLabelsContext *ctx) override;
 
   /**
-   * @return unordered_map<storage::Property, Expression*>
+   * @return unordered_map<PropertyIx, Expression*>
    */
   antlrcpp::Any visitProperties(
       MemgraphCypher::PropertiesContext *ctx) override;
@@ -433,7 +431,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
       MemgraphCypher::ListLiteralContext *ctx) override;
 
   /**
-   * @return storage::Property
+   * @return PropertyIx
    */
   antlrcpp::Any visitPropertyKeyName(
       MemgraphCypher::PropertyKeyNameContext *ctx) override;
@@ -488,7 +486,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
       MemgraphCypher::RelationshipLambdaContext *ctx) override;
 
   /**
-   * @return vector<storage::EdgeType>
+   * @return vector<EdgeTypeIx>
    */
   antlrcpp::Any visitRelationshipTypes(
       MemgraphCypher::RelationshipTypesContext *ctx) override;
@@ -776,9 +774,12 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
   const static std::string kAnonPrefix;
 
  private:
+  LabelIx AddLabel(const std::string &name);
+  PropertyIx AddProperty(const std::string &name);
+  EdgeTypeIx AddEdgeType(const std::string &name);
+
   ParsingContext context_;
   AstStorage *storage_;
-  database::GraphDbAccessor *dba_;
 
   // Set of identifiers from queries.
   std::unordered_set<std::string> users_identifiers;
diff --git a/src/query/frontend/ast/pretty_print.cpp b/src/query/frontend/ast/pretty_print.cpp
index ec9833ad3..3a1b4f988 100644
--- a/src/query/frontend/ast/pretty_print.cpp
+++ b/src/query/frontend/ast/pretty_print.cpp
@@ -10,7 +10,8 @@ namespace {
 
 class ExpressionPrettyPrinter : public ExpressionVisitor<void> {
  public:
-  explicit ExpressionPrettyPrinter(std::ostream *out);
+  ExpressionPrettyPrinter(const AstStorage *storage, std::ostream *out)
+      : storage_(storage), out_(out) {}
 
   // Unary operators
   void Visit(NotOperator &op) override;
@@ -56,58 +57,78 @@ class ExpressionPrettyPrinter : public ExpressionVisitor<void> {
   void Visit(NamedExpression &op) override;
 
  private:
+  const AstStorage *storage_;
   std::ostream *out_;
+
+  // Declare all of the different `PrintObject` overloads upfront since they're
+  // mutually recursive. Without this, overload resolution depends on the
+  // ordering of the overloads within the source, which is quite fragile.
+
+  template <typename T>
+  void PrintObject(std::ostream *out, const T &arg);
+
+  void PrintObject(std::ostream *out, const std::string &str);
+
+  void PrintObject(std::ostream *out, Aggregation::Op op);
+
+  void PrintObject(std::ostream *out, Expression *expr);
+
+  void PrintObject(std::ostream *out, const PropertyValue &value);
+
+  template <typename T>
+  void PrintObject(std::ostream *out, const std::vector<T> &vec);
+
+  template <typename K, typename V>
+  void PrintObject(std::ostream *out, const std::map<K, V> &map);
+
+  template <typename T>
+  void PrintOperatorArgs(std::ostream *out, const T &arg) {
+    *out << " ";
+    PrintObject(out, arg);
+    *out << ")";
+  }
+
+  template <typename T, typename... Ts>
+  void PrintOperatorArgs(std::ostream *out, const T &arg, const Ts &... args) {
+    *out << " ";
+    PrintObject(out, arg);
+    PrintOperatorArgs(out, args...);
+  }
+
+  template <typename... Ts>
+  void PrintOperator(std::ostream *out, const std::string &name,
+                     const Ts &... args) {
+    *out << "(" << name;
+    PrintOperatorArgs(out, args...);
+  }
 };
 
-// Declare all of the different `PrintObject` overloads upfront since they're
-// mutually recursive. Without this, overload resolution depends on the ordering
-// of the overloads within the source, which is quite fragile.
-
 template <typename T>
-void PrintObject(std::ostream *out, const T &arg);
-
-void PrintObject(std::ostream *out, const std::string &str);
-
-void PrintObject(std::ostream *out, Aggregation::Op op);
-
-void PrintObject(std::ostream *out, Expression *expr);
-
-void PrintObject(std::ostream *out, const PropertyValue &value);
-
-template <typename T>
-void PrintObject(std::ostream *out, const std::vector<T> &vec);
-
-template <typename K, typename V>
-void PrintObject(std::ostream *out, const std::map<K, V> &map);
-
-void PrintObject(
-    std::ostream *out,
-    const std::unordered_map<std::pair<std::string, storage::Property>,
-                             Expression *> &map);
-
-template <typename T>
-void PrintObject(std::ostream *out, const T &arg) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out, const T &arg) {
   *out << arg;
 }
 
-void PrintObject(std::ostream *out, const std::string &str) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
+                                          const std::string &str) {
   *out << utils::Escape(str);
 }
 
-void PrintObject(std::ostream *out, Aggregation::Op op) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
+                                          Aggregation::Op op) {
   *out << Aggregation::OpToString(op);
 }
 
-void PrintObject(std::ostream *out, Expression *expr) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out, Expression *expr) {
   if (expr) {
-    ExpressionPrettyPrinter printer{out};
+    ExpressionPrettyPrinter printer{storage_, out};
     expr->Accept(printer);
   } else {
     *out << "<null>";
   }
 }
 
-void PrintObject(std::ostream *out, const PropertyValue &value) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
+                                          const PropertyValue &value) {
   switch (value.type()) {
     case PropertyValue::Type::Null:
       *out << "null";
@@ -140,62 +161,27 @@ void PrintObject(std::ostream *out, const PropertyValue &value) {
 }
 
 template <typename T>
-void PrintObject(std::ostream *out, const std::vector<T> &vec) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
+                                          const std::vector<T> &vec) {
   *out << "[";
-  utils::PrintIterable(*out, vec, ", ", [](auto &stream, const auto &item) {
-    PrintObject(&stream, item);
+  utils::PrintIterable(*out, vec, ", ", [this](auto &stream, const auto &item) {
+    this->PrintObject(&stream, item);
   });
   *out << "]";
 }
 
 template <typename K, typename V>
-void PrintObject(std::ostream *out, const std::map<K, V> &map) {
+void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
+                                          const std::map<K, V> &map) {
   *out << "{";
-  utils::PrintIterable(*out, map, ", ", [](auto &stream, const auto &item) {
-    PrintObject(&stream, item.first);
+  utils::PrintIterable(*out, map, ", ", [this](auto &stream, const auto &item) {
+    this->PrintObject(&stream, item.first);
     stream << ": ";
-    PrintObject(&stream, item.second);
+    this->PrintObject(&stream, item.second);
   });
   *out << "}";
 }
 
-void PrintObject(
-    std::ostream *out,
-    const std::unordered_map<std::pair<std::string, storage::Property>,
-                             Expression *> &map) {
-  *out << "{";
-  utils::PrintIterable(*out, map, ", ", [](auto &stream, const auto &item) {
-    PrintObject(&stream, item.first.first);
-    stream << ": ";
-    PrintObject(&stream, item.second);
-  });
-  *out << "}";
-}
-
-template <typename T>
-void PrintOperatorArgs(std::ostream *out, const T &arg) {
-  *out << " ";
-  PrintObject(out, arg);
-  *out << ")";
-}
-
-template <typename T, typename... Ts>
-void PrintOperatorArgs(std::ostream *out, const T &arg, const Ts &... args) {
-  *out << " ";
-  PrintObject(out, arg);
-  PrintOperatorArgs(out, args...);
-}
-
-template <typename... Ts>
-void PrintOperator(std::ostream *out, const std::string &name,
-                   const Ts &... args) {
-  *out << "(" << name;
-  PrintOperatorArgs(out, args...);
-}
-
-ExpressionPrettyPrinter::ExpressionPrettyPrinter(std::ostream *out)
-    : out_(out) {}
-
 #define UNARY_OPERATOR_VISIT(OP_NODE, OP_STR)        \
   void ExpressionPrettyPrinter::Visit(OP_NODE &op) { \
     PrintOperator(out_, OP_STR, op.expression_);     \
@@ -247,7 +233,11 @@ void ExpressionPrettyPrinter::Visit(ListLiteral &op) {
 }
 
 void ExpressionPrettyPrinter::Visit(MapLiteral &op) {
-  PrintObject(out_, op.elements_);
+  std::map<std::string, Expression *> map;
+  for (const auto &kv : op.elements_) {
+    map[storage_->properties_[kv.first.ix]] = kv.second;
+  }
+  PrintObject(out_, map);
 }
 
 void ExpressionPrettyPrinter::Visit(LabelsTest &op) {
@@ -294,7 +284,8 @@ void ExpressionPrettyPrinter::Visit(PrimitiveLiteral &op) {
 }
 
 void ExpressionPrettyPrinter::Visit(PropertyLookup &op) {
-  PrintOperator(out_, "PropertyLookup", op.expression_, op.property_name_);
+  const auto &prop_name = storage_->properties_[op.property_.ix];
+  PrintOperator(out_, "PropertyLookup", op.expression_, prop_name);
 }
 
 void ExpressionPrettyPrinter::Visit(ParameterLookup &op) {
@@ -307,13 +298,15 @@ void ExpressionPrettyPrinter::Visit(NamedExpression &op) {
 
 }  // namespace
 
-void PrintExpression(Expression *expr, std::ostream *out) {
-  ExpressionPrettyPrinter printer{out};
+void PrintExpression(const AstStorage &storage, Expression *expr,
+                     std::ostream *out) {
+  ExpressionPrettyPrinter printer{&storage, out};
   expr->Accept(printer);
 }
 
-void PrintExpression(NamedExpression *expr, std::ostream *out) {
-  ExpressionPrettyPrinter printer{out};
+void PrintExpression(const AstStorage &storage, NamedExpression *expr,
+                     std::ostream *out) {
+  ExpressionPrettyPrinter printer{&storage, out};
   expr->Accept(printer);
 }
 
diff --git a/src/query/frontend/ast/pretty_print.hpp b/src/query/frontend/ast/pretty_print.hpp
index 450036d47..863f9e572 100644
--- a/src/query/frontend/ast/pretty_print.hpp
+++ b/src/query/frontend/ast/pretty_print.hpp
@@ -6,7 +6,9 @@
 
 namespace query {
 
-void PrintExpression(Expression *expr, std::ostream *out);
-void PrintExpression(NamedExpression *expr, std::ostream *out);
+void PrintExpression(const AstStorage &storage, Expression *expr,
+                     std::ostream *out);
+void PrintExpression(const AstStorage &storage, NamedExpression *expr,
+                     std::ostream *out);
 
 }  // namespace query
diff --git a/src/query/interpret/eval.hpp b/src/query/interpret/eval.hpp
index 295b2a784..fac369258 100644
--- a/src/query/interpret/eval.hpp
+++ b/src/query/interpret/eval.hpp
@@ -270,14 +270,14 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
         return TypedValue::Null;
       case TypedValue::Type::Vertex:
         return expression_result.Value<VertexAccessor>().PropsAt(
-            property_lookup.property_);
+            GetProperty(property_lookup.property_));
       case TypedValue::Type::Edge:
         return expression_result.Value<EdgeAccessor>().PropsAt(
-            property_lookup.property_);
+            GetProperty(property_lookup.property_));
       case TypedValue::Type::Map: {
         auto &map =
             expression_result.Value<std::map<std::string, TypedValue>>();
-        auto found = map.find(property_lookup.property_name_);
+        auto found = map.find(property_lookup.property_.name);
         if (found == map.end()) return TypedValue::Null;
         return found->second;
       }
@@ -295,7 +295,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
       case TypedValue::Type::Vertex: {
         auto vertex = expression_result.Value<VertexAccessor>();
         for (const auto label : labels_test.labels_) {
-          if (!vertex.has_label(label)) {
+          if (!vertex.has_label(GetLabel(label))) {
             return false;
           }
         }
@@ -323,7 +323,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
   TypedValue Visit(MapLiteral &literal) override {
     std::map<std::string, TypedValue> result;
     for (const auto &pair : literal.elements_)
-      result.emplace(pair.first.first, pair.second->Accept(*this));
+      result.emplace(pair.first.name, pair.second->Accept(*this));
     return result;
   }
 
@@ -481,6 +481,14 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
   }
 
  private:
+  storage::Property GetProperty(PropertyIx prop) {
+    return ctx_->properties[prop.ix];
+  }
+
+  storage::Label GetLabel(LabelIx label) {
+    return ctx_->labels[label.ix];
+  }
+
   // If the given TypedValue contains accessors, switch them to New or Old,
   // depending on use_new_ flag.
   void SwitchAccessors(TypedValue &value) {
diff --git a/src/query/interpreter.cpp b/src/query/interpreter.cpp
index 3d596d71f..fd743e232 100644
--- a/src/query/interpreter.cpp
+++ b/src/query/interpreter.cpp
@@ -42,6 +42,7 @@ class SingleNodeLogicalPlan final : public LogicalPlan {
   const plan::LogicalOperator &GetRoot() const override { return *root_; }
   double GetCost() const override { return cost_; }
   const SymbolTable &GetSymbolTable() const override { return symbol_table_; }
+  const AstStorage &GetAstStorage() const override { return storage_; }
 
  private:
   std::unique_ptr<plan::LogicalOperator> root_;
@@ -70,13 +71,19 @@ TypedValue EvaluateOptionalExpression(Expression *expression,
 }
 
 Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
-                         const EvaluationContext &evaluation_context,
+                         const Parameters &parameters,
                          database::GraphDbAccessor *db_accessor) {
   // Empty frame for evaluation of password expression. This is OK since
   // password should be either null or string literal and it's evaluation
   // should not depend on frame.
   Frame frame(0);
   SymbolTable symbol_table;
+  EvaluationContext evaluation_context;
+  evaluation_context.timestamp =
+      std::chrono::duration_cast<std::chrono::milliseconds>(
+          std::chrono::system_clock::now().time_since_epoch())
+          .count();
+  evaluation_context.parameters = parameters;
   ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
                                 db_accessor, GraphView::OLD);
 
@@ -355,12 +362,18 @@ Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
 
 Callback HandleStreamQuery(StreamQuery *stream_query,
                            integrations::kafka::Streams *streams,
-                           const EvaluationContext &evaluation_context,
+                           const Parameters &parameters,
                            database::GraphDbAccessor *db_accessor) {
   // Empty frame and symbol table for evaluation of expressions. This is OK
   // since all expressions should be literals or parameter lookups.
   Frame frame(0);
   SymbolTable symbol_table;
+  EvaluationContext evaluation_context;
+  evaluation_context.timestamp =
+      std::chrono::duration_cast<std::chrono::milliseconds>(
+          std::chrono::system_clock::now().time_since_epoch())
+          .count();
+  evaluation_context.parameters = parameters;
   ExpressionEvaluator eval(&frame, symbol_table, evaluation_context,
                            db_accessor, GraphView::OLD);
 
@@ -513,8 +526,12 @@ Callback HandleIndexQuery(IndexQuery *index_query,
                           std::function<void()> invalidate_plan_cache,
                           database::GraphDbAccessor *db_accessor) {
   auto action = index_query->action_;
-  auto label = index_query->label_;
-  auto properties = index_query->properties_;
+  auto label = db_accessor->Label(index_query->label_.name);
+  std::vector<storage::Property> properties;
+  properties.reserve(index_query->properties_.size());
+  for (const auto &prop : index_query->properties_) {
+    properties.push_back(db_accessor->Property(prop.name));
+  }
 
   if (properties.size() > 1) {
     throw utils::NotYetImplemented("index on multiple properties");
@@ -574,12 +591,12 @@ Interpreter::Results Interpreter::operator()(
 
   AstStorage ast_storage;
   Context execution_context(db_accessor);
-  auto &evaluation_context = execution_context.evaluation_context_;
+  Parameters parameters;
   std::map<std::string, TypedValue> summary;
 
   utils::Timer parsing_timer;
-  auto queries = StripAndParseQuery(query_string, &execution_context,
-                                    &ast_storage, &db_accessor, params);
+  auto queries = StripAndParseQuery(query_string, &parameters, &ast_storage,
+                                    &db_accessor, params);
   StrippedQuery &stripped_query = queries.first;
   ParsedQuery &parsed_query = queries.second;
   auto parsing_time = parsing_timer.Elapsed();
@@ -600,15 +617,12 @@ Interpreter::Results Interpreter::operator()(
 
   if (auto *cypher_query = dynamic_cast<CypherQuery *>(parsed_query.query)) {
     plan = CypherQueryToPlan(stripped_query.hash(), cypher_query,
-                             std::move(ast_storage),
-                             evaluation_context.parameters, &db_accessor);
+                             std::move(ast_storage), parameters, &db_accessor);
     auto planning_time = planning_timer.Elapsed();
     summary["planning_time"] = planning_time.count();
     summary["cost_estimate"] = plan->cost();
 
-    execution_context.symbol_table_ = plan->symbol_table();
-    auto output_symbols =
-        plan->plan().OutputSymbols(execution_context.symbol_table_);
+    auto output_symbols = plan->plan().OutputSymbols(plan->symbol_table());
 
     std::vector<std::string> header;
     for (const auto &symbol : output_symbols) {
@@ -620,11 +634,8 @@ Interpreter::Results Interpreter::operator()(
                            .first);
     }
 
-    auto cursor = plan->plan().MakeCursor(db_accessor);
-
-    return Results(std::move(execution_context), plan, std::move(cursor),
-                   output_symbols, header, summary,
-                   parsed_query.required_privileges);
+    return Results(&db_accessor, parameters, plan, output_symbols, header,
+                   summary, parsed_query.required_privileges);
   }
 
   if (auto *explain_query = dynamic_cast<ExplainQuery *>(parsed_query.query)) {
@@ -636,9 +647,9 @@ Interpreter::Results Interpreter::operator()(
 
     auto cypher_query_hash =
         fnv(stripped_query.query().substr(kExplainQueryStart.size()));
-    std::shared_ptr<CachedPlan> cypher_query_plan = CypherQueryToPlan(
-        cypher_query_hash, explain_query->cypher_query_, std::move(ast_storage),
-        evaluation_context.parameters, &db_accessor);
+    std::shared_ptr<CachedPlan> cypher_query_plan =
+        CypherQueryToPlan(cypher_query_hash, explain_query->cypher_query_,
+                          std::move(ast_storage), parameters, &db_accessor);
 
     std::stringstream printed_plan;
     PrettyPrintPlan(db_accessor, &cypher_query_plan->plan(), &printed_plan);
@@ -649,27 +660,23 @@ Interpreter::Results Interpreter::operator()(
       printed_plan_rows.push_back(std::vector<TypedValue>{row});
     }
 
-    auto query_plan_symbol =
-        execution_context.symbol_table_.CreateSymbol("QUERY PLAN", false);
+    SymbolTable symbol_table;
+    auto query_plan_symbol = symbol_table.CreateSymbol("QUERY PLAN", false);
     std::vector<Symbol> output_symbols{query_plan_symbol};
-    std::vector<std::string> header{query_plan_symbol.name()};
 
     auto output_plan =
         std::make_unique<plan::OutputTable>(output_symbols, printed_plan_rows);
 
     plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
-        std::move(output_plan), 0.0, AstStorage{},
-        execution_context.symbol_table_));
+        std::move(output_plan), 0.0, AstStorage{}, symbol_table));
 
     auto planning_time = planning_timer.Elapsed();
     summary["planning_time"] = planning_time.count();
 
-    execution_context.symbol_table_ = plan->symbol_table();
-    auto cursor = plan->plan().MakeCursor(db_accessor);
+    std::vector<std::string> header{query_plan_symbol.name()};
 
-    return Results(std::move(execution_context), plan, std::move(cursor),
-                   output_symbols, header, summary,
-                   parsed_query.required_privileges);
+    return Results(&db_accessor, parameters, plan, output_symbols, header,
+                   summary, parsed_query.required_privileges);
   }
 
   if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {
@@ -689,22 +696,20 @@ Interpreter::Results Interpreter::operator()(
 
     auto cypher_query_hash =
         fnv(stripped_query.query().substr(kProfileQueryStart.size()));
-    auto cypher_query_plan = CypherQueryToPlan(
-        cypher_query_hash, profile_query->cypher_query_, std::move(ast_storage),
-        evaluation_context.parameters, &db_accessor);
+    auto cypher_query_plan =
+        CypherQueryToPlan(cypher_query_hash, profile_query->cypher_query_,
+                          std::move(ast_storage), parameters, &db_accessor);
 
     // Copy the symbol table and add our own symbols (used by the `OutputTable`
     // operator below)
-    execution_context.symbol_table_ = cypher_query_plan->symbol_table();
+    SymbolTable symbol_table(cypher_query_plan->symbol_table());
 
-    auto operator_symbol =
-        execution_context.symbol_table_.CreateSymbol("OPERATOR", false);
-    auto actual_hits_symbol =
-        execution_context.symbol_table_.CreateSymbol("ACTUAL HITS", false);
+    auto operator_symbol = symbol_table.CreateSymbol("OPERATOR", false);
+    auto actual_hits_symbol = symbol_table.CreateSymbol("ACTUAL HITS", false);
     auto relative_time_symbol =
-        execution_context.symbol_table_.CreateSymbol("RELATIVE TIME", false);
+        symbol_table.CreateSymbol("RELATIVE TIME", false);
     auto absolute_time_symbol =
-        execution_context.symbol_table_.CreateSymbol("ABSOLUTE TIME", false);
+        symbol_table.CreateSymbol("ABSOLUTE TIME", false);
 
     std::vector<Symbol> output_symbols = {operator_symbol, actual_hits_symbol,
                                           relative_time_symbol,
@@ -714,9 +719,9 @@ Interpreter::Results Interpreter::operator()(
         relative_time_symbol.name(), absolute_time_symbol.name()};
 
     auto output_plan = std::make_unique<plan::OutputTable>(
-        output_symbols,
-        [cypher_query_plan, &db_accessor](Frame *frame, Context *context) {
-          auto cursor = cypher_query_plan->plan().MakeCursor(db_accessor);
+        output_symbols, [cypher_query_plan](Frame *frame, Context *context) {
+          auto cursor =
+              cypher_query_plan->plan().MakeCursor(context->db_accessor_);
 
           // Pull everything to profile the execution
           utils::Timer timer;
@@ -726,18 +731,14 @@ Interpreter::Results Interpreter::operator()(
         });
 
     plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
-        std::move(output_plan), 0.0, AstStorage{},
-        execution_context.symbol_table_));
+        std::move(output_plan), 0.0, AstStorage{}, symbol_table));
 
     auto planning_time = planning_timer.Elapsed();
     summary["planning_time"] = planning_time.count();
 
-    execution_context.is_profile_query_ = true;
-    auto cursor = plan->plan().MakeCursor(db_accessor);
-
-    return Results(std::move(execution_context), plan, std::move(cursor),
-                   output_symbols, header, summary,
-                   parsed_query.required_privileges);
+    return Results(&db_accessor, parameters, plan, output_symbols, header,
+                   summary, parsed_query.required_privileges,
+                   /* is_profile_query */ true);
   }
 
   Callback callback;
@@ -758,40 +759,36 @@ Interpreter::Results Interpreter::operator()(
     if (in_explicit_transaction) {
       throw UserModificationInMulticommandTxException();
     }
-    callback =
-        HandleAuthQuery(auth_query, auth_, evaluation_context, &db_accessor);
+    callback = HandleAuthQuery(auth_query, auth_, parameters, &db_accessor);
   } else if (auto *stream_query =
                  dynamic_cast<StreamQuery *>(parsed_query.query)) {
     if (in_explicit_transaction) {
       throw StreamClauseInMulticommandTxException();
     }
-    callback = HandleStreamQuery(stream_query, kafka_streams_,
-                                 evaluation_context, &db_accessor);
+    callback = HandleStreamQuery(stream_query, kafka_streams_, parameters,
+                                 &db_accessor);
   } else {
     LOG(FATAL) << "Should not get here -- unknown query type!";
   }
 
+  SymbolTable symbol_table;
   std::vector<Symbol> output_symbols;
   for (const auto &column : callback.header) {
-    output_symbols.emplace_back(
-        execution_context.symbol_table_.CreateSymbol(column, "false"));
+    output_symbols.emplace_back(symbol_table.CreateSymbol(column, "false"));
   }
 
   plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
       std::make_unique<plan::OutputTable>(
           output_symbols,
           [fn = callback.fn](Frame *, Context *) { return fn(); }),
-      0.0, AstStorage{}, execution_context.symbol_table_));
+      0.0, AstStorage{}, symbol_table));
 
   auto planning_time = planning_timer.Elapsed();
   summary["planning_time"] = planning_time.count();
   summary["cost_estimate"] = 0.0;
 
-  auto cursor = plan->plan().MakeCursor(db_accessor);
-
-  return Results(std::move(execution_context), plan, std::move(cursor),
-                 output_symbols, callback.header, summary,
-                 parsed_query.required_privileges);
+  return Results(&db_accessor, parameters, plan, output_symbols,
+                 callback.header, summary, parsed_query.required_privileges);
 }
 
 std::shared_ptr<Interpreter::CachedPlan> Interpreter::CypherQueryToPlan(
@@ -825,7 +822,7 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
       return std::make_unique<frontend::opencypher::Parser>(original_query);
     }();
     // Convert antlr4 AST into Memgraph AST.
-    frontend::CypherMainVisitor visitor(context, ast_storage, db_accessor);
+    frontend::CypherMainVisitor visitor(context, ast_storage);
     visitor.visit(parser->tree());
     return ParsedQuery{visitor.query(),
                        query::GetRequiredPrivileges(visitor.query())};
@@ -855,8 +852,7 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
     }();
     // Convert antlr4 AST into Memgraph AST.
     AstStorage cached_ast_storage;
-    frontend::CypherMainVisitor visitor(context, &cached_ast_storage,
-                                        db_accessor);
+    frontend::CypherMainVisitor visitor(context, &cached_ast_storage);
     visitor.visit(parser->tree());
     CachedQuery cached_query{std::move(cached_ast_storage), visitor.query(),
                              query::GetRequiredPrivileges(visitor.query())};
@@ -865,31 +861,28 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
         ast_cache_accessor.insert(stripped_query_hash, std::move(cached_query))
             .first;
   }
+  ast_storage->properties_ = ast_it->second.ast_storage.properties_;
+  ast_storage->labels_ = ast_it->second.ast_storage.labels_;
+  ast_storage->edge_types_ = ast_it->second.ast_storage.edge_types_;
   return ParsedQuery{ast_it->second.query->Clone(*ast_storage),
                      ast_it->second.required_privileges};
 }
 
 std::pair<StrippedQuery, Interpreter::ParsedQuery>
 Interpreter::StripAndParseQuery(
-    const std::string &query_string, Context *execution_context,
+    const std::string &query_string, Parameters *parameters,
     AstStorage *ast_storage, database::GraphDbAccessor *db_accessor,
     const std::map<std::string, PropertyValue> &params) {
   StrippedQuery stripped_query(query_string);
 
-  auto &evaluation_context = execution_context->evaluation_context_;
-  evaluation_context.timestamp =
-      std::chrono::duration_cast<std::chrono::milliseconds>(
-          std::chrono::system_clock::now().time_since_epoch())
-          .count();
-
-  evaluation_context.parameters = stripped_query.literals();
+  *parameters = stripped_query.literals();
   for (const auto &param_pair : stripped_query.parameters()) {
     auto param_it = params.find(param_pair.second);
     if (param_it == params.end()) {
       throw query::UnprovidedParameterError("Parameter ${} not provided.",
                                             param_pair.second);
     }
-    evaluation_context.parameters.Add(param_pair.first, param_it->second);
+    parameters->Add(param_pair.first, param_it->second);
   }
 
   ParsingContext parsing_context;
@@ -904,15 +897,15 @@ Interpreter::StripAndParseQuery(
 std::unique_ptr<LogicalPlan> Interpreter::MakeLogicalPlan(
     CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
     database::GraphDbAccessor *db_accessor) {
-  auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
+  auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
 
   auto symbol_table = MakeSymbolTable(query);
 
-  auto planning_context = plan::MakePlanningContext(ast_storage, symbol_table,
-                                                    query, vertex_counts);
+  auto planning_context = plan::MakePlanningContext(&ast_storage, &symbol_table,
+                                                    query, &vertex_counts);
   std::unique_ptr<plan::LogicalOperator> root;
   double cost;
-  std::tie(root, cost) = plan::MakeLogicalPlan(planning_context, parameters,
+  std::tie(root, cost) = plan::MakeLogicalPlan(&planning_context, parameters,
                                                FLAGS_query_cost_planner);
   return std::make_unique<SingleNodeLogicalPlan>(
       std::move(root), cost, std::move(ast_storage), std::move(symbol_table));
diff --git a/src/query/interpreter.hpp b/src/query/interpreter.hpp
index 8680fd011..48c3574c7 100644
--- a/src/query/interpreter.hpp
+++ b/src/query/interpreter.hpp
@@ -35,6 +35,7 @@ class LogicalPlan {
   virtual const plan::LogicalOperator &GetRoot() const = 0;
   virtual double GetCost() const = 0;
   virtual const SymbolTable &GetSymbolTable() const = 0;
+  virtual const AstStorage &GetAstStorage() const = 0;
 };
 
 class Interpreter {
@@ -46,6 +47,7 @@ class Interpreter {
     const auto &plan() const { return plan_->GetRoot(); }
     double cost() const { return plan_->GetCost(); }
     const auto &symbol_table() const { return plan_->GetSymbolTable(); }
+    const auto &ast_storage() const { return plan_->GetAstStorage(); }
 
     bool IsExpired() const {
       return cache_timer_.Elapsed() >
@@ -81,19 +83,33 @@ class Interpreter {
    */
   class Results {
     friend Interpreter;
-    Results(Context ctx, std::shared_ptr<CachedPlan> plan,
-            std::unique_ptr<query::plan::Cursor> cursor,
+    Results(database::GraphDbAccessor *db_accessor,
+            const query::Parameters &parameters,
+            std::shared_ptr<CachedPlan> plan,
             std::vector<Symbol> output_symbols, std::vector<std::string> header,
             std::map<std::string, TypedValue> summary,
-            std::vector<AuthQuery::Privilege> privileges)
-        : ctx_(std::move(ctx)),
+            std::vector<AuthQuery::Privilege> privileges,
+            bool is_profile_query = false)
+        : ctx_(*db_accessor),
           plan_(plan),
-          cursor_(std::move(cursor)),
-          frame_(ctx_.symbol_table_.max_position()),
+          cursor_(plan_->plan().MakeCursor(*db_accessor)),
+          frame_(plan_->symbol_table().max_position()),
           output_symbols_(output_symbols),
           header_(header),
           summary_(summary),
-          privileges_(std::move(privileges)) {}
+          privileges_(std::move(privileges)) {
+      ctx_.is_profile_query_ = is_profile_query;
+      ctx_.symbol_table_ = plan_->symbol_table();
+      ctx_.evaluation_context_.timestamp =
+          std::chrono::duration_cast<std::chrono::milliseconds>(
+              std::chrono::system_clock::now().time_since_epoch())
+              .count();
+      ctx_.evaluation_context_.parameters = parameters;
+      ctx_.evaluation_context_.properties =
+          NamesToProperties(plan_->ast_storage().properties_, db_accessor);
+      ctx_.evaluation_context_.labels =
+          NamesToLabels(plan_->ast_storage().labels_, db_accessor);
+    }
 
    public:
     Results(const Results &) = delete;
@@ -185,7 +201,7 @@ class Interpreter {
 
  protected:
   std::pair<StrippedQuery, ParsedQuery> StripAndParseQuery(
-      const std::string &, Context *, AstStorage *ast_storage,
+      const std::string &, Parameters *, AstStorage *ast_storage,
       database::GraphDbAccessor *,
       const std::map<std::string, PropertyValue> &);
 
diff --git a/src/query/plan/cost_estimator.hpp b/src/query/plan/cost_estimator.hpp
index 7de0a43d3..4e1f40877 100644
--- a/src/query/plan/cost_estimator.hpp
+++ b/src/query/plan/cost_estimator.hpp
@@ -65,18 +65,18 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
   using HierarchicalLogicalOperatorVisitor::PostVisit;
   using HierarchicalLogicalOperatorVisitor::PreVisit;
 
-  CostEstimator(const TDbAccessor &db_accessor, const Parameters &parameters)
+  CostEstimator(TDbAccessor *db_accessor, const Parameters &parameters)
       : db_accessor_(db_accessor), parameters(parameters) {}
 
   bool PostVisit(ScanAll &) override {
-    cardinality_ *= db_accessor_.VerticesCount();
+    cardinality_ *= db_accessor_->VerticesCount();
     // ScanAll performs some work for every element that is produced
     IncrementCost(CostParam::kScanAll);
     return true;
   }
 
   bool PostVisit(ScanAllByLabel &scan_all_by_label) override {
-    cardinality_ *= db_accessor_.VerticesCount(scan_all_by_label.label_);
+    cardinality_ *= db_accessor_->VerticesCount(scan_all_by_label.label_);
     // ScanAll performs some work for every element that is produced
     IncrementCost(CostParam::kScanAllByLabel);
     return true;
@@ -90,13 +90,13 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
     double factor = 1.0;
     if (property_value)
       // get the exact influence based on ScanAll(label, property, value)
-      factor = db_accessor_.VerticesCount(
+      factor = db_accessor_->VerticesCount(
           logical_op.label_, logical_op.property_, property_value.value());
     else
       // estimate the influence as ScanAll(label, property) * filtering
-      factor = db_accessor_.VerticesCount(logical_op.label_,
-                                          logical_op.property_) *
-               CardParam::kFilter;
+      factor =
+          db_accessor_->VerticesCount(logical_op.label_, logical_op.property_) *
+          CardParam::kFilter;
 
     cardinality_ *= factor;
 
@@ -114,12 +114,12 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
     int64_t factor = 1;
     if (upper || lower)
       // if we have either Bound<PropertyValue>, use the value index
-      factor = db_accessor_.VerticesCount(logical_op.label_,
-                                          logical_op.property_, lower, upper);
+      factor = db_accessor_->VerticesCount(logical_op.label_,
+                                           logical_op.property_, lower, upper);
     else
       // no values, but we still have the label
       factor =
-          db_accessor_.VerticesCount(logical_op.label_, logical_op.property_);
+          db_accessor_->VerticesCount(logical_op.label_, logical_op.property_);
 
     // if we failed to take either bound from the op into account, then apply
     // the filtering constant to the factor
@@ -197,7 +197,7 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
   double cardinality_{1};
 
   // accessor used for cardinality estimates in ScanAll and ScanAllByLabel
-  const TDbAccessor &db_accessor_;
+  TDbAccessor *db_accessor_;
   const Parameters &parameters;
 
   void IncrementCost(double param) { cost_ += param * cardinality_; }
@@ -231,7 +231,7 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
 
 /** Returns the estimated cost of the given plan. */
 template <class TDbAccessor>
-double EstimatePlanCost(const TDbAccessor &db, const Parameters &parameters,
+double EstimatePlanCost(TDbAccessor *db, const Parameters &parameters,
                         LogicalOperator &plan) {
   CostEstimator<TDbAccessor> estimator(db, parameters);
   plan.Accept(estimator);
diff --git a/src/query/plan/distributed.cpp b/src/query/plan/distributed.cpp
index 713ee76b4..b8fd2fde3 100644
--- a/src/query/plan/distributed.cpp
+++ b/src/query/plan/distributed.cpp
@@ -139,7 +139,10 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
       auto ident = storage_->Create<Identifier>(
           scan.output_symbol_.name(), scan.output_symbol_.user_declared());
       (*symbol_table_)[*ident] = scan.output_symbol_;
-      return storage_->Create<PropertyLookup>(ident, "", scan.property_);
+      // TODO: When this extraction of a filter is removed, also remove
+      // property_name from ScanAll operators.
+      return storage_->Create<PropertyLookup>(
+          ident, storage_->GetPropertyIx(scan.property_name_));
     };
     Expression *extracted_filter = nullptr;
     std::shared_ptr<ScanAll> new_scan;
@@ -163,7 +166,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
         // Case 1.a)
         new_scan = std::make_shared<ScanAllByLabelPropertyRange>(
             scan.input(), scan.output_symbol_, scan.label_, scan.property_,
-            std::experimental::nullopt, scan.upper_bound_, scan.graph_view_);
+            scan.property_name_, std::experimental::nullopt, scan.upper_bound_,
+            scan.graph_view_);
       }
     }
     if (upper_depends) {
@@ -192,7 +196,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
           // Case 1.a)
           new_scan = std::make_shared<ScanAllByLabelPropertyRange>(
               scan.input(), scan.output_symbol_, scan.label_, scan.property_,
-              scan.lower_bound_, std::experimental::nullopt, scan.graph_view_);
+              scan.property_name_, scan.lower_bound_,
+              std::experimental::nullopt, scan.graph_view_);
         } else {
           // Case 1.b)
           new_scan = std::make_shared<ScanAllByLabel>(
@@ -231,8 +236,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
       auto ident = storage_->Create<Identifier>(
           scan.output_symbol_.name(), scan.output_symbol_.user_declared());
       (*symbol_table_)[*ident] = scan.output_symbol_;
-      auto prop_lookup =
-          storage_->Create<PropertyLookup>(ident, "", scan.property_);
+      auto prop_lookup = storage_->Create<PropertyLookup>(
+          ident, storage_->GetPropertyIx(scan.property_name_));
       auto prop_equal =
           storage_->Create<EqualOperator>(prop_lookup, scan.expression_);
       auto parent = std::make_shared<Filter>(subtree, prop_equal);
@@ -1630,9 +1635,12 @@ class DistributedPlanner : public HierarchicalLogicalOperatorVisitor {
 
 }  // namespace
 
-DistributedPlan MakeDistributedPlan(const LogicalOperator &original_plan,
-                                    const SymbolTable &symbol_table,
-                                    std::atomic<int64_t> &next_plan_id) {
+DistributedPlan MakeDistributedPlan(
+    const AstStorage &ast_storage, const LogicalOperator &original_plan,
+    const SymbolTable &symbol_table, std::atomic<int64_t> &next_plan_id,
+    // TODO: Remove this variable when we remove creating a `PropertyLookup`
+    // expression during distributed planning.
+    const std::vector<storage::Property> &properties_by_ix) {
   DistributedPlan distributed_plan;
   // If we will generate multiple worker plans, we will need to increment the
   // next_plan_id for each one.
diff --git a/src/query/plan/distributed.hpp b/src/query/plan/distributed.hpp
index 828c4f268..55625f7c2 100644
--- a/src/query/plan/distributed.hpp
+++ b/src/query/plan/distributed.hpp
@@ -23,8 +23,9 @@ struct DistributedPlan {
 };
 
 /// Creates a `DistributedPlan` from a regular plan.
-DistributedPlan MakeDistributedPlan(const LogicalOperator &plan,
-                                    const SymbolTable &symbol_table,
-                                    std::atomic<int64_t> &next_plan_id);
+DistributedPlan MakeDistributedPlan(
+    const AstStorage &ast_storage, const LogicalOperator &plan,
+    const SymbolTable &symbol_table, std::atomic<int64_t> &next_plan_id,
+    const std::vector<storage::Property> &properties_by_ix);
 
 }  // namespace query::plan
diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp
index 5a9123f79..4ad5f3ca8 100644
--- a/src/query/plan/operator.cpp
+++ b/src/query/plan/operator.cpp
@@ -346,11 +346,13 @@ std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(
 ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
     const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
     storage::Label label, storage::Property property,
+    const std::string &property_name,
     std::experimental::optional<Bound> lower_bound,
     std::experimental::optional<Bound> upper_bound, GraphView graph_view)
     : ScanAll(input, output_symbol, graph_view),
       label_(label),
       property_(property),
+      property_name_(property_name),
       lower_bound_(lower_bound),
       upper_bound_(upper_bound) {
   DCHECK(lower_bound_ || upper_bound_) << "Only one bound can be left out";
@@ -397,11 +399,13 @@ std::unique_ptr<Cursor> ScanAllByLabelPropertyRange::MakeCursor(
 
 ScanAllByLabelPropertyValue::ScanAllByLabelPropertyValue(
     const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
-    storage::Label label, storage::Property property, Expression *expression,
+    storage::Label label, storage::Property property,
+    const std::string &property_name, Expression *expression,
     GraphView graph_view)
     : ScanAll(input, output_symbol, graph_view),
       label_(label),
       property_(property),
+      property_name_(property_name),
       expression_(expression) {
   DCHECK(expression) << "Expression is not optional.";
 }
@@ -1833,8 +1837,9 @@ void Delete::DeleteCursor::Shutdown() { input_cursor_->Shutdown(); }
 void Delete::DeleteCursor::Reset() { input_cursor_->Reset(); }
 
 SetProperty::SetProperty(const std::shared_ptr<LogicalOperator> &input,
-                         PropertyLookup *lhs, Expression *rhs)
-    : input_(input), lhs_(lhs), rhs_(rhs) {}
+                         storage::Property property, PropertyLookup *lhs,
+                         Expression *rhs)
+    : input_(input), property_(property), lhs_(lhs), rhs_(rhs) {}
 
 ACCEPT_WITH_INPUT(SetProperty)
 
@@ -1866,18 +1871,17 @@ bool SetProperty::SetPropertyCursor::Pull(Frame &frame, Context &context) {
 
   switch (lhs.type()) {
     case TypedValue::Type::Vertex:
-      PropsSetChecked(&lhs.Value<VertexAccessor>(), self_.lhs_->property_, rhs);
+      PropsSetChecked(&lhs.Value<VertexAccessor>(), self_.property_, rhs);
       break;
     case TypedValue::Type::Edge:
-      PropsSetChecked(&lhs.Value<EdgeAccessor>(), self_.lhs_->property_, rhs);
+      PropsSetChecked(&lhs.Value<EdgeAccessor>(), self_.property_, rhs);
       break;
     case TypedValue::Type::Null:
       // Skip setting properties on Null (can occur in optional match).
       break;
     case TypedValue::Type::Map:
     // Semantically modifying a map makes sense, but it's not supported due
-    // to
-    // all the copying we do (when PropertyValue -> TypedValue and in
+    // to all the copying we do (when PropertyValue -> TypedValue and in
     // ExpressionEvaluator). So even though we set a map property here, that
     // is never visible to the user and it's not stored.
     // TODO: fix above described bug
@@ -2041,8 +2045,8 @@ void SetLabels::SetLabelsCursor::Shutdown() { input_cursor_->Shutdown(); }
 void SetLabels::SetLabelsCursor::Reset() { input_cursor_->Reset(); }
 
 RemoveProperty::RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
-                               PropertyLookup *lhs)
-    : input_(input), lhs_(lhs) {}
+                               storage::Property property, PropertyLookup *lhs)
+    : input_(input), property_(property), lhs_(lhs) {}
 
 ACCEPT_WITH_INPUT(RemoveProperty)
 
@@ -2075,7 +2079,7 @@ bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
   switch (lhs.type()) {
     case TypedValue::Type::Vertex:
       try {
-        lhs.Value<VertexAccessor>().PropsErase(self_.lhs_->property_);
+        lhs.Value<VertexAccessor>().PropsErase(self_.property_);
       } catch (const RecordDeletedError &) {
         throw QueryRuntimeException(
             "Trying to remove properties from a deleted node.");
@@ -2083,7 +2087,7 @@ bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
       break;
     case TypedValue::Type::Edge:
       try {
-        lhs.Value<EdgeAccessor>().PropsErase(self_.lhs_->property_);
+        lhs.Value<EdgeAccessor>().PropsErase(self_.property_);
       } catch (const RecordDeletedError &) {
         throw QueryRuntimeException(
             "Trying to remove properties from a deleted edge.");
diff --git a/src/query/plan/operator.lcp b/src/query/plan/operator.lcp
index 87085443a..9f629982e 100644
--- a/src/query/plan/operator.lcp
+++ b/src/query/plan/operator.lcp
@@ -777,6 +777,7 @@ given label.
 (lcp:define-class scan-all-by-label-property-range (scan-all)
   ((label "storage::Label" :scope :public)
    (property "storage::Property" :scope :public)
+   (property-name "std::string" :scope :public)
    (lower-bound "std::experimental::optional<Bound>" :scope :public
                 :capnp-type "Utils.Optional(Utils.Bound(Ast.Tree))"
                 :slk-save #'slk-save-optional-bound
@@ -816,6 +817,7 @@ property value which is inside a range (inclusive or exlusive).
    ScanAllByLabelPropertyRange(const std::shared_ptr<LogicalOperator> &input,
                                Symbol output_symbol, storage::Label label,
                                storage::Property property,
+                               const std::string &property_name,
                                std::experimental::optional<Bound> lower_bound,
                                std::experimental::optional<Bound> upper_bound,
                                GraphView graph_view = GraphView::OLD);
@@ -829,6 +831,7 @@ property value which is inside a range (inclusive or exlusive).
 (lcp:define-class scan-all-by-label-property-value (scan-all)
   ((label "storage::Label" :scope :public)
    (property "storage::Property" :scope :public)
+   (property-name "std::string" :scope :public)
    (expression "Expression *" :scope :public
                :slk-save #'slk-save-ast-pointer
                :slk-load (slk-load-ast-pointer "Expression")
@@ -857,6 +860,7 @@ property value.
    ScanAllByLabelPropertyValue(const std::shared_ptr<LogicalOperator> &input,
                                Symbol output_symbol, storage::Label label,
                                storage::Property property,
+                               const std::string &property_name,
                                Expression *expression,
                                GraphView graph_view = GraphView::OLD);
 
@@ -1386,6 +1390,7 @@ Has a flag for using DETACH DELETE when deleting vertices.")
           :slk-load #'slk-load-operator-pointer
           :capnp-save #'save-operator-pointer
           :capnp-load #'load-operator-pointer)
+   (property "storage::Property" :scope :public)
    (lhs "PropertyLookup *" :scope :public
         :slk-save #'slk-save-ast-pointer
         :slk-load (slk-load-ast-pointer "PropertyLookup")
@@ -1406,7 +1411,8 @@ can be stored (a TypedValue that can be converted to PropertyValue).")
    SetProperty() {}
 
    SetProperty(const std::shared_ptr<LogicalOperator> &input,
-               PropertyLookup *lhs, Expression *rhs);
+               storage::Property property, PropertyLookup *lhs,
+               Expression *rhs);
    bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
    std::unique_ptr<Cursor> MakeCursor(
        database::GraphDbAccessor &db) const override;
@@ -1560,6 +1566,7 @@ It does NOT remove labels that are already set on that Vertex.")
           :slk-load #'slk-load-operator-pointer
           :capnp-save #'save-operator-pointer
           :capnp-load #'load-operator-pointer)
+   (property "storage::Property" :scope :public)
    (lhs "PropertyLookup *" :scope :public
         :slk-save #'slk-save-ast-pointer
         :slk-load (slk-load-ast-pointer "PropertyLookup")
@@ -1572,7 +1579,7 @@ It does NOT remove labels that are already set on that Vertex.")
    RemoveProperty() {}
 
    RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
-                  PropertyLookup *lhs);
+                  storage::Property property, PropertyLookup *lhs);
    bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
    std::unique_ptr<Cursor> MakeCursor(
        database::GraphDbAccessor &db) const override;
diff --git a/src/query/plan/planner.hpp b/src/query/plan/planner.hpp
index 7e5cfe6ae..9b6039ddb 100644
--- a/src/query/plan/planner.hpp
+++ b/src/query/plan/planner.hpp
@@ -36,9 +36,10 @@ namespace plan {
 template <template <class> class TPlanner, class TDbAccessor>
 auto MakeLogicalPlanForSingleQuery(
     std::vector<SingleQueryPart> single_query_parts,
-    PlanningContext<TDbAccessor> &context) {
-  context.bound_symbols.clear();
-  return TPlanner<decltype(context)>(context).Plan(single_query_parts);
+    PlanningContext<TDbAccessor> *context) {
+  context->bound_symbols.clear();
+  return TPlanner<PlanningContext<TDbAccessor>>(context).Plan(
+      single_query_parts);
 }
 
 /// Generates the LogicalOperator tree and returns the resulting plan.
@@ -50,11 +51,11 @@ auto MakeLogicalPlanForSingleQuery(
 /// @return pair consisting of the plan's first logical operator @c
 /// LogicalOperator and the estimated cost of that plan
 template <class TPlanningContext>
-auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
+auto MakeLogicalPlan(TPlanningContext *context, const Parameters &parameters,
                      bool use_variable_planner) {
-  auto query_parts = CollectQueryParts(context.symbol_table,
-                                       context.ast_storage, context.query);
-  auto &vertex_counts = context.db;
+  auto query_parts = CollectQueryParts(*context->symbol_table,
+                                       *context->ast_storage, context->query);
+  auto &vertex_counts = *context->db;
   double total_cost = 0;
   std::unique_ptr<LogicalOperator> last_op;
 
@@ -66,7 +67,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
       auto plans = MakeLogicalPlanForSingleQuery<VariableStartPlanner>(
           query_part.single_query_parts, context);
       for (auto plan : plans) {
-        auto cost = EstimatePlanCost(vertex_counts, parameters, *plan);
+        auto cost = EstimatePlanCost(&vertex_counts, parameters, *plan);
         if (!op || cost < min_cost) {
           // Plans are generated lazily and the current plan will disappear, so
           // it's ok to move it.
@@ -77,7 +78,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
     } else {
       op = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
           query_part.single_query_parts, context);
-      min_cost = EstimatePlanCost(vertex_counts, parameters, *op);
+      min_cost = EstimatePlanCost(&vertex_counts, parameters, *op);
     }
 
     total_cost += min_cost;
@@ -86,7 +87,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
       std::shared_ptr<LogicalOperator> curr_op(std::move(op));
       std::shared_ptr<LogicalOperator> prev_op(std::move(last_op));
       last_op = std::unique_ptr<LogicalOperator>(
-          impl::GenUnion(*union_, prev_op, curr_op, context.symbol_table));
+          impl::GenUnion(*union_, prev_op, curr_op, *context->symbol_table));
     } else if (query_part.query_combinator) {
       throw utils::NotYetImplemented("query combinator");
     } else {
@@ -97,7 +98,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
   if (query_parts.distinct) {
     std::shared_ptr<LogicalOperator> prev_op(std::move(last_op));
     last_op = std::make_unique<Distinct>(
-        prev_op, prev_op->OutputSymbols(context.symbol_table));
+        prev_op, prev_op->OutputSymbols(*context->symbol_table));
   }
 
   return std::make_pair(std::move(last_op), total_cost);
diff --git a/src/query/plan/preprocess.cpp b/src/query/plan/preprocess.cpp
index 7db95378b..62e777dc9 100644
--- a/src/query/plan/preprocess.cpp
+++ b/src/query/plan/preprocess.cpp
@@ -152,7 +152,7 @@ auto SplitExpressionOnAnd(Expression *expression) {
 }  // namespace
 
 PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
-                               const Symbol &symbol, storage::Property property,
+                               const Symbol &symbol, PropertyIx property,
                                Expression *value)
     : symbol_(symbol), property_(property), value_(value) {
   UsedSymbolsCollector collector(symbol_table);
@@ -162,7 +162,7 @@ PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
 
 PropertyFilter::PropertyFilter(
     const SymbolTable &symbol_table, const Symbol &symbol,
-    storage::Property property,
+    PropertyIx property,
     const std::experimental::optional<PropertyFilter::Bound> &lower_bound,
     const std::experimental::optional<PropertyFilter::Bound> &upper_bound)
     : symbol_(symbol),
@@ -189,7 +189,7 @@ void Filters::EraseFilter(const FilterInfo &filter) {
                      all_filters_.end());
 }
 
-void Filters::EraseLabelFilter(const Symbol &symbol, storage::Label label) {
+void Filters::EraseLabelFilter(const Symbol &symbol, LabelIx label) {
   for (auto filter_it = all_filters_.begin();
        filter_it != all_filters_.end();) {
     if (filter_it->type != FilterInfo::Type::Label) {
@@ -282,7 +282,7 @@ void Filters::CollectPatternFilters(Pattern &pattern, SymbolTable &symbol_table,
                              collector.symbols_};
       // Store a PropertyFilter on the value of the property.
       filter_info.property_filter.emplace(
-          symbol_table, symbol, prop_pair.first.second, prop_pair.second);
+          symbol_table, symbol, prop_pair.first, prop_pair.second);
       all_filters_.emplace_back(filter_info);
     }
   };
diff --git a/src/query/plan/preprocess.hpp b/src/query/plan/preprocess.hpp
index 92a65bac3..b7f9262a1 100644
--- a/src/query/plan/preprocess.hpp
+++ b/src/query/plan/preprocess.hpp
@@ -80,15 +80,14 @@ class PropertyFilter {
  public:
   using Bound = ScanAllByLabelPropertyRange::Bound;
 
-  PropertyFilter(const SymbolTable &, const Symbol &, storage::Property,
-                 Expression *);
-  PropertyFilter(const SymbolTable &, const Symbol &, storage::Property,
+  PropertyFilter(const SymbolTable &, const Symbol &, PropertyIx, Expression *);
+  PropertyFilter(const SymbolTable &, const Symbol &, PropertyIx,
                  const std::experimental::optional<Bound> &,
                  const std::experimental::optional<Bound> &);
 
   /// Symbol whose property is looked up.
   Symbol symbol_;
-  storage::Property property_;
+  PropertyIx property_;
   /// True if the same symbol is used in expressions for value or bounds.
   bool is_symbol_in_value_ = false;
   /// Expression which when evaluated produces the value a property must
@@ -113,7 +112,7 @@ struct FilterInfo {
   /// Set of used symbols by the filter @c expression.
   std::unordered_set<Symbol> used_symbols;
   /// Labels for Type::Label filtering.
-  std::vector<storage::Label> labels;
+  std::vector<LabelIx> labels;
   /// Property information for Type::Property filtering.
   std::experimental::optional<PropertyFilter> property_filter;
 };
@@ -144,7 +143,7 @@ class Filters {
   }
 
   auto FilteredLabels(const Symbol &symbol) const {
-    std::unordered_set<storage::Label> labels;
+    std::unordered_set<LabelIx> labels;
     for (const auto &filter : all_filters_) {
       if (filter.type == FilterInfo::Type::Label &&
           utils::Contains(filter.used_symbols, symbol)) {
@@ -162,7 +161,7 @@ class Filters {
   void EraseFilter(const FilterInfo &);
 
   // Remove a label filter for symbol; may invalidate iterators.
-  void EraseLabelFilter(const Symbol &, storage::Label);
+  void EraseLabelFilter(const Symbol &, LabelIx);
 
   // Returns a vector of FilterInfo for properties.
   auto PropertyFilters(const Symbol &symbol) const {
diff --git a/src/query/plan/rule_based_planner.cpp b/src/query/plan/rule_based_planner.cpp
index 2fd2435d6..014e04883 100644
--- a/src/query/plan/rule_based_planner.cpp
+++ b/src/query/plan/rule_based_planner.cpp
@@ -21,58 +21,6 @@ namespace query::plan {
 
 namespace {
 
-/// Utility function for iterating pattern atoms and accumulating a result.
-///
-/// Each pattern is of the form `NodeAtom (, EdgeAtom, NodeAtom)*`. Therefore,
-/// the `base` function is called on the first `NodeAtom`, while the `collect`
-/// is called for the whole triplet. Result of the function is passed to the
-/// next call. Final result is returned.
-///
-/// Example usage of counting edge atoms in the pattern.
-///
-///    auto base = [](NodeAtom *first_node) { return 0; };
-///    auto collect = [](int accum, NodeAtom *prev_node, EdgeAtom *edge,
-///                      NodeAtom *node) {
-///      return accum + 1;
-///    };
-///    int edge_count = ReducePattern<int>(pattern, base, collect);
-///
-// TODO: It might be a good idea to move this somewhere else, for easier usage
-// in other files.
-template <typename T>
-auto ReducePattern(
-    Pattern &pattern, std::function<T(NodeAtom *)> base,
-    std::function<T(T, NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
-  DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
-  auto atoms_it = pattern.atoms_.begin();
-  auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
-  DCHECK(current_node) << "First pattern atom is not a node";
-  auto last_res = base(current_node);
-  // Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
-  while (atoms_it != pattern.atoms_.end()) {
-    auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
-    DCHECK(edge) << "Expected an edge atom in pattern.";
-    DCHECK(atoms_it != pattern.atoms_.end())
-        << "Edge atom should not end the pattern.";
-    auto prev_node = current_node;
-    current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
-    DCHECK(current_node) << "Expected a node atom in pattern.";
-    last_res = collect(std::move(last_res), prev_node, edge, current_node);
-  }
-  return last_res;
-}
-
-auto GenCreate(Create &create, std::unique_ptr<LogicalOperator> input_op,
-               const SymbolTable &symbol_table,
-               std::unordered_set<Symbol> &bound_symbols) {
-  auto last_op = std::move(input_op);
-  for (auto pattern : create.patterns_) {
-    last_op = impl::GenCreateForPattern(*pattern, std::move(last_op),
-                                        symbol_table, bound_symbols);
-  }
-  return last_op;
-}
-
 bool HasBoundFilterSymbols(const std::unordered_set<Symbol> &bound_symbols,
                            const FilterInfo &filter) {
   for (const auto &symbol : filter.used_symbols) {
@@ -612,110 +560,6 @@ std::unique_ptr<LogicalOperator> GenReturn(
   return GenReturnBody(std::move(input_op), advance_command, body, accumulate);
 }
 
-std::unique_ptr<LogicalOperator> GenCreateForPattern(
-    Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
-    const SymbolTable &symbol_table,
-    std::unordered_set<Symbol> &bound_symbols) {
-  auto node_to_creation_info = [&](const NodeAtom &node) {
-    const auto &node_symbol = symbol_table.at(*node.identifier_);
-    std::vector<storage::Label> labels(node.labels_);
-    std::vector<std::pair<storage::Property, Expression *>> properties;
-    properties.reserve(node.properties_.size());
-    for (const auto &kv : node.properties_) {
-      properties.push_back({kv.first.second, kv.second});
-    }
-    return NodeCreationInfo{node_symbol, labels, properties};
-  };
-
-  auto base = [&](NodeAtom *node) -> std::unique_ptr<LogicalOperator> {
-    if (bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
-      auto node_info = node_to_creation_info(*node);
-      return std::make_unique<CreateNode>(std::move(input_op), node_info);
-    } else {
-      return std::move(input_op);
-    }
-  };
-
-  auto collect = [&](std::unique_ptr<LogicalOperator> last_op,
-                     NodeAtom *prev_node, EdgeAtom *edge, NodeAtom *node) {
-    // Store the symbol from the first node as the input to CreateExpand.
-    const auto &input_symbol = symbol_table.at(*prev_node->identifier_);
-    // If the expand node was already bound, then we need to indicate this,
-    // so that CreateExpand only creates an edge.
-    bool node_existing = false;
-    if (!bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
-      node_existing = true;
-    }
-    const auto &edge_symbol = symbol_table.at(*edge->identifier_);
-    if (!bound_symbols.insert(edge_symbol).second) {
-      LOG(FATAL) << "Symbols used for created edges cannot be redeclared.";
-    }
-    auto node_info = node_to_creation_info(*node);
-    std::vector<std::pair<storage::Property, Expression *>> properties;
-    properties.reserve(edge->properties_.size());
-    for (const auto &kv : edge->properties_) {
-      properties.push_back({kv.first.second, kv.second});
-    }
-    CHECK(edge->edge_types_.size() == 1)
-        << "Creating an edge with a single type should be required by syntax";
-    EdgeCreationInfo edge_info{edge_symbol, properties, edge->edge_types_[0],
-                               edge->direction_};
-    return std::make_unique<CreateExpand>(
-        node_info, edge_info, std::move(last_op), input_symbol, node_existing);
-  };
-
-  auto last_op =
-      ReducePattern<std::unique_ptr<LogicalOperator>>(pattern, base, collect);
-
-  // If the pattern is named, append the path constructing logical operator.
-  if (pattern.identifier_->user_declared_) {
-    std::vector<Symbol> path_elements;
-    for (const PatternAtom *atom : pattern.atoms_)
-      path_elements.emplace_back(symbol_table.at(*atom->identifier_));
-    last_op = std::make_unique<ConstructNamedPath>(
-        std::move(last_op), symbol_table.at(*pattern.identifier_),
-        path_elements);
-  }
-
-  return last_op;
-}
-
-// Generate an operator for a clause which writes to the database. Ownership of
-// input_op is transferred to the newly created operator. If the clause isn't
-// handled, returns nullptr and input_op is left as is.
-std::unique_ptr<LogicalOperator> HandleWriteClause(
-    Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
-    const SymbolTable &symbol_table,
-    std::unordered_set<Symbol> &bound_symbols) {
-  if (auto *create = dynamic_cast<Create *>(clause)) {
-    return GenCreate(*create, std::move(input_op), symbol_table, bound_symbols);
-  } else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
-    return std::make_unique<plan::Delete>(std::move(input_op),
-                                          del->expressions_, del->detach_);
-  } else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
-    return std::make_unique<plan::SetProperty>(
-        std::move(input_op), set->property_lookup_, set->expression_);
-  } else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
-    auto op = set->update_ ? plan::SetProperties::Op::UPDATE
-                           : plan::SetProperties::Op::REPLACE;
-    const auto &input_symbol = symbol_table.at(*set->identifier_);
-    return std::make_unique<plan::SetProperties>(
-        std::move(input_op), input_symbol, set->expression_, op);
-  } else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
-    const auto &input_symbol = symbol_table.at(*set->identifier_);
-    return std::make_unique<plan::SetLabels>(std::move(input_op), input_symbol,
-                                             set->labels_);
-  } else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
-    return std::make_unique<plan::RemoveProperty>(std::move(input_op),
-                                                  rem->property_lookup_);
-  } else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
-    const auto &input_symbol = symbol_table.at(*rem->identifier_);
-    return std::make_unique<plan::RemoveLabels>(std::move(input_op),
-                                                input_symbol, rem->labels_);
-  }
-  return nullptr;
-}
-
 std::unique_ptr<LogicalOperator> GenWith(
     With &with, std::unique_ptr<LogicalOperator> input_op,
     SymbolTable &symbol_table, bool is_write,
diff --git a/src/query/plan/rule_based_planner.hpp b/src/query/plan/rule_based_planner.hpp
index 51e14a0c9..3a84f282e 100644
--- a/src/query/plan/rule_based_planner.hpp
+++ b/src/query/plan/rule_based_planner.hpp
@@ -20,15 +20,15 @@ struct PlanningContext {
   /// operators.
   ///
   /// Newly created AST nodes may be added to reference existing symbols.
-  SymbolTable &symbol_table;
+  SymbolTable *symbol_table{nullptr};
   /// @brief The storage is used to create new AST nodes for use in operators.
-  AstStorage &ast_storage;
+  AstStorage *ast_storage{nullptr};
   /// @brief Cypher query to be planned
-  CypherQuery *query;
+  CypherQuery *query{nullptr};
   /// @brief TDbAccessor, which may be used to get some information from the
   /// database to generate better plans. The accessor is required only to live
   /// long enough for the plan generation to finish.
-  const TDbAccessor &db;
+  TDbAccessor *db{nullptr};
   /// @brief Symbol set is used to differentiate cycles in pattern matching.
   /// During planning, symbols will be added as each operator produces values
   /// for them. This way, the operator can be correctly initialized whether to
@@ -39,8 +39,8 @@ struct PlanningContext {
 };
 
 template <class TDbAccessor>
-auto MakePlanningContext(AstStorage &ast_storage, SymbolTable &symbol_table,
-                         CypherQuery *query, const TDbAccessor &db) {
+auto MakePlanningContext(AstStorage *ast_storage, SymbolTable *symbol_table,
+                         CypherQuery *query, TDbAccessor *db) {
   return PlanningContext<TDbAccessor>{symbol_table, ast_storage, query, db};
 }
 
@@ -74,6 +74,47 @@ std::unique_ptr<LogicalOperator> GenFilters(std::unique_ptr<LogicalOperator>,
                                             const std::unordered_set<Symbol> &,
                                             Filters &, AstStorage &);
 
+/// Utility function for iterating pattern atoms and accumulating a result.
+///
+/// Each pattern is of the form `NodeAtom (, EdgeAtom, NodeAtom)*`. Therefore,
+/// the `base` function is called on the first `NodeAtom`, while the `collect`
+/// is called for the whole triplet. Result of the function is passed to the
+/// next call. Final result is returned.
+///
+/// Example usage of counting edge atoms in the pattern.
+///
+///    auto base = [](NodeAtom *first_node) { return 0; };
+///    auto collect = [](int accum, NodeAtom *prev_node, EdgeAtom *edge,
+///                      NodeAtom *node) {
+///      return accum + 1;
+///    };
+///    int edge_count = ReducePattern<int>(pattern, base, collect);
+///
+// TODO: It might be a good idea to move this somewhere else, for easier usage
+// in other files.
+template <typename T>
+auto ReducePattern(
+    Pattern &pattern, std::function<T(NodeAtom *)> base,
+    std::function<T(T, NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
+  DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
+  auto atoms_it = pattern.atoms_.begin();
+  auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
+  DCHECK(current_node) << "First pattern atom is not a node";
+  auto last_res = base(current_node);
+  // Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
+  while (atoms_it != pattern.atoms_.end()) {
+    auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
+    DCHECK(edge) << "Expected an edge atom in pattern.";
+    DCHECK(atoms_it != pattern.atoms_.end())
+        << "Edge atom should not end the pattern.";
+    auto prev_node = current_node;
+    current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
+    DCHECK(current_node) << "Expected a node atom in pattern.";
+    last_res = collect(std::move(last_res), prev_node, edge, current_node);
+  }
+  return last_res;
+}
+
 // For all given `named_paths` checks if all its symbols have been bound.
 // If so, it creates a logical operator for named path generation, binds its
 // symbol, removes that path from the collection of unhandled ones and returns
@@ -88,14 +129,6 @@ std::unique_ptr<LogicalOperator> GenReturn(
     SymbolTable &symbol_table, bool is_write,
     const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage);
 
-std::unique_ptr<LogicalOperator> GenCreateForPattern(
-    Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
-    const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
-
-std::unique_ptr<LogicalOperator> HandleWriteClause(
-    Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
-    const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
-
 std::unique_ptr<LogicalOperator> GenWith(
     With &with, std::unique_ptr<LogicalOperator> input_op,
     SymbolTable &symbol_table, bool is_write,
@@ -122,23 +155,23 @@ Expression *BoolJoin(AstStorage &storage, Expression *expr1,
 template <class TPlanningContext>
 class RuleBasedPlanner {
  public:
-  explicit RuleBasedPlanner(TPlanningContext &context) : context_(context) {}
+  explicit RuleBasedPlanner(TPlanningContext *context) : context_(context) {}
 
   /// @brief The result of plan generation is the root of the generated operator
   /// tree.
   using PlanResult = std::unique_ptr<LogicalOperator>;
   /// @brief Generates the operator tree based on explicitly set rules.
   PlanResult Plan(const std::vector<SingleQueryPart> &query_parts) {
-    auto &context = context_;
+    auto &context = *context_;
     std::unique_ptr<LogicalOperator> input_op;
     // Set to true if a query command writes to the database.
     bool is_write = false;
     for (const auto &query_part : query_parts) {
-      MatchContext match_ctx{query_part.matching, context.symbol_table,
+      MatchContext match_ctx{query_part.matching, *context.symbol_table,
                              context.bound_symbols};
       input_op = PlanMatching(match_ctx, std::move(input_op));
       for (const auto &matching : query_part.optional_matching) {
-        MatchContext opt_ctx{matching, context.symbol_table,
+        MatchContext opt_ctx{matching, *context.symbol_table,
                              context.bound_symbols};
         auto match_op = PlanMatching(opt_ctx, nullptr);
         if (match_op) {
@@ -152,8 +185,8 @@ class RuleBasedPlanner {
             << "Unexpected Match in remaining clauses";
         if (auto *ret = dynamic_cast<Return *>(clause)) {
           input_op = impl::GenReturn(
-              *ret, std::move(input_op), context.symbol_table, is_write,
-              context.bound_symbols, context.ast_storage);
+              *ret, std::move(input_op), *context.symbol_table, is_write,
+              context.bound_symbols, *context.ast_storage);
         } else if (auto *merge = dynamic_cast<query::Merge *>(clause)) {
           input_op = GenMerge(*merge, std::move(input_op),
                               query_part.merge_matching[merge_id++]);
@@ -162,18 +195,18 @@ class RuleBasedPlanner {
           is_write = true;
         } else if (auto *with = dynamic_cast<query::With *>(clause)) {
           input_op = impl::GenWith(*with, std::move(input_op),
-                                   context.symbol_table, is_write,
-                                   context.bound_symbols, context.ast_storage);
+                                   *context.symbol_table, is_write,
+                                   context.bound_symbols, *context.ast_storage);
           // WITH clause advances the command, so reset the flag.
           is_write = false;
-        } else if (auto op = impl::HandleWriteClause(clause, input_op,
-                                                     context.symbol_table,
-                                                     context.bound_symbols)) {
+        } else if (auto op = HandleWriteClause(clause, input_op,
+                                               *context.symbol_table,
+                                               context.bound_symbols)) {
           is_write = true;
           input_op = std::move(op);
         } else if (auto *unwind = dynamic_cast<query::Unwind *>(clause)) {
           const auto &symbol =
-              context.symbol_table.at(*unwind->named_expression_);
+              context.symbol_table->at(*unwind->named_expression_);
           context.bound_symbols.insert(symbol);
           input_op = std::make_unique<plan::Unwind>(
               std::move(input_op), unwind->named_expression_->expression_,
@@ -187,15 +220,164 @@ class RuleBasedPlanner {
   }
 
  private:
-  TPlanningContext &context_;
+  TPlanningContext *context_;
 
   struct LabelPropertyIndex {
-    storage::Label label;
+    LabelIx label;
     // FilterInfo with PropertyFilter.
     FilterInfo filter;
     int64_t vertex_count;
   };
 
+  storage::Label GetLabel(LabelIx label) {
+    return context_->db->Label(context_->ast_storage->labels_[label.ix]);
+  }
+
+  storage::Property GetProperty(PropertyIx prop) {
+    return context_->db->Property(context_->ast_storage->properties_[prop.ix]);
+  }
+
+  storage::EdgeType GetEdgeType(EdgeTypeIx edge_type) {
+    return context_->db->EdgeType(
+        context_->ast_storage->edge_types_[edge_type.ix]);
+  }
+
+  std::unique_ptr<LogicalOperator> GenCreate(
+      Create &create, std::unique_ptr<LogicalOperator> input_op,
+      const SymbolTable &symbol_table,
+      std::unordered_set<Symbol> &bound_symbols) {
+    auto last_op = std::move(input_op);
+    for (auto pattern : create.patterns_) {
+      last_op = GenCreateForPattern(*pattern, std::move(last_op), symbol_table,
+                                    bound_symbols);
+    }
+    return last_op;
+  }
+
+  std::unique_ptr<LogicalOperator> GenCreateForPattern(
+      Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
+      const SymbolTable &symbol_table,
+      std::unordered_set<Symbol> &bound_symbols) {
+    auto node_to_creation_info = [&](const NodeAtom &node) {
+      const auto &node_symbol = symbol_table.at(*node.identifier_);
+      std::vector<storage::Label> labels;
+      labels.reserve(node.labels_.size());
+      for (const auto &label : node.labels_) {
+        labels.push_back(GetLabel(label));
+      }
+      std::vector<std::pair<storage::Property, Expression *>> properties;
+      properties.reserve(node.properties_.size());
+      for (const auto &kv : node.properties_) {
+        properties.push_back({GetProperty(kv.first), kv.second});
+      }
+      return NodeCreationInfo{node_symbol, labels, properties};
+    };
+
+    auto base = [&](NodeAtom *node) -> std::unique_ptr<LogicalOperator> {
+      const auto &node_symbol = symbol_table.at(*node->identifier_);
+      if (bound_symbols.insert(node_symbol).second) {
+        auto node_info = node_to_creation_info(*node);
+        return std::make_unique<CreateNode>(std::move(input_op), node_info);
+      } else {
+        return std::move(input_op);
+      }
+    };
+
+    auto collect = [&](std::unique_ptr<LogicalOperator> last_op,
+                       NodeAtom *prev_node, EdgeAtom *edge, NodeAtom *node) {
+      // Store the symbol from the first node as the input to CreateExpand.
+      const auto &input_symbol = symbol_table.at(*prev_node->identifier_);
+      // If the expand node was already bound, then we need to indicate this,
+      // so that CreateExpand only creates an edge.
+      bool node_existing = false;
+      if (!bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
+        node_existing = true;
+      }
+      const auto &edge_symbol = symbol_table.at(*edge->identifier_);
+      if (!bound_symbols.insert(edge_symbol).second) {
+        LOG(FATAL) << "Symbols used for created edges cannot be redeclared.";
+      }
+      auto node_info = node_to_creation_info(*node);
+      std::vector<std::pair<storage::Property, Expression *>> properties;
+      properties.reserve(edge->properties_.size());
+      for (const auto &kv : edge->properties_) {
+        properties.push_back({GetProperty(kv.first), kv.second});
+      }
+      CHECK(edge->edge_types_.size() == 1)
+          << "Creating an edge with a single type should be required by syntax";
+      EdgeCreationInfo edge_info{edge_symbol, properties,
+                                 GetEdgeType(edge->edge_types_[0]),
+                                 edge->direction_};
+      return std::make_unique<CreateExpand>(node_info, edge_info,
+                                            std::move(last_op), input_symbol,
+                                            node_existing);
+    };
+
+    auto last_op = impl::ReducePattern<std::unique_ptr<LogicalOperator>>(
+        pattern, base, collect);
+
+    // If the pattern is named, append the path constructing logical operator.
+    if (pattern.identifier_->user_declared_) {
+      std::vector<Symbol> path_elements;
+      for (const PatternAtom *atom : pattern.atoms_)
+        path_elements.emplace_back(symbol_table.at(*atom->identifier_));
+      last_op = std::make_unique<ConstructNamedPath>(
+          std::move(last_op), symbol_table.at(*pattern.identifier_),
+          path_elements);
+    }
+
+    return last_op;
+  }
+
+  // Generate an operator for a clause which writes to the database. Ownership
+  // of input_op is transferred to the newly created operator. If the clause
+  // isn't handled, returns nullptr and input_op is left as is.
+  std::unique_ptr<LogicalOperator> HandleWriteClause(
+      Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
+      const SymbolTable &symbol_table,
+      std::unordered_set<Symbol> &bound_symbols) {
+    if (auto *create = dynamic_cast<Create *>(clause)) {
+      return GenCreate(*create, std::move(input_op), symbol_table,
+                       bound_symbols);
+    } else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
+      return std::make_unique<plan::Delete>(std::move(input_op),
+                                            del->expressions_, del->detach_);
+    } else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
+      return std::make_unique<plan::SetProperty>(
+          std::move(input_op), GetProperty(set->property_lookup_->property_),
+          set->property_lookup_, set->expression_);
+    } else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
+      auto op = set->update_ ? plan::SetProperties::Op::UPDATE
+                             : plan::SetProperties::Op::REPLACE;
+      const auto &input_symbol = symbol_table.at(*set->identifier_);
+      return std::make_unique<plan::SetProperties>(
+          std::move(input_op), input_symbol, set->expression_, op);
+    } else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
+      const auto &input_symbol = symbol_table.at(*set->identifier_);
+      std::vector<storage::Label> labels;
+      labels.reserve(set->labels_.size());
+      for (const auto &label : set->labels_) {
+        labels.push_back(GetLabel(label));
+      }
+      return std::make_unique<plan::SetLabels>(std::move(input_op),
+                                               input_symbol, labels);
+    } else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
+      return std::make_unique<plan::RemoveProperty>(
+          std::move(input_op), GetProperty(rem->property_lookup_->property_),
+          rem->property_lookup_);
+    } else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
+      const auto &input_symbol = symbol_table.at(*rem->identifier_);
+      std::vector<storage::Label> labels;
+      labels.reserve(rem->labels_.size());
+      for (const auto &label : rem->labels_) {
+        labels.push_back(GetLabel(label));
+      }
+      return std::make_unique<plan::RemoveLabels>(std::move(input_op),
+                                                  input_symbol, labels);
+    }
+    return nullptr;
+  }
+
   // Finds the label-property combination which has indexed the lowest amount of
   // vertices. If the index cannot be found, nullopt is returned.
   std::experimental::optional<LabelPropertyIndex> FindBestLabelPropertyIndex(
@@ -213,8 +395,10 @@ class RuleBasedPlanner {
     for (const auto &label : filters.FilteredLabels(symbol)) {
       for (const auto &filter : filters.PropertyFilters(symbol)) {
         const auto &property = filter.property_filter->property_;
-        if (context_.db.LabelPropertyIndexExists(label, property)) {
-          int64_t vertex_count = context_.db.VerticesCount(label, property);
+        if (context_->db->LabelPropertyIndexExists(GetLabel(label),
+                                                   GetProperty(property))) {
+          int64_t vertex_count = context_->db->VerticesCount(
+              GetLabel(label), GetProperty(property));
           if (!found || vertex_count < found->vertex_count) {
             if (filter.property_filter->is_symbol_in_value_) {
               // Skip filter expressions which use the symbol whose property
@@ -234,15 +418,15 @@ class RuleBasedPlanner {
     return found;
   }
 
-  storage::Label FindBestLabelIndex(
-      const std::unordered_set<storage::Label> &labels) {
-    DCHECK(!labels.empty())
+  LabelIx FindBestLabelIndex(const std::unordered_set<LabelIx> &labels) {
+    CHECK(!labels.empty())
         << "Trying to find the best label without any labels.";
-    return *std::min_element(labels.begin(), labels.end(),
-                             [this](const auto &label1, const auto &label2) {
-                               return context_.db.VerticesCount(label1) <
-                                      context_.db.VerticesCount(label2);
-                             });
+    return *std::min_element(
+        labels.begin(), labels.end(),
+        [this](const auto &label1, const auto &label2) {
+          return context_->db->VerticesCount(GetLabel(label1)) <
+                 context_->db->VerticesCount(GetLabel(label2));
+        });
   }
 
   // Creates a ScanAll by the best possible index for the `node_symbol`. Best
@@ -275,33 +459,35 @@ class RuleBasedPlanner {
       filters.EraseLabelFilter(node_symbol, found_index->label);
       if (prop_filter.lower_bound_ || prop_filter.upper_bound_) {
         return std::make_unique<ScanAllByLabelPropertyRange>(
-            std::move(last_op), node_symbol, found_index->label,
-            prop_filter.property_, prop_filter.lower_bound_,
-            prop_filter.upper_bound_, match_ctx.graph_view);
+            std::move(last_op), node_symbol, GetLabel(found_index->label),
+            GetProperty(prop_filter.property_), prop_filter.property_.name,
+            prop_filter.lower_bound_, prop_filter.upper_bound_,
+            match_ctx.graph_view);
       } else {
         DCHECK(prop_filter.value_) << "Property filter should either have "
                                       "bounds or a value expression.";
         return std::make_unique<ScanAllByLabelPropertyValue>(
-            std::move(last_op), node_symbol, found_index->label,
-            prop_filter.property_, prop_filter.value_, match_ctx.graph_view);
+            std::move(last_op), node_symbol, GetLabel(found_index->label),
+            GetProperty(prop_filter.property_), prop_filter.property_.name,
+            prop_filter.value_, match_ctx.graph_view);
       }
     }
     auto label = FindBestLabelIndex(labels);
     if (max_vertex_count &&
-        context_.db.VerticesCount(label) > *max_vertex_count) {
+        context_->db->VerticesCount(GetLabel(label)) > *max_vertex_count) {
       // Don't create an indexed lookup, since we have more labeled vertices
       // than the allowed count.
       return nullptr;
     }
     filters.EraseLabelFilter(node_symbol, label);
-    return std::make_unique<ScanAllByLabel>(std::move(last_op), node_symbol,
-                                            label, match_ctx.graph_view);
+    return std::make_unique<ScanAllByLabel>(
+        std::move(last_op), node_symbol, GetLabel(label), match_ctx.graph_view);
   }
 
   std::unique_ptr<LogicalOperator> PlanMatching(
       MatchContext &match_context, std::unique_ptr<LogicalOperator> input_op) {
     auto &bound_symbols = match_context.bound_symbols;
-    auto &storage = context_.ast_storage;
+    auto &storage = *context_->ast_storage;
     const auto &symbol_table = match_context.symbol_table;
     const auto &matching = match_context.matching;
     // Copy filters, because we will modify them as we generate Filters.
@@ -347,6 +533,11 @@ class RuleBasedPlanner {
         const auto &edge_symbol = symbol_table.at(*edge->identifier_);
         DCHECK(!utils::Contains(bound_symbols, edge_symbol))
             << "Existing edges are not supported";
+        std::vector<storage::EdgeType> edge_types;
+        edge_types.reserve(edge->edge_types_.size());
+        for (const auto &type : edge->edge_types_) {
+          edge_types.push_back(GetEdgeType(type));
+        }
         if (edge->IsVariable()) {
           std::experimental::optional<ExpansionLambda> weight_lambda;
           std::experimental::optional<Symbol> total_weight;
@@ -405,7 +596,7 @@ class RuleBasedPlanner {
           // TODO: Pass weight lambda.
           last_op = std::make_unique<ExpandVariable>(
               std::move(last_op), node1_symbol, node_symbol, edge_symbol,
-              edge->type_, expansion.direction, edge->edge_types_,
+              edge->type_, expansion.direction, edge_types,
               expansion.is_flipped, edge->lower_bound_, edge->upper_bound_,
               existing_node, filter_lambda, weight_lambda, total_weight,
               match_context.graph_view);
@@ -428,7 +619,7 @@ class RuleBasedPlanner {
           }
           last_op = std::make_unique<Expand>(
               std::move(last_op), node1_symbol, node_symbol, edge_symbol,
-              expansion.direction, edge->edge_types_, existing_node,
+              expansion.direction, edge_types, existing_node,
               match_context.graph_view);
         }
 
@@ -481,22 +672,22 @@ class RuleBasedPlanner {
                 const Matching &matching) {
     // Copy the bound symbol set, because we don't want to use the updated
     // version when generating the create part.
-    std::unordered_set<Symbol> bound_symbols_copy(context_.bound_symbols);
-    MatchContext match_ctx{matching, context_.symbol_table, bound_symbols_copy,
-                           GraphView::NEW};
+    std::unordered_set<Symbol> bound_symbols_copy(context_->bound_symbols);
+    MatchContext match_ctx{matching, *context_->symbol_table,
+                           bound_symbols_copy, GraphView::NEW};
     auto on_match = PlanMatching(match_ctx, nullptr);
     // Use the original bound_symbols, so we fill it with new symbols.
-    auto on_create = impl::GenCreateForPattern(*merge.pattern_, nullptr,
-                                               context_.symbol_table,
-                                               context_.bound_symbols);
+    auto on_create =
+        GenCreateForPattern(*merge.pattern_, nullptr, *context_->symbol_table,
+                            context_->bound_symbols);
     for (auto &set : merge.on_create_) {
-      on_create = impl::HandleWriteClause(set, on_create, context_.symbol_table,
-                                          context_.bound_symbols);
+      on_create = HandleWriteClause(set, on_create, *context_->symbol_table,
+                                    context_->bound_symbols);
       DCHECK(on_create) << "Expected SET in MERGE ... ON CREATE";
     }
     for (auto &set : merge.on_match_) {
-      on_match = impl::HandleWriteClause(set, on_match, context_.symbol_table,
-                                         context_.bound_symbols);
+      on_match = HandleWriteClause(set, on_match, *context_->symbol_table,
+                                   context_->bound_symbols);
       DCHECK(on_match) << "Expected SET in MERGE ... ON MATCH";
     }
     return std::make_unique<plan::Merge>(
diff --git a/src/query/plan/variable_start_planner.hpp b/src/query/plan/variable_start_planner.hpp
index 1e4918f28..c4051b337 100644
--- a/src/query/plan/variable_start_planner.hpp
+++ b/src/query/plan/variable_start_planner.hpp
@@ -308,7 +308,7 @@ class VaryQueryPartMatching {
 template <class TPlanningContext>
 class VariableStartPlanner {
  private:
-  TPlanningContext &context_;
+  TPlanningContext *context_;
 
   // Generates different, equivalent query parts by taking different graph
   // matching routes for each query part.
@@ -325,18 +325,18 @@ class VariableStartPlanner {
   }
 
  public:
-  explicit VariableStartPlanner(TPlanningContext &context)
+  explicit VariableStartPlanner(TPlanningContext *context)
       : context_(context) {}
 
   /// @brief Generate multiple plans by varying the order of graph traversal.
   auto Plan(const std::vector<SingleQueryPart> &query_parts) {
     return iter::imap(
-        [context = &context_](const auto &alternative_query_parts) {
-          RuleBasedPlanner<TPlanningContext> rule_planner(*context);
+        [context = context_](const auto &alternative_query_parts) {
+          RuleBasedPlanner<TPlanningContext> rule_planner(context);
           context->bound_symbols.clear();
           return rule_planner.Plan(alternative_query_parts);
         },
-        VaryQueryMatching(query_parts, context_.symbol_table));
+        VaryQueryMatching(query_parts, *context_->symbol_table));
   }
 
   /// @brief The result of plan generation is an iterable of roots to multiple
diff --git a/src/query/plan/vertex_count_cache.hpp b/src/query/plan/vertex_count_cache.hpp
index fe4cf31c6..b3d820400 100644
--- a/src/query/plan/vertex_count_cache.hpp
+++ b/src/query/plan/vertex_count_cache.hpp
@@ -15,54 +15,56 @@ namespace query::plan {
 template <class TDbAccessor>
 class VertexCountCache {
  public:
-  VertexCountCache(const TDbAccessor &db) : db_(db) {}
+  VertexCountCache(TDbAccessor *db) : db_(db) {}
 
-  int64_t VerticesCount() const {
-    if (!vertices_count_) vertices_count_ = db_.VerticesCount();
+  auto Label(const std::string &name) { return db_->Label(name); }
+  auto Property(const std::string &name) { return db_->Property(name); }
+  auto EdgeType(const std::string &name) { return db_->EdgeType(name); }
+
+  int64_t VerticesCount() {
+    if (!vertices_count_) vertices_count_ = db_->VerticesCount();
     return *vertices_count_;
   }
 
-  int64_t VerticesCount(storage::Label label) const {
+  int64_t VerticesCount(storage::Label label) {
     if (label_vertex_count_.find(label) == label_vertex_count_.end())
-      label_vertex_count_[label] = db_.VerticesCount(label);
+      label_vertex_count_[label] = db_->VerticesCount(label);
     return label_vertex_count_.at(label);
   }
 
-  int64_t VerticesCount(storage::Label label,
-                        storage::Property property) const {
+  int64_t VerticesCount(storage::Label label, storage::Property property) {
     auto key = std::make_pair(label, property);
     if (label_property_vertex_count_.find(key) ==
         label_property_vertex_count_.end())
-      label_property_vertex_count_[key] = db_.VerticesCount(label, property);
+      label_property_vertex_count_[key] = db_->VerticesCount(label, property);
     return label_property_vertex_count_.at(key);
   }
 
   int64_t VerticesCount(storage::Label label, storage::Property property,
-                        const PropertyValue &value) const {
+                        const PropertyValue &value) {
     auto label_prop = std::make_pair(label, property);
     auto &value_vertex_count = property_value_vertex_count_[label_prop];
     if (value_vertex_count.find(value) == value_vertex_count.end())
-      value_vertex_count[value] = db_.VerticesCount(label, property, value);
+      value_vertex_count[value] = db_->VerticesCount(label, property, value);
     return value_vertex_count.at(value);
   }
 
   int64_t VerticesCount(
       storage::Label label, storage::Property property,
       const std::experimental::optional<utils::Bound<PropertyValue>> &lower,
-      const std::experimental::optional<utils::Bound<PropertyValue>> &upper)
-      const {
+      const std::experimental::optional<utils::Bound<PropertyValue>> &upper) {
     auto label_prop = std::make_pair(label, property);
     auto &bounds_vertex_count = property_bounds_vertex_count_[label_prop];
     BoundsKey bounds = std::make_pair(lower, upper);
     if (bounds_vertex_count.find(bounds) == bounds_vertex_count.end())
       bounds_vertex_count[bounds] =
-          db_.VerticesCount(label, property, lower, upper);
+          db_->VerticesCount(label, property, lower, upper);
     return bounds_vertex_count.at(bounds);
   }
 
   bool LabelPropertyIndexExists(storage::Label label,
-                                storage::Property property) const {
-    return db_.LabelPropertyIndexExists(label, property);
+                                storage::Property property) {
+    return db_->LabelPropertyIndexExists(label, property);
   }
 
  private:
@@ -109,18 +111,18 @@ class VertexCountCache {
     }
   };
 
-  const TDbAccessor &db_;
-  mutable std::experimental::optional<int64_t> vertices_count_;
-  mutable std::unordered_map<storage::Label, int64_t> label_vertex_count_;
-  mutable std::unordered_map<LabelPropertyKey, int64_t, LabelPropertyHash>
+  TDbAccessor *db_;
+  std::experimental::optional<int64_t> vertices_count_;
+  std::unordered_map<storage::Label, int64_t> label_vertex_count_;
+  std::unordered_map<LabelPropertyKey, int64_t, LabelPropertyHash>
       label_property_vertex_count_;
-  mutable std::unordered_map<
+  std::unordered_map<
       LabelPropertyKey,
       std::unordered_map<query::TypedValue, int64_t, query::TypedValue::Hash,
                          query::TypedValue::BoolEqual>,
       LabelPropertyHash>
       property_value_vertex_count_;
-  mutable std::unordered_map<
+  std::unordered_map<
       LabelPropertyKey,
       std::unordered_map<BoundsKey, int64_t, BoundsHash, BoundsEqual>,
       LabelPropertyHash>
@@ -128,7 +130,7 @@ class VertexCountCache {
 };
 
 template <class TDbAccessor>
-auto MakeVertexCountCache(const TDbAccessor &db) {
+auto MakeVertexCountCache(TDbAccessor *db) {
   return VertexCountCache<TDbAccessor>(db);
 }
 
diff --git a/src/query/serialization.capnp b/src/query/serialization.capnp
index 58c025336..02ebc22ef 100644
--- a/src/query/serialization.capnp
+++ b/src/query/serialization.capnp
@@ -17,11 +17,6 @@ struct TypedValueVectorCompare {
   ordering @0 :List(Ast.Ordering);
 }
 
-struct EvaluationContext {
-  timestamp @0 : Int64;
-  params @1 : Utils.Map(Utils.BoxInt64, Storage.PropertyValue);
-}
-
 struct TypedValue {
   union {
     nullType @0 :Void;
diff --git a/src/query/serialization.cpp b/src/query/serialization.cpp
index 2d9e1de71..0685dadeb 100644
--- a/src/query/serialization.cpp
+++ b/src/query/serialization.cpp
@@ -2,6 +2,7 @@
 
 #include "distributed/data_manager.hpp"
 #include "query/frontend/ast/ast_serialization.hpp"
+#include "utils/serialization.hpp"
 
 namespace query {
 
@@ -140,13 +141,12 @@ void LoadCapnpTypedValue(const capnp::TypedValue::Reader &reader,
   }
 }
 
-void SaveEvaluationContext(const EvaluationContext &ctx,
-                           capnp::EvaluationContext::Builder *builder) {
-  builder->setTimestamp(ctx.timestamp);
-  auto params_builder =
-      builder->initParams().initEntries(ctx.parameters.size());
+void Save(const Parameters &parameters,
+          utils::capnp::Map<utils::capnp::BoxInt64,
+                            storage::capnp::PropertyValue>::Builder *builder) {
+  auto params_builder = builder->initEntries(parameters.size());
   size_t i = 0;
-  for (auto &entry : ctx.parameters) {
+  for (auto &entry : parameters) {
     auto builder = params_builder[i];
     auto key_builder = builder.initKey();
     key_builder.setValue(entry.first);
@@ -156,13 +156,14 @@ void SaveEvaluationContext(const EvaluationContext &ctx,
   }
 }
 
-void LoadEvaluationContext(const capnp::EvaluationContext::Reader &reader,
-                           EvaluationContext *ctx) {
-  ctx->timestamp = reader.getTimestamp();
-  for (const auto &entry_reader : reader.getParams().getEntries()) {
+void Load(
+    Parameters *parameters,
+    const utils::capnp::Map<utils::capnp::BoxInt64,
+                            storage::capnp::PropertyValue>::Reader &reader) {
+  for (const auto &entry_reader : reader.getEntries()) {
     PropertyValue value;
     storage::LoadCapnpPropertyValue(entry_reader.getValue(), &value);
-    ctx->parameters.Add(entry_reader.getKey().getValue(), value);
+    parameters->Add(entry_reader.getKey().getValue(), value);
   }
 }
 
@@ -351,22 +352,20 @@ void Load(query::TypedValue *value, slk::Reader *reader,
   }
 }
 
-void Save(const query::EvaluationContext &ctx, slk::Builder *builder) {
-  slk::Save(ctx.timestamp, builder);
-  slk::Save(ctx.parameters.size(), builder);
-  for (auto &entry : ctx.parameters) {
+void Save(const query::Parameters &parameters, slk::Builder *builder) {
+  slk::Save(parameters.size(), builder);
+  for (auto &entry : parameters) {
     slk::Save(entry, builder);
   }
 }
 
-void Load(query::EvaluationContext *ctx, slk::Reader *reader) {
-  slk::Load(&ctx->timestamp, reader);
+void Load(query::Parameters *parameters, slk::Reader *reader) {
   size_t size = 0;
   slk::Load(&size, reader);
   for (size_t i = 0; i < size; ++i) {
     std::pair<int, PropertyValue> entry;
     slk::Load(&entry, reader);
-    ctx->parameters.Add(entry.first, entry.second);
+    parameters->Add(entry.first, entry.second);
   }
 }
 
diff --git a/src/query/serialization.hpp b/src/query/serialization.hpp
index 1bb026ada..d7f7ef23c 100644
--- a/src/query/serialization.hpp
+++ b/src/query/serialization.hpp
@@ -23,12 +23,6 @@ void LoadCapnpTypedValue(const capnp::TypedValue::Reader &reader,
                          database::GraphDbAccessor *dba,
                          distributed::DataManager *data_manager);
 
-void SaveEvaluationContext(const EvaluationContext &ctx,
-                           capnp::EvaluationContext::Builder *builder);
-
-void LoadEvaluationContext(const capnp::EvaluationContext::Reader &reader,
-                           EvaluationContext *ctx);
-
 void Save(const TypedValueVectorCompare &comparator,
           capnp::TypedValueVectorCompare::Builder *builder);
 
@@ -60,6 +54,15 @@ inline void Load(SymbolTable *symbol_table,
   }
 }
 
+void Save(const Parameters &parameters,
+          utils::capnp::Map<utils::capnp::BoxInt64,
+                            storage::capnp::PropertyValue>::Builder *builder);
+
+void Load(
+    Parameters *parameters,
+    const utils::capnp::Map<utils::capnp::BoxInt64,
+                            storage::capnp::PropertyValue>::Reader &reader);
+
 }  // namespace query
 
 namespace slk {
@@ -75,10 +78,6 @@ inline void Load(query::SymbolTable *symbol_table, slk::Reader *reader) {
   slk::Load(&symbol_table->table_, reader);
 }
 
-void Save(const query::EvaluationContext &ctx, slk::Builder *builder);
-
-void Load(query::EvaluationContext *ctx, slk::Reader *reader);
-
 void Save(const query::TypedValue &value, slk::Builder *builder,
           storage::SendVersions versions, int16_t worker_id);
 
@@ -95,4 +94,8 @@ void Save(const query::TypedValueVectorCompare &comparator,
 
 void Load(query::TypedValueVectorCompare *comparator, slk::Reader *reader);
 
+void Save(const query::Parameters &parameters, slk::Builder *builder);
+
+void Load(query::Parameters *parameters, slk::Reader *reader);
+
 }  // namespace slk
diff --git a/tests/benchmark/query/planner.cpp b/tests/benchmark/query/planner.cpp
index d16b44ad8..bf272db5d 100644
--- a/tests/benchmark/query/planner.cpp
+++ b/tests/benchmark/query/planner.cpp
@@ -42,8 +42,8 @@ static void BM_PlanChainedMatches(benchmark::State &state) {
     int num_matches = state.range(0);
     auto *query = AddChainedMatches(num_matches, storage);
     auto symbol_table = query::MakeSymbolTable(query);
-    auto ctx =
-        query::plan::MakePlanningContext(storage, symbol_table, query, *dba);
+    auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
+                                                dba.get());
     state.ResumeTiming();
     auto query_parts =
         query::plan::CollectQueryParts(symbol_table, storage, query);
@@ -52,7 +52,7 @@ static void BM_PlanChainedMatches(benchmark::State &state) {
     }
     auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
     auto plans = query::plan::MakeLogicalPlanForSingleQuery<
-        query::plan::VariableStartPlanner>(single_query_parts, ctx);
+        query::plan::VariableStartPlanner>(single_query_parts, &ctx);
     for (const auto &plan : plans) {
       // Exhaust through all generated plans, since they are lazily generated.
       benchmark::DoNotOptimize(plan.get());
@@ -65,10 +65,10 @@ BENCHMARK(BM_PlanChainedMatches)
     ->Range(50, 400)
     ->Unit(benchmark::kMillisecond);
 
-static query::CypherQuery *AddIndexedMatches(
-    int num_matches, storage::Label label,
-    const std::pair<std::string, storage::Property> &property,
-    query::AstStorage &storage) {
+static query::CypherQuery *AddIndexedMatches(int num_matches,
+                                             const std::string &label,
+                                             const std::string &property,
+                                             query::AstStorage &storage) {
   auto *query = storage.Create<query::CypherQuery>();
   for (int i = 0; i < num_matches; ++i) {
     auto *match = storage.Create<query::Match>();
@@ -79,8 +79,9 @@ static query::CypherQuery *AddIndexedMatches(
     std::string node1_name = "node" + std::to_string(i - 1);
     auto *node = storage.Create<query::NodeAtom>(
         storage.Create<query::Identifier>(node1_name));
-    node->labels_.emplace_back(label);
-    node->properties_[property] = storage.Create<query::PrimitiveLiteral>(i);
+    node->labels_.emplace_back(storage.GetLabelIx(label));
+    node->properties_[storage.GetPropertyIx(property)] =
+        storage.Create<query::PrimitiveLiteral>(i);
     pattern->atoms_.emplace_back(node);
     single_query->clauses_.emplace_back(match);
     query->single_query_ = single_query;
@@ -102,13 +103,13 @@ static auto CreateIndexedVertices(int index_count, int vertex_count,
     }
   }
   dba->Commit();
-  return std::make_pair(label, prop);
+  return std::make_pair("label", "prop");
 }
 
 static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
   database::GraphDb db;
-  storage::Label label;
-  storage::Property prop;
+  std::string label;
+  std::string prop;
   int index_count = state.range(0);
   int vertex_count = state.range(1);
   std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
@@ -117,12 +118,11 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
   while (state.KeepRunning()) {
     state.PauseTiming();
     query::AstStorage storage;
-    auto *query = AddIndexedMatches(index_count, label,
-                                    std::make_pair("prop", prop), storage);
+    auto *query = AddIndexedMatches(index_count, label, prop, storage);
     auto symbol_table = query::MakeSymbolTable(query);
     state.ResumeTiming();
-    auto ctx =
-        query::plan::MakePlanningContext(storage, symbol_table, query, *dba);
+    auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
+                                                dba.get());
     auto query_parts =
         query::plan::CollectQueryParts(symbol_table, storage, query);
     if (query_parts.query_parts.size() == 0) {
@@ -130,9 +130,9 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
     }
     auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
     auto plans = query::plan::MakeLogicalPlanForSingleQuery<
-        query::plan::VariableStartPlanner>(single_query_parts, ctx);
+        query::plan::VariableStartPlanner>(single_query_parts, &ctx);
     for (auto plan : plans) {
-      query::plan::EstimatePlanCost(*dba, parameters, *plan);
+      query::plan::EstimatePlanCost(dba.get(), parameters, *plan);
     }
   }
 }
@@ -140,23 +140,22 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
 static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
     benchmark::State &state) {
   database::GraphDb db;
-  storage::Label label;
-  storage::Property prop;
+  std::string label;
+  std::string prop;
   int index_count = state.range(0);
   int vertex_count = state.range(1);
   std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
   auto dba = db.Access();
-  auto vertex_counts = query::plan::MakeVertexCountCache(*dba);
+  auto vertex_counts = query::plan::MakeVertexCountCache(dba.get());
   query::Parameters parameters;
   while (state.KeepRunning()) {
     state.PauseTiming();
     query::AstStorage storage;
-    auto *query = AddIndexedMatches(index_count, label,
-                                    std::make_pair("prop", prop), storage);
+    auto *query = AddIndexedMatches(index_count, label, prop, storage);
     auto symbol_table = query::MakeSymbolTable(query);
     state.ResumeTiming();
-    auto ctx = query::plan::MakePlanningContext(storage, symbol_table, query,
-                                                vertex_counts);
+    auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
+                                                &vertex_counts);
     auto query_parts =
         query::plan::CollectQueryParts(symbol_table, storage, query);
     if (query_parts.query_parts.size() == 0) {
@@ -164,9 +163,9 @@ static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
     }
     auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
     auto plans = query::plan::MakeLogicalPlanForSingleQuery<
-        query::plan::VariableStartPlanner>(single_query_parts, ctx);
+        query::plan::VariableStartPlanner>(single_query_parts, &ctx);
     for (auto plan : plans) {
-      query::plan::EstimatePlanCost(vertex_counts, parameters, *plan);
+      query::plan::EstimatePlanCost(&vertex_counts, parameters, *plan);
     }
   }
 }
diff --git a/tests/manual/distributed_query_planner.cpp b/tests/manual/distributed_query_planner.cpp
index b22f72ff7..189ce7a23 100644
--- a/tests/manual/distributed_query_planner.cpp
+++ b/tests/manual/distributed_query_planner.cpp
@@ -17,7 +17,10 @@ DEFCOMMAND(ShowDistributed) {
   if (ss.fail() || !ss.eof() || plan_ix >= plans.size()) return;
   const auto &plan = plans[plan_ix].first;
   std::atomic<int64_t> plan_id{0};
-  auto distributed_plan = MakeDistributedPlan(*plan, symbol_table, plan_id);
+  std::vector<storage::Property> properties_by_ix =
+      query::NamesToProperties(ast_storage.properties_, &dba);
+  auto distributed_plan = MakeDistributedPlan(ast_storage, *plan, symbol_table,
+                                              plan_id, properties_by_ix);
   {
     std::cout << "---- Master Plan ---- " << std::endl;
     query::plan::DistributedPrettyPrint(dba,
diff --git a/tests/manual/interactive_planning.cpp b/tests/manual/interactive_planning.cpp
index 1fd28f642..399639205 100644
--- a/tests/manual/interactive_planning.cpp
+++ b/tests/manual/interactive_planning.cpp
@@ -129,14 +129,18 @@ class Timer {
 // Dummy DbAccessor which forwards user input for various vertex counts.
 class InteractiveDbAccessor {
  public:
-  InteractiveDbAccessor(database::GraphDbAccessor &dba, int64_t vertices_count,
+  InteractiveDbAccessor(database::GraphDbAccessor *dba, int64_t vertices_count,
                         Timer &timer)
       : dba_(dba), vertices_count_(vertices_count), timer_(timer) {}
 
-  int64_t VerticesCount() const { return vertices_count_; }
+  auto Label(const std::string &name) { return dba_->Label(name); }
+  auto Property(const std::string &name) { return dba_->Property(name); }
+  auto EdgeType(const std::string &name) { return dba_->EdgeType(name); }
 
-  int64_t VerticesCount(storage::Label label_id) const {
-    auto label = dba_.LabelName(label_id);
+  int64_t VerticesCount() { return vertices_count_; }
+
+  int64_t VerticesCount(storage::Label label_id) {
+    auto label = dba_->LabelName(label_id);
     if (label_vertex_count_.find(label) == label_vertex_count_.end()) {
       label_vertex_count_[label] = ReadVertexCount("label '" + label + "'");
     }
@@ -144,9 +148,9 @@ class InteractiveDbAccessor {
   }
 
   int64_t VerticesCount(storage::Label label_id,
-                        storage::Property property_id) const {
-    auto label = dba_.LabelName(label_id);
-    auto property = dba_.PropertyName(property_id);
+                        storage::Property property_id) {
+    auto label = dba_->LabelName(label_id);
+    auto property = dba_->PropertyName(property_id);
     auto key = std::make_pair(label, property);
     if (label_property_vertex_count_.find(key) ==
         label_property_vertex_count_.end()) {
@@ -157,9 +161,9 @@ class InteractiveDbAccessor {
   }
 
   int64_t VerticesCount(storage::Label label_id, storage::Property property_id,
-                        const PropertyValue &value) const {
-    auto label = dba_.LabelName(label_id);
-    auto property = dba_.PropertyName(property_id);
+                        const PropertyValue &value) {
+    auto label = dba_->LabelName(label_id);
+    auto property = dba_->PropertyName(property_id);
     auto label_prop = std::make_pair(label, property);
     if (label_property_index_.find(label_prop) == label_property_index_.end()) {
       return 0;
@@ -178,10 +182,9 @@ class InteractiveDbAccessor {
   int64_t VerticesCount(
       storage::Label label_id, storage::Property property_id,
       const std::experimental::optional<utils::Bound<PropertyValue>> lower,
-      const std::experimental::optional<utils::Bound<PropertyValue>> upper)
-      const {
-    auto label = dba_.LabelName(label_id);
-    auto property = dba_.PropertyName(property_id);
+      const std::experimental::optional<utils::Bound<PropertyValue>> upper) {
+    auto label = dba_->LabelName(label_id);
+    auto property = dba_->PropertyName(property_id);
     std::stringstream range_string;
     if (lower) {
       range_string << (lower->IsInclusive() ? "[" : "(") << lower->value()
@@ -197,9 +200,9 @@ class InteractiveDbAccessor {
   }
 
   bool LabelPropertyIndexExists(storage::Label label_id,
-                                storage::Property property_id) const {
-    auto label = dba_.LabelName(label_id);
-    auto property = dba_.PropertyName(property_id);
+                                storage::Property property_id) {
+    auto label = dba_->LabelName(label_id);
+    auto property = dba_->PropertyName(property_id);
     auto key = std::make_pair(label, property);
     if (label_property_index_.find(key) == label_property_index_.end()) {
       bool resp = timer_.WithPause([&label, &property]() {
@@ -312,15 +315,14 @@ class InteractiveDbAccessor {
  private:
   typedef std::pair<std::string, std::string> LabelPropertyKey;
 
-  database::GraphDbAccessor &dba_;
+  database::GraphDbAccessor *dba_;
   int64_t vertices_count_;
   Timer &timer_;
-  mutable std::map<std::string, int64_t> label_vertex_count_;
-  mutable std::map<std::pair<std::string, std::string>, int64_t>
+  std::map<std::string, int64_t> label_vertex_count_;
+  std::map<std::pair<std::string, std::string>, int64_t>
       label_property_vertex_count_;
-  mutable std::map<std::pair<std::string, std::string>, bool>
-      label_property_index_;
-  mutable std::map<
+  std::map<std::pair<std::string, std::string>, bool> label_property_index_;
+  std::map<
       std::pair<std::string, std::string>,
       std::unordered_map<query::TypedValue, int64_t, query::TypedValue::Hash,
                          query::TypedValue::BoolEqual>>
@@ -408,7 +410,8 @@ DEFCOMMAND(Help) {
 void ExaminePlans(
     database::GraphDbAccessor &dba, const query::SymbolTable &symbol_table,
     std::vector<std::pair<std::unique_ptr<query::plan::LogicalOperator>,
-                          double>> &plans) {
+                          double>> &plans,
+    const query::AstStorage &ast) {
   while (true) {
     auto line = ReadLine("plan? ");
     if (!line || *line == "quit") break;
@@ -428,18 +431,17 @@ void ExaminePlans(
                 << " arguments" << std::endl;
       continue;
     }
-    command.function(dba, symbol_table, plans, args);
+    command.function(dba, symbol_table, plans, args, ast);
   }
 }
 
-query::Query *MakeAst(const std::string &query, query::AstStorage *storage,
-                      database::GraphDbAccessor &dba) {
+query::Query *MakeAst(const std::string &query, query::AstStorage *storage) {
   query::ParsingContext parsing_context;
   parsing_context.is_query_cached = false;
   // query -> AST
   auto parser = std::make_unique<query::frontend::opencypher::Parser>(query);
   // AST -> high level tree
-  query::frontend::CypherMainVisitor visitor(parsing_context, storage, &dba);
+  query::frontend::CypherMainVisitor visitor(parsing_context, storage);
   visitor.visit(parser->tree());
   return visitor.query();
 }
@@ -448,24 +450,22 @@ query::Query *MakeAst(const std::string &query, query::AstStorage *storage,
 // order by cost.
 auto MakeLogicalPlans(query::CypherQuery *query, query::AstStorage &ast,
                       query::SymbolTable &symbol_table,
-                      InteractiveDbAccessor &dba) {
+                      InteractiveDbAccessor *dba) {
   auto query_parts = query::plan::CollectQueryParts(symbol_table, ast, query);
   std::vector<std::pair<std::unique_ptr<query::plan::LogicalOperator>, double>>
       plans_with_cost;
-  auto ctx = query::plan::MakePlanningContext(ast, symbol_table, query, dba);
+  auto ctx = query::plan::MakePlanningContext(&ast, &symbol_table, query, dba);
   if (query_parts.query_parts.size() <= 0) {
     std::cerr << "Failed to extract query parts" << std::endl;
     std::exit(EXIT_FAILURE);
   }
   auto plans = query::plan::MakeLogicalPlanForSingleQuery<
       query::plan::VariableStartPlanner>(
-      query_parts.query_parts.at(0).single_query_parts, ctx);
+      query_parts.query_parts.at(0).single_query_parts, &ctx);
   query::Parameters parameters;
   for (auto plan : plans) {
-    query::plan::CostEstimator<InteractiveDbAccessor> estimator(dba,
-                                                                parameters);
-    plan->Accept(estimator);
-    plans_with_cost.emplace_back(std::move(plan), estimator.cost());
+    auto cost = query::plan::EstimatePlanCost(dba, parameters, *plan);
+    plans_with_cost.emplace_back(std::move(plan), cost);
   }
   std::stable_sort(
       plans_with_cost.begin(), plans_with_cost.end(),
@@ -482,7 +482,7 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
   }
   Timer planning_timer;
   InteractiveDbAccessor interactive_db(
-      *dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
+      dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
       planning_timer);
   if (!in_db_filename.empty()) {
     std::ifstream db_file(in_db_filename);
@@ -495,7 +495,7 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
     try {
       query::AstStorage ast;
       auto *query =
-          dynamic_cast<query::CypherQuery *>(MakeAst(*line, &ast, *dba));
+          dynamic_cast<query::CypherQuery *>(MakeAst(*line, &ast));
       if (!query) {
         throw utils::BasicException(
             "Interactive planning is only avaialable for regular openCypher "
@@ -503,14 +503,14 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
       }
       auto symbol_table = query::MakeSymbolTable(query);
       planning_timer.Start();
-      auto plans = MakeLogicalPlans(query, ast, symbol_table, interactive_db);
+      auto plans = MakeLogicalPlans(query, ast, symbol_table, &interactive_db);
       auto planning_time = planning_timer.Elapsed();
       std::cout
           << "Planning took "
           << std::chrono::duration<double, std::milli>(planning_time).count()
           << "ms" << std::endl;
       std::cout << "Generated " << plans.size() << " plans" << std::endl;
-      ExaminePlans(*dba, symbol_table, plans);
+      ExaminePlans(*dba, symbol_table, plans, ast);
     } catch (const utils::BasicException &e) {
       std::cout << "Error: " << e.what() << std::endl;
     }
diff --git a/tests/manual/interactive_planning.hpp b/tests/manual/interactive_planning.hpp
index da45f2fa7..05c9e7adc 100644
--- a/tests/manual/interactive_planning.hpp
+++ b/tests/manual/interactive_planning.hpp
@@ -23,7 +23,7 @@ struct Command {
   typedef std::vector<std::string> Args;
   // Function of this command
   std::function<void(database::GraphDbAccessor &, const query::SymbolTable &,
-                     PlansWithCost &, const Args &)>
+                     PlansWithCost &, const Args &, const query::AstStorage &)>
       function;
   // Number of arguments the function works with.
   int arg_count;
@@ -31,10 +31,11 @@ struct Command {
   std::string documentation;
 };
 
-#define DEFCOMMAND(Name)                                     \
-  void Name##Command(database::GraphDbAccessor &dba,         \
-                     const query::SymbolTable &symbol_table, \
-                     PlansWithCost &plans, const Command::Args &args)
+#define DEFCOMMAND(Name)                                              \
+  void Name##Command(database::GraphDbAccessor &dba,                  \
+                     const query::SymbolTable &symbol_table,          \
+                     PlansWithCost &plans, const Command::Args &args, \
+                     const query::AstStorage &ast_storage)
 
 void AddCommand(const std::string &name, const Command &command);
 
diff --git a/tests/unit/ast_serialization.cpp b/tests/unit/ast_serialization.cpp
index 1ea032015..1e5c51d89 100644
--- a/tests/unit/ast_serialization.cpp
+++ b/tests/unit/ast_serialization.cpp
@@ -10,7 +10,6 @@
 #include <gtest/gtest.h>
 
 #include "communication/rpc/serialization.hpp"
-#include "database/distributed/distributed_graph_db.hpp"
 #include "query/context.hpp"
 #include "query/frontend/ast/ast.hpp"
 #include "query/frontend/ast/ast_serialization.hpp"
@@ -30,26 +29,20 @@ using testing::ElementsAre;
 using testing::Pair;
 using testing::UnorderedElementsAre;
 
-static std::unique_ptr<database::Master> db_;
-
 // Base class for all test types
 class Base {
  public:
-  explicit Base(const std::string &query) : query_string_(query) {
-    db_accessor_ = db_->Access();
-  }
-  std::unique_ptr<database::GraphDbAccessor> db_accessor_;
+  explicit Base(const std::string &query) : query_string_(query) {}
+
   ParsingContext context_;
   Parameters parameters_;
   std::string query_string_;
 
-  auto Prop(const std::string &prop_name) {
-    return db_accessor_->Property(prop_name);
-  }
+  virtual PropertyIx Prop(const std::string &prop_name) = 0;
 
-  auto PropPair(const std::string &prop_name) {
-    return std::make_pair(prop_name, Prop(prop_name));
-  }
+  virtual LabelIx Label(const std::string &label_name) = 0;
+
+  virtual EdgeTypeIx EdgeType(const std::string &edge_type_name) = 0;
 
   TypedValue LiteralValue(Expression *expression) {
     if (context_.is_query_cached) {
@@ -86,7 +79,7 @@ class CapnpAstGenerator : public Base {
   explicit CapnpAstGenerator(const std::string &query) : Base(query) {
     ::frontend::opencypher::Parser parser(query);
     AstStorage tmp_storage;
-    CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
+    CypherMainVisitor visitor(context_, &tmp_storage);
     visitor.visit(parser.tree());
 
     ::capnp::MallocMessageBuilder message;
@@ -105,6 +98,18 @@ class CapnpAstGenerator : public Base {
     }
   }
 
+  PropertyIx Prop(const std::string &prop_name) override {
+    return storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return storage_.GetEdgeTypeIx(name);
+  }
+
   AstStorage storage_;
   Query *query_;
 };
@@ -114,7 +119,7 @@ class SlkAstGenerator : public Base {
   explicit SlkAstGenerator(const std::string &query) : Base(query) {
     ::frontend::opencypher::Parser parser(query);
     AstStorage tmp_storage;
-    CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
+    CypherMainVisitor visitor(context_, &tmp_storage);
     visitor.visit(parser.tree());
 
     slk::Builder builder;
@@ -130,24 +135,24 @@ class SlkAstGenerator : public Base {
     }
   }
 
+  PropertyIx Prop(const std::string &prop_name) override {
+    return storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return storage_.GetEdgeTypeIx(name);
+  }
+
   AstStorage storage_;
   Query *query_;
 };
 
 template <typename T>
-class CypherMainVisitorTest : public ::testing::Test {
- public:
-  static void SetUpTestCase() {
-    db_ = std::make_unique<database::Master>();
-    db_->Start();
-  }
-
-  static void TearDownTestCase() {
-    db_->Shutdown();
-    db_->AwaitShutdown();
-    db_ = nullptr;
-  }
-};
+class CypherMainVisitorTest : public ::testing::Test {};
 
 typedef ::testing::Types<CapnpAstGenerator, SlkAstGenerator> AstGeneratorTypes;
 
@@ -175,8 +180,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) {
   auto identifier = dynamic_cast<Identifier *>(property_lookup->expression_);
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
-  ASSERT_EQ(property_lookup->property_,
-            ast_generator.db_accessor_->Property("x"));
+  ASSERT_EQ(property_lookup->property_, ast_generator.Prop("x"));
 }
 
 TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
@@ -194,8 +198,7 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("x"),
-                          ast_generator.db_accessor_->Label("y")));
+              ElementsAre(ast_generator.Label("x"), ast_generator.Label("y")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
@@ -211,7 +214,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l")));
+              ElementsAre(ast_generator.Label("l-$\"'ab`e``l")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
@@ -227,8 +230,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
         return_clause->body_.named_expressions[0]->expression_);
     auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
     ASSERT_EQ(identifier->name_, "n");
-    ASSERT_THAT(labels_test->labels_,
-                ElementsAre(ast_generator.db_accessor_->Label(label)));
+    ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label(label)));
   }
 }
 
@@ -244,8 +246,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
       return_clause->body_.named_expressions[0]->expression_);
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   EXPECT_EQ(identifier->name_, "n");
-  ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("a")));
+  ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label("a")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) {
@@ -884,12 +885,12 @@ TYPED_TEST(CypherMainVisitorTest, MapLiteral) {
       return_clause->body_.named_expressions[0]->expression_);
   ASSERT_TRUE(map_literal);
   ASSERT_EQ(3, map_literal->elements_.size());
-  ast_generator.CheckLiteral(
-      map_literal->elements_[ast_generator.PropPair("a")], 1);
-  ast_generator.CheckLiteral(
-      map_literal->elements_[ast_generator.PropPair("b")], "bla");
+  ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("a")],
+                             1);
+  ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("b")],
+                             "bla");
   auto *elem_2 = dynamic_cast<ListLiteral *>(
-      map_literal->elements_[ast_generator.PropPair("c")]);
+      map_literal->elements_[ast_generator.Prop("c")]);
   ASSERT_TRUE(elem_2);
   EXPECT_EQ(2, elem_2->elements_.size());
   auto *elem_2_1 = dynamic_cast<MapLiteral *>(elem_2->elements_[1]);
@@ -918,19 +919,19 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
   EXPECT_EQ(node->identifier_->name_,
             CypherMainVisitor::kAnonPrefix + std::to_string(1));
   EXPECT_FALSE(node->identifier_->user_declared_);
-  EXPECT_THAT(node->labels_, UnorderedElementsAre(
-                                 ast_generator.db_accessor_->Label("label1"),
-                                 ast_generator.db_accessor_->Label("label2"),
-                                 ast_generator.db_accessor_->Label("label3")));
-  std::map<std::pair<std::string, storage::Property>, int64_t> properties;
+  EXPECT_THAT(node->labels_,
+              UnorderedElementsAre(ast_generator.Label("label1"),
+                                   ast_generator.Label("label2"),
+                                   ast_generator.Label("label3")));
+  std::unordered_map<PropertyIx, int64_t> properties;
   for (auto x : node->properties_) {
     TypedValue value = ast_generator.LiteralValue(x.second);
     ASSERT_TRUE(value.type() == TypedValue::Type::Int);
     properties[x.first] = value.Value<int64_t>();
   }
   EXPECT_THAT(properties,
-              UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
-                                   Pair(ast_generator.PropPair("b"), 10)));
+              UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
+                                   Pair(ast_generator.Prop("b"), 10)));
 }
 
 TYPED_TEST(CypherMainVisitorTest, PropertyMapSameKeyAppearsTwice) {
@@ -1022,19 +1023,18 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
   auto *edge = dynamic_cast<EdgeAtom *>(match->patterns_[0]->atoms_[1]);
   ASSERT_TRUE(edge);
   EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN);
-  EXPECT_THAT(
-      edge->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
-  std::map<std::pair<std::string, storage::Property>, int64_t> properties;
+  EXPECT_THAT(edge->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
+  std::unordered_map<PropertyIx, int64_t> properties;
   for (auto x : edge->properties_) {
     TypedValue value = ast_generator.LiteralValue(x.second);
     ASSERT_TRUE(value.type() == TypedValue::Type::Int);
     properties[x.first] = value.Value<int64_t>();
   }
   EXPECT_THAT(properties,
-              UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
-                                   Pair(ast_generator.PropPair("b"), 10)));
+              UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
+                                   Pair(ast_generator.Prop("b"), 10)));
 }
 
 TYPED_TEST(CypherMainVisitorTest, RelationshipPatternVariable) {
@@ -1169,8 +1169,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUnboundedWithProperty) {
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   EXPECT_EQ(edge->upper_bound_, nullptr);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
 }
 
 TYPED_TEST(CypherMainVisitorTest,
@@ -1187,10 +1186,9 @@ TYPED_TEST(CypherMainVisitorTest,
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   EXPECT_EQ(edge->upper_bound_, nullptr);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
   ASSERT_EQ(edge->edge_types_.size(), 1U);
-  auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type");
+  auto edge_type = ast_generator.EdgeType("edge_type");
   EXPECT_EQ(edge->edge_types_[0], edge_type);
 }
 
@@ -1207,8 +1205,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUpperBoundedWithProperty) {
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   ast_generator.CheckLiteral(edge->upper_bound_, 2);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
 }
 
 TYPED_TEST(CypherMainVisitorTest, ReturnUnanemdIdentifier) {
@@ -1316,7 +1313,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(set_property->property_lookup_->property_,
-              ast_generator.db_accessor_->Property("x"));
+              ast_generator.Prop("x"));
     auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_);
     ASSERT_EQ(identifier2->name_, "b");
   }
@@ -1351,8 +1348,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(set_labels->identifier_);
     ASSERT_EQ(set_labels->identifier_->name_, "g");
     ASSERT_THAT(set_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
-                                     ast_generator.db_accessor_->Label("i")));
+                UnorderedElementsAre(ast_generator.Label("h"),
+                                     ast_generator.Label("i")));
   }
 }
 
@@ -1363,7 +1360,6 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
   ASSERT_TRUE(query->single_query_);
   auto *single_query = query->single_query_;
   ASSERT_EQ(single_query->clauses_.size(), 2U);
-
   {
     auto *remove_property =
         dynamic_cast<RemoveProperty *>(single_query->clauses_[0]);
@@ -1374,7 +1370,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(remove_property->property_lookup_->property_,
-              ast_generator.db_accessor_->Property("x"));
+              ast_generator.Prop("x"));
   }
   {
     auto *remove_labels =
@@ -1383,8 +1379,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(remove_labels->identifier_);
     ASSERT_EQ(remove_labels->identifier_->name_, "g");
     ASSERT_THAT(remove_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
-                                     ast_generator.db_accessor_->Label("i")));
+                UnorderedElementsAre(ast_generator.Label("h"),
+                                     ast_generator.Label("i")));
   }
 }
 
@@ -1591,9 +1587,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties = {
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
   EXPECT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1602,10 +1597,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko"),
-      ast_generator.db_accessor_->Property("pero")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko"),
+                                              ast_generator.Prop("pero")};
   ASSERT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1619,9 +1613,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndexWithSingleProperty) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
   ASSERT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1630,9 +1623,8 @@ TYPED_TEST(CypherMainVisitorTest, DropIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::DROP);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
   EXPECT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1748,10 +1740,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) {
   ASSERT_TRUE(bfs);
   EXPECT_TRUE(bfs->IsVariable());
   EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      bfs->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(bfs->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   EXPECT_EQ(bfs->identifier_->name_, "r");
   EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e");
   EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_);
@@ -1798,10 +1789,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) {
   EXPECT_TRUE(shortest->IsVariable());
   EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(shortest->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   ast_generator.CheckLiteral(shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
@@ -1838,10 +1828,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) {
   EXPECT_TRUE(shortest->IsVariable());
   EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(shortest->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   ast_generator.CheckLiteral(shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
diff --git a/tests/unit/bfs_common.hpp b/tests/unit/bfs_common.hpp
index 049d245a9..7c2e43b6b 100644
--- a/tests/unit/bfs_common.hpp
+++ b/tests/unit/bfs_common.hpp
@@ -398,6 +398,10 @@ void BfsTest(Database *db, int lower_bound, int upper_bound,
       query::plan::ExpansionLambda{inner_edge_sym, inner_node_sym,
                                    filter_expr});
 
+  context.evaluation_context_.properties =
+      query::NamesToProperties(storage.properties_, &dba);
+  context.evaluation_context_.labels =
+      query::NamesToLabels(storage.labels_, &dba);
   std::vector<std::vector<query::TypedValue>> results;
 
   // An exception should be thrown on one of the pulls.
diff --git a/tests/unit/cypher_main_visitor.cpp b/tests/unit/cypher_main_visitor.cpp
index c57aeef13..f016668b8 100644
--- a/tests/unit/cypher_main_visitor.cpp
+++ b/tests/unit/cypher_main_visitor.cpp
@@ -29,19 +29,15 @@ using testing::UnorderedElementsAre;
 class Base {
  public:
   explicit Base(const std::string &query) : query_string_(query) {}
-  database::GraphDb db_;
-  std::unique_ptr<database::GraphDbAccessor> db_accessor_{db_.Access()};
   ParsingContext context_;
   Parameters parameters_;
   std::string query_string_;
 
-  auto Prop(const std::string &prop_name) {
-    return db_accessor_->Property(prop_name);
-  }
+  virtual PropertyIx Prop(const std::string &prop_name) = 0;
 
-  auto PropPair(const std::string &prop_name) {
-    return std::make_pair(prop_name, Prop(prop_name));
-  }
+  virtual LabelIx Label(const std::string &label_name) = 0;
+
+  virtual EdgeTypeIx EdgeType(const std::string &edge_type_name) = 0;
 
   TypedValue LiteralValue(Expression *expression) {
     if (context_.is_query_cached) {
@@ -78,11 +74,23 @@ class AstGenerator : public Base {
  public:
   explicit AstGenerator(const std::string &query) : Base(query) {
     ::frontend::opencypher::Parser parser(query);
-    CypherMainVisitor visitor(context_, &ast_storage_, db_accessor_.get());
+    CypherMainVisitor visitor(context_, &ast_storage_);
     visitor.visit(parser.tree());
     query_ = visitor.query();
   }
 
+  PropertyIx Prop(const std::string &prop_name) override {
+    return ast_storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return ast_storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return ast_storage_.GetEdgeTypeIx(name);
+  }
+
   AstStorage ast_storage_;
   Query *query_;
 };
@@ -96,6 +104,18 @@ class OriginalAfterCloningAstGenerator : public AstGenerator {
     AstStorage storage;
     query_->Clone(storage);
   }
+
+  PropertyIx Prop(const std::string &prop_name) override {
+    return ast_storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return ast_storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return ast_storage_.GetEdgeTypeIx(name);
+  }
 };
 
 // This generator clones parsed ast and uses that one.
@@ -106,11 +126,23 @@ class ClonedAstGenerator : public Base {
   explicit ClonedAstGenerator(const std::string &query) : Base(query) {
     ::frontend::opencypher::Parser parser(query);
     AstStorage tmp_storage;
-    CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
+    CypherMainVisitor visitor(context_, &tmp_storage);
     visitor.visit(parser.tree());
     query_ = visitor.query()->Clone(ast_storage_);
   }
 
+  PropertyIx Prop(const std::string &prop_name) override {
+    return ast_storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return ast_storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return ast_storage_.GetEdgeTypeIx(name);
+  }
+
   AstStorage ast_storage_;
   Query *query_;
 };
@@ -125,11 +157,23 @@ class CachedAstGenerator : public Base {
     parameters_ = stripped.literals();
     ::frontend::opencypher::Parser parser(stripped.query());
     AstStorage tmp_storage;
-    CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
+    CypherMainVisitor visitor(context_, &tmp_storage);
     visitor.visit(parser.tree());
     query_ = visitor.query()->Clone(ast_storage_);
   }
 
+  PropertyIx Prop(const std::string &prop_name) override {
+    return ast_storage_.GetPropertyIx(prop_name);
+  }
+
+  LabelIx Label(const std::string &name) override {
+    return ast_storage_.GetLabelIx(name);
+  }
+
+  EdgeTypeIx EdgeType(const std::string &name) override {
+    return ast_storage_.GetEdgeTypeIx(name);
+  }
+
   AstStorage ast_storage_;
   Query *query_;
 };
@@ -165,8 +209,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) {
   auto identifier = dynamic_cast<Identifier *>(property_lookup->expression_);
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
-  ASSERT_EQ(property_lookup->property_,
-            ast_generator.db_accessor_->Property("x"));
+  ASSERT_EQ(property_lookup->property_, ast_generator.Prop("x"));
 }
 
 TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
@@ -184,8 +227,7 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
   ASSERT_TRUE(identifier);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("x"),
-                          ast_generator.db_accessor_->Label("y")));
+              ElementsAre(ast_generator.Label("x"), ast_generator.Label("y")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
@@ -201,7 +243,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   ASSERT_EQ(identifier->name_, "n");
   ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l")));
+              ElementsAre(ast_generator.Label("l-$\"'ab`e``l")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
@@ -217,8 +259,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
         return_clause->body_.named_expressions[0]->expression_);
     auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
     ASSERT_EQ(identifier->name_, "n");
-    ASSERT_THAT(labels_test->labels_,
-                ElementsAre(ast_generator.db_accessor_->Label(label)));
+    ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label(label)));
   }
 }
 
@@ -234,8 +275,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
       return_clause->body_.named_expressions[0]->expression_);
   auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
   EXPECT_EQ(identifier->name_, "n");
-  ASSERT_THAT(labels_test->labels_,
-              ElementsAre(ast_generator.db_accessor_->Label("a")));
+  ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label("a")));
 }
 
 TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) {
@@ -874,12 +914,12 @@ TYPED_TEST(CypherMainVisitorTest, MapLiteral) {
       return_clause->body_.named_expressions[0]->expression_);
   ASSERT_TRUE(map_literal);
   ASSERT_EQ(3, map_literal->elements_.size());
-  ast_generator.CheckLiteral(
-      map_literal->elements_[ast_generator.PropPair("a")], 1);
-  ast_generator.CheckLiteral(
-      map_literal->elements_[ast_generator.PropPair("b")], "bla");
+  ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("a")],
+                             1);
+  ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("b")],
+                             "bla");
   auto *elem_2 = dynamic_cast<ListLiteral *>(
-      map_literal->elements_[ast_generator.PropPair("c")]);
+      map_literal->elements_[ast_generator.Prop("c")]);
   ASSERT_TRUE(elem_2);
   EXPECT_EQ(2, elem_2->elements_.size());
   auto *elem_2_1 = dynamic_cast<MapLiteral *>(elem_2->elements_[1]);
@@ -908,19 +948,19 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
   EXPECT_EQ(node->identifier_->name_,
             CypherMainVisitor::kAnonPrefix + std::to_string(1));
   EXPECT_FALSE(node->identifier_->user_declared_);
-  EXPECT_THAT(node->labels_, UnorderedElementsAre(
-                                 ast_generator.db_accessor_->Label("label1"),
-                                 ast_generator.db_accessor_->Label("label2"),
-                                 ast_generator.db_accessor_->Label("label3")));
-  std::map<std::pair<std::string, storage::Property>, int64_t> properties;
+  EXPECT_THAT(node->labels_,
+              UnorderedElementsAre(ast_generator.Label("label1"),
+                                   ast_generator.Label("label2"),
+                                   ast_generator.Label("label3")));
+  std::unordered_map<PropertyIx, int64_t> properties;
   for (auto x : node->properties_) {
     TypedValue value = ast_generator.LiteralValue(x.second);
     ASSERT_TRUE(value.type() == TypedValue::Type::Int);
     properties[x.first] = value.Value<int64_t>();
   }
   EXPECT_THAT(properties,
-              UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
-                                   Pair(ast_generator.PropPair("b"), 10)));
+              UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
+                                   Pair(ast_generator.Prop("b"), 10)));
 }
 
 TYPED_TEST(CypherMainVisitorTest, PropertyMapSameKeyAppearsTwice) {
@@ -1012,19 +1052,18 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
   auto *edge = dynamic_cast<EdgeAtom *>(match->patterns_[0]->atoms_[1]);
   ASSERT_TRUE(edge);
   EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN);
-  EXPECT_THAT(
-      edge->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
-  std::map<std::pair<std::string, storage::Property>, int64_t> properties;
+  EXPECT_THAT(edge->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
+  std::unordered_map<PropertyIx, int64_t> properties;
   for (auto x : edge->properties_) {
     TypedValue value = ast_generator.LiteralValue(x.second);
     ASSERT_TRUE(value.type() == TypedValue::Type::Int);
     properties[x.first] = value.Value<int64_t>();
   }
   EXPECT_THAT(properties,
-              UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
-                                   Pair(ast_generator.PropPair("b"), 10)));
+              UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
+                                   Pair(ast_generator.Prop("b"), 10)));
 }
 
 TYPED_TEST(CypherMainVisitorTest, RelationshipPatternVariable) {
@@ -1159,8 +1198,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUnboundedWithProperty) {
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   EXPECT_EQ(edge->upper_bound_, nullptr);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
 }
 
 TYPED_TEST(CypherMainVisitorTest,
@@ -1177,10 +1215,9 @@ TYPED_TEST(CypherMainVisitorTest,
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   EXPECT_EQ(edge->upper_bound_, nullptr);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
   ASSERT_EQ(edge->edge_types_.size(), 1U);
-  auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type");
+  auto edge_type = ast_generator.EdgeType("edge_type");
   EXPECT_EQ(edge->edge_types_[0], edge_type);
 }
 
@@ -1197,8 +1234,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUpperBoundedWithProperty) {
   EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
   EXPECT_EQ(edge->lower_bound_, nullptr);
   ast_generator.CheckLiteral(edge->upper_bound_, 2);
-  ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
-                             42);
+  ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
 }
 
 // TODO maybe uncomment
@@ -1323,7 +1359,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(set_property->property_lookup_->property_,
-              ast_generator.db_accessor_->Property("x"));
+              ast_generator.Prop("x"));
     auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_);
     ASSERT_EQ(identifier2->name_, "b");
   }
@@ -1358,8 +1394,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
     ASSERT_TRUE(set_labels->identifier_);
     ASSERT_EQ(set_labels->identifier_->name_, "g");
     ASSERT_THAT(set_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
-                                     ast_generator.db_accessor_->Label("i")));
+                UnorderedElementsAre(ast_generator.Label("h"),
+                                     ast_generator.Label("i")));
   }
 }
 
@@ -1381,7 +1417,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(identifier1);
     ASSERT_EQ(identifier1->name_, "a");
     ASSERT_EQ(remove_property->property_lookup_->property_,
-              ast_generator.db_accessor_->Property("x"));
+              ast_generator.Prop("x"));
   }
   {
     auto *remove_labels =
@@ -1390,8 +1426,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
     ASSERT_TRUE(remove_labels->identifier_);
     ASSERT_EQ(remove_labels->identifier_->name_, "g");
     ASSERT_THAT(remove_labels->labels_,
-                UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
-                                     ast_generator.db_accessor_->Label("i")));
+                UnorderedElementsAre(ast_generator.Label("h"),
+                                     ast_generator.Label("i")));
   }
 }
 
@@ -1598,9 +1634,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties = {
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties = {ast_generator.Prop("slavko")};
   EXPECT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1609,10 +1644,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko"),
-      ast_generator.db_accessor_->Property("pero")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko"),
+                                              ast_generator.Prop("pero")};
   ASSERT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1626,9 +1660,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndexWithSingleProperty) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
   ASSERT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1637,9 +1670,8 @@ TYPED_TEST(CypherMainVisitorTest, DropIndex) {
   auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
   ASSERT_TRUE(index_query);
   EXPECT_EQ(index_query->action_, IndexQuery::Action::DROP);
-  EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
-  std::vector<storage::Property> expected_properties{
-      ast_generator.db_accessor_->Property("slavko")};
+  EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
+  std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
   EXPECT_EQ(index_query->properties_, expected_properties);
 }
 
@@ -1755,10 +1787,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) {
   ASSERT_TRUE(bfs);
   EXPECT_TRUE(bfs->IsVariable());
   EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      bfs->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(bfs->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   EXPECT_EQ(bfs->identifier_->name_, "r");
   EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e");
   EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_);
@@ -1805,10 +1836,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) {
   EXPECT_TRUE(shortest->IsVariable());
   EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(shortest->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   ast_generator.CheckLiteral(shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
@@ -1845,10 +1875,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) {
   EXPECT_TRUE(shortest->IsVariable());
   EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
   EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
-  EXPECT_THAT(
-      shortest->edge_types_,
-      UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
-                           ast_generator.db_accessor_->EdgeType("type2")));
+  EXPECT_THAT(shortest->edge_types_,
+              UnorderedElementsAre(ast_generator.EdgeType("type1"),
+                                   ast_generator.EdgeType("type2")));
   ast_generator.CheckLiteral(shortest->upper_bound_, 10);
   EXPECT_FALSE(shortest->lower_bound_);
   EXPECT_EQ(shortest->identifier_->name_, "r");
diff --git a/tests/unit/distributed_query_plan.cpp b/tests/unit/distributed_query_plan.cpp
index 541474668..646188c41 100644
--- a/tests/unit/distributed_query_plan.cpp
+++ b/tests/unit/distributed_query_plan.cpp
@@ -65,31 +65,32 @@ class DistributedQueryPlan : public DistributedGraphDbTest {
 
 TEST_F(DistributedQueryPlan, PullProduceRpc) {
   auto dba = master().Access();
-  Context ctx{*dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
 
   // Query plan for: UNWIND [42, true, "bla", 1, 2] as x RETURN x
   using namespace query;
   auto list =
       LIST(LITERAL(42), LITERAL(true), LITERAL("bla"), LITERAL(1), LITERAL(2));
-  auto x = ctx.symbol_table_.CreateSymbol("x", true);
+  auto x = symbol_table.CreateSymbol("x", true);
   auto unwind = std::make_shared<plan::Unwind>(nullptr, list, x);
   auto x_expr = IDENT("x");
-  ctx.symbol_table_[*x_expr] = x;
+  symbol_table[*x_expr] = x;
   auto x_ne = NEXPR("x", x_expr);
-  ctx.symbol_table_[*x_ne] = ctx.symbol_table_.CreateSymbol("x_ne", true);
+  symbol_table[*x_ne] = symbol_table.CreateSymbol("x_ne", true);
   auto produce = MakeProduce(unwind, x_ne);
 
   // Test that the plan works locally.
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba);
+  Context ctx = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &ctx);
   ASSERT_EQ(results.size(), 5);
 
   const int plan_id = 42;
   master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
 
   tx::CommandId command_id = dba->transaction().cid();
-  EvaluationContext evaluation_context;
+  auto &evaluation_context = ctx.evaluation_context_;
   std::vector<query::Symbol> symbols{ctx.symbol_table_[*x_ne]};
   auto remote_pull = [this, &command_id, &evaluation_context, &symbols](
                          GraphDbAccessor &dba, int worker_id) {
@@ -152,33 +153,33 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
   }
 
   auto dba = master().Access();
-  Context ctx{*dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
 
   // Query plan for: MATCH p = (n)-[r]->(m) return [n, r], m, p
   // Use this query to test graph elements are transferred correctly in
   // collections too.
-  auto n = MakeScanAll(storage, ctx.symbol_table_, "n");
-  auto r_m = MakeDistributedExpand(storage, ctx.symbol_table_, n.op_, n.sym_,
-                                   "r", EdgeAtom::Direction::OUT, {}, "m",
-                                   false, GraphView::OLD);
-  auto p_sym = ctx.symbol_table_.CreateSymbol("p", true);
+  auto n = MakeScanAll(storage, symbol_table, "n");
+  auto r_m = MakeDistributedExpand(storage, symbol_table, n.op_, n.sym_, "r",
+                                   EdgeAtom::Direction::OUT, {}, "m", false,
+                                   GraphView::OLD);
+  auto p_sym = symbol_table.CreateSymbol("p", true);
   auto p = std::make_shared<query::plan::ConstructNamedPath>(
       r_m.op_, p_sym,
       std::vector<Symbol>{n.sym_, r_m.edge_sym_, r_m.node_sym_});
   auto return_n = IDENT("n");
-  ctx.symbol_table_[*return_n] = n.sym_;
+  symbol_table[*return_n] = n.sym_;
   auto return_r = IDENT("r");
-  ctx.symbol_table_[*return_r] = r_m.edge_sym_;
+  symbol_table[*return_r] = r_m.edge_sym_;
   auto return_n_r = NEXPR("[n, r]", LIST(return_n, return_r));
-  ctx.symbol_table_[*return_n_r] = ctx.symbol_table_.CreateSymbol("", true);
+  symbol_table[*return_n_r] = symbol_table.CreateSymbol("", true);
   auto return_m = NEXPR("m", IDENT("m"));
-  ctx.symbol_table_[*return_m->expression_] = r_m.node_sym_;
-  ctx.symbol_table_[*return_m] = ctx.symbol_table_.CreateSymbol("", true);
+  symbol_table[*return_m->expression_] = r_m.node_sym_;
+  symbol_table[*return_m] = symbol_table.CreateSymbol("", true);
   auto return_p = NEXPR("p", IDENT("p"));
-  ctx.symbol_table_[*return_p->expression_] = p_sym;
-  ctx.symbol_table_[*return_p] = ctx.symbol_table_.CreateSymbol("", true);
+  symbol_table[*return_p->expression_] = p_sym;
+  symbol_table[*return_p] = symbol_table.CreateSymbol("", true);
   auto produce = MakeProduce(p, return_n_r, return_m, return_p);
 
   auto check_result = [prop](int worker_id,
@@ -201,14 +202,15 @@ TEST_F(DistributedQueryPlan, PullProduceRpcWithGraphElements) {
   };
 
   // Test that the plan works locally.
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, *dba);
+  Context ctx = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &ctx);
   check_result(0, results);
 
   const int plan_id = 42;
   master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
 
   tx::CommandId command_id = dba->transaction().cid();
-  EvaluationContext evaluation_context;
+  auto &evaluation_context = ctx.evaluation_context_;
   std::vector<query::Symbol> symbols{ctx.symbol_table_[*return_n_r],
                                      ctx.symbol_table_[*return_m], p_sym};
   auto remote_pull = [this, &command_id, &evaluation_context, &symbols](
@@ -232,24 +234,25 @@ TEST_F(DistributedQueryPlan, Synchronize) {
   // gets reconstructed.
   auto dba_ptr = master().Access();
   auto &dba = *dba_ptr;
-  Context ctx{dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
   // MATCH
-  auto n = MakeScanAll(storage, ctx.symbol_table_, "n");
-  auto r_m = MakeDistributedExpand(storage, ctx.symbol_table_, n.op_, n.sym_,
-                                   "r", EdgeAtom::Direction::BOTH, {}, "m",
-                                   false, GraphView::OLD);
+  auto n = MakeScanAll(storage, symbol_table, "n");
+  auto r_m = MakeDistributedExpand(storage, symbol_table, n.op_, n.sym_, "r",
+                                   EdgeAtom::Direction::BOTH, {}, "m", false,
+                                   GraphView::OLD);
 
   // SET
   auto literal = LITERAL(42);
   auto prop = PROPERTY_PAIR("prop");
   auto m_p = PROPERTY_LOOKUP("m", prop);
-  ctx.symbol_table_[*m_p->expression_] = r_m.node_sym_;
-  auto set_m_p = std::make_shared<plan::SetProperty>(r_m.op_, m_p, literal);
+  symbol_table[*m_p->expression_] = r_m.node_sym_;
+  auto set_m_p =
+      std::make_shared<plan::SetProperty>(r_m.op_, prop.second, m_p, literal);
 
   const int plan_id = 42;
-  master().plan_dispatcher().DispatchPlan(plan_id, set_m_p, ctx.symbol_table_);
+  master().plan_dispatcher().DispatchPlan(plan_id, set_m_p, symbol_table);
 
   // Master-side PullRemote, Synchronize
   auto pull_remote = std::make_shared<query::plan::PullRemote>(
@@ -258,15 +261,14 @@ TEST_F(DistributedQueryPlan, Synchronize) {
       std::make_shared<query::plan::Synchronize>(set_m_p, pull_remote, true);
 
   // RETURN
-  auto n_p =
-      storage.Create<PropertyLookup>(storage.Create<Identifier>("n"), prop);
-  ctx.symbol_table_[*n_p->expression_] = n.sym_;
+  auto n_p = PROPERTY_LOOKUP("n", prop);
+  symbol_table[*n_p->expression_] = n.sym_;
   auto return_n_p = NEXPR("n.prop", n_p);
-  auto return_n_p_sym = ctx.symbol_table_.CreateSymbol("n.p", true);
-  ctx.symbol_table_[*return_n_p] = return_n_p_sym;
+  auto return_n_p_sym = symbol_table.CreateSymbol("n.p", true);
+  symbol_table[*return_n_p] = return_n_p_sym;
   auto produce = MakeProduce(synchronize, return_n_p);
-
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba);
+  Context ctx = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &ctx);
   ASSERT_EQ(results.size(), 2);
   ASSERT_EQ(results[0].size(), 1);
   EXPECT_EQ(results[0][0].ValueInt(), 42);
@@ -279,19 +281,19 @@ TEST_F(DistributedQueryPlan, Synchronize) {
 TEST_F(DistributedQueryPlan, Create) {
   // Query: UNWIND range(0, 1000) as x CREATE ()
   auto dba = master().Access();
-  Context ctx{*dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
   auto range = FN("range", LITERAL(0), LITERAL(1000));
-  auto x = ctx.symbol_table_.CreateSymbol("x", true);
+  auto x = symbol_table.CreateSymbol("x", true);
   auto unwind = std::make_shared<plan::Unwind>(nullptr, range, x);
   NodeCreationInfo node;
-  node.symbol = ctx.symbol_table_.CreateSymbol("n", true);
+  node.symbol = symbol_table.CreateSymbol("n", true);
   auto create =
       std::make_shared<query::plan::DistributedCreateNode>(unwind, node, true);
-  PullAll(create, *dba, ctx.symbol_table_);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  PullAll(*create, &context);
   dba->Commit();
-
   EXPECT_GT(VertexCount(master()), 200);
   EXPECT_GT(VertexCount(worker(1)), 200);
   EXPECT_GT(VertexCount(worker(2)), 200);
@@ -323,29 +325,30 @@ TEST_F(DistributedQueryPlan, PullRemoteOrderBy) {
 
   auto dba_ptr = master().Access();
   auto &dba = *dba_ptr;
-  Context ctx{dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
 
   // Query plan for:  MATCH (n) RETURN n.prop ORDER BY n.prop;
-  auto n = MakeScanAll(storage, ctx.symbol_table_, "n");
+  auto n = MakeScanAll(storage, symbol_table, "n");
   auto n_p = PROPERTY_LOOKUP("n", prop);
-  ctx.symbol_table_[*n_p->expression_] = n.sym_;
+  symbol_table[*n_p->expression_] = n.sym_;
   auto order_by = std::make_shared<plan::OrderBy>(
       n.op_, std::vector<SortItem>{{Ordering::ASC, n_p}},
       std::vector<Symbol>{n.sym_});
 
   const int plan_id = 42;
-  master().plan_dispatcher().DispatchPlan(plan_id, order_by, ctx.symbol_table_);
+  master().plan_dispatcher().DispatchPlan(plan_id, order_by, symbol_table);
 
   auto pull_remote_order_by = std::make_shared<plan::PullRemoteOrderBy>(
       order_by, plan_id, std::vector<SortItem>{{Ordering::ASC, n_p}},
       std::vector<Symbol>{n.sym_});
 
   auto n_p_ne = NEXPR("n.prop", n_p);
-  ctx.symbol_table_[*n_p_ne] = ctx.symbol_table_.CreateSymbol("n.prop", true);
+  symbol_table[*n_p_ne] = symbol_table.CreateSymbol("n.prop", true);
   auto produce = MakeProduce(pull_remote_order_by, n_p_ne);
-  auto results = CollectProduce(produce.get(), ctx.symbol_table_, dba);
+  Context ctx = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &ctx);
 
   ASSERT_EQ(results.size(), 300);
   for (int j = 0; j < 300; ++j) {
@@ -365,24 +368,26 @@ TEST_F(DistributedTransactionTimeout, Timeout) {
   InsertVertex(worker(1));
 
   auto dba = master().Access();
-  Context ctx{*dba};
-  SymbolGenerator symbol_generator{ctx.symbol_table_};
+  SymbolTable symbol_table;
+  SymbolGenerator symbol_generator{symbol_table};
   AstStorage storage;
 
   // Make distributed plan for MATCH (n) RETURN n
-  auto scan_all = MakeScanAll(storage, ctx.symbol_table_, "n");
+  auto scan_all = MakeScanAll(storage, symbol_table, "n");
   auto output = NEXPR("n", IDENT("n"));
   auto produce = MakeProduce(scan_all.op_, output);
-  ctx.symbol_table_[*output->expression_] = scan_all.sym_;
-  ctx.symbol_table_[*output] =
-      ctx.symbol_table_.CreateSymbol("named_expression_1", true);
+  symbol_table[*output->expression_] = scan_all.sym_;
+  symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
 
   const int plan_id = 42;
-  master().plan_dispatcher().DispatchPlan(plan_id, produce, ctx.symbol_table_);
+  master().plan_dispatcher().DispatchPlan(plan_id, produce, symbol_table);
   tx::CommandId command_id = dba->transaction().cid();
 
   EvaluationContext evaluation_context;
-  std::vector<query::Symbol> symbols{ctx.symbol_table_[*output]};
+  evaluation_context.properties =
+      NamesToProperties(storage.properties_, dba.get());
+  evaluation_context.labels = NamesToLabels(storage.labels_, dba.get());
+  std::vector<query::Symbol> symbols{symbol_table[*output]};
   auto remote_pull = [this, &command_id, &evaluation_context, &symbols,
                       &dba]() {
     return master()
@@ -595,11 +600,11 @@ class CapnpPlanner {
  public:
   template <class TDbAccessor>
   CapnpPlanner(std::vector<SingleQueryPart> single_query_parts,
-               PlanningContext<TDbAccessor> &context) {
+               PlanningContext<TDbAccessor> context) {
     ::capnp::MallocMessageBuilder message;
     {
       auto original_plan = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
-          single_query_parts, context);
+          single_query_parts, &context);
       SavePlan(*original_plan, &message);
     }
     {
@@ -625,9 +630,10 @@ DistributedPlan MakeDistributedPlan(query::CypherQuery *query,
                                     query::AstStorage &storage) {
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TPlanner>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TPlanner>(&dba, storage, symbol_table, query);
   std::atomic<int64_t> next_plan_id{0};
-  return MakeDistributedPlan(planner.plan(), symbol_table, next_plan_id);
+  return MakeDistributedPlan(storage, planner.plan(), symbol_table,
+                             next_plan_id, {});
 }
 
 void CheckDistributedPlan(DistributedPlan &distributed_plan,
@@ -651,11 +657,14 @@ void CheckDistributedPlan(DistributedPlan &distributed_plan,
   }
 }
 
-void CheckDistributedPlan(const LogicalOperator &plan,
-                          const SymbolTable &symbol_table,
-                          ExpectedDistributedPlan &expected_distributed_plan) {
+void CheckDistributedPlan(
+    const AstStorage &ast_storage, const LogicalOperator &plan,
+    const SymbolTable &symbol_table,
+    const std::vector<storage::Property> &properties_by_ix,
+    ExpectedDistributedPlan &expected_distributed_plan) {
   std::atomic<int64_t> next_plan_id{0};
-  auto distributed_plan = MakeDistributedPlan(plan, symbol_table, next_plan_id);
+  auto distributed_plan = MakeDistributedPlan(ast_storage, plan, symbol_table,
+                                              next_plan_id, properties_by_ix);
   EXPECT_EQ(next_plan_id - 1, distributed_plan.worker_plans.size());
   CheckDistributedPlan(distributed_plan, expected_distributed_plan);
 }
@@ -708,10 +717,9 @@ class Planner {
  public:
   template <class TDbAccessor>
   Planner(std::vector<SingleQueryPart> single_query_parts,
-          PlanningContext<TDbAccessor> &context) {
-    plan_ = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(single_query_parts,
-                                                            context);
-  }
+          PlanningContext<TDbAccessor> context)
+      : plan_(MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
+            single_query_parts, &context)) {}
 
   auto &plan() { return *plan_; }
 
@@ -733,12 +741,17 @@ TYPED_TEST(TestPlanner, MatchNodeReturn) {
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectProduce(), pull),
                         MakeCheckers(ExpectScanAll(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, CreateNodeReturn) {
@@ -750,13 +763,13 @@ TYPED_TEST(TestPlanner, CreateNodeReturn) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectDistributedCreateNode(true), ExpectSynchronize(false),
                    ExpectProduce()));
   std::atomic<int64_t> next_plan_id{0};
-  auto distributed_plan =
-      MakeDistributedPlan(planner.plan(), symbol_table, next_plan_id);
+  auto distributed_plan = MakeDistributedPlan(storage, planner.plan(),
+                                              symbol_table, next_plan_id, {});
   CheckDistributedPlan(distributed_plan, expected);
 }
 
@@ -764,7 +777,7 @@ TYPED_TEST(TestPlanner, CreateExpand) {
   // Test CREATE (n) -[r :rel1]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
       NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
   ExpectedDistributedPlan expected{
@@ -790,7 +803,7 @@ TYPED_TEST(TestPlanner, CreateNodeExpandNode) {
   // Test CREATE (n) -[r :rel]-> (m), (l)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("rel");
+  auto relationship = "rel";
   auto *query = QUERY(SINGLE_QUERY(CREATE(
       PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")),
       PATTERN(NODE("l")))));
@@ -806,7 +819,7 @@ TYPED_TEST(TestPlanner, CreateNamedPattern) {
   // Test CREATE p = (n) -[r :rel]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("rel");
+  auto relationship = "rel";
   auto *query = QUERY(SINGLE_QUERY(CREATE(NAMED_PATTERN(
       "p", NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
   ExpectedDistributedPlan expected{
@@ -821,7 +834,7 @@ TYPED_TEST(TestPlanner, MatchCreateExpand) {
   // Test MATCH (n) CREATE (n) -[r :rel1]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))),
       CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}),
@@ -837,45 +850,55 @@ TYPED_TEST(TestPlanner, MatchLabeledNodes) {
   // Test MATCH (n :label) RETURN n
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *as_n = NEXPR("n", IDENT("n"));
   auto *query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAllByLabel(), ExpectProduce(), pull),
       MakeCheckers(ExpectScanAllByLabel(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchPathReturn) {
   // Test MATCH (n) -[r :relationship]- (m) RETURN n
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_n = NEXPR("n", IDENT("n"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {relationship}),
                     NODE("m"))),
       RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                                      ExpectProduce(), pull),
                         MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                                      ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
   // Test MATCH p = (n) -[r :relationship]- (m) RETURN p
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_p = NEXPR("p", IDENT("p"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(NAMED_PATTERN("p", NODE("n"),
@@ -883,21 +906,26 @@ TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
                           NODE("m"))),
       RETURN(as_p)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_p)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                    ExpectConstructNamedPath(), ExpectProduce(), pull),
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                    ExpectConstructNamedPath(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
   // Test MATCH p = (n) -[r :relationship]- (m) WHERE 2 = p RETURN p
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_p = NEXPR("p", IDENT("p"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(NAMED_PATTERN("p", NODE("n"),
@@ -905,7 +933,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
                           NODE("m"))),
       WHERE(EQ(LITERAL(2), IDENT("p"))), RETURN(as_p)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_p)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
@@ -914,7 +942,12 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
                         MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                                      ExpectConstructNamedPath(), ExpectFilter(),
                                      ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, OptionalMatchNamedPatternReturn) {
@@ -933,7 +966,7 @@ TYPED_TEST(TestPlanner, OptionalMatchNamedPatternReturn) {
   std::vector<Symbol> optional_symbols{get_symbol(pattern), get_symbol(node_n),
                                        get_symbol(edge), get_symbol(node_m)};
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   std::list<BaseOpChecker *> optional{
       new ExpectScanAll(), new ExpectDistributedExpand(),
       new ExpectConstructNamedPath(), new ExpectPullRemote(optional_symbols)};
@@ -942,7 +975,12 @@ TYPED_TEST(TestPlanner, OptionalMatchNamedPatternReturn) {
                    ExpectProduce()),
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                    ExpectConstructNamedPath()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchWhereReturn) {
@@ -955,12 +993,17 @@ TYPED_TEST(TestPlanner, MatchWhereReturn) {
       MATCH(PATTERN(NODE("n"))),
       WHERE(LESS(PROPERTY_LOOKUP("n", property), LITERAL(42))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectFilter(), ExpectProduce(), pull),
       MakeCheckers(ExpectScanAll(), ExpectFilter(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchDelete) {
@@ -979,7 +1022,7 @@ TYPED_TEST(TestPlanner, MatchNodeSet) {
   AstStorage storage;
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
                                    SET(PROPERTY_LOOKUP("n", prop), LITERAL(42)),
                                    SET("n", IDENT("n")), SET("n", {label})));
@@ -996,7 +1039,7 @@ TYPED_TEST(TestPlanner, MatchRemove) {
   AstStorage storage;
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
                                    REMOVE(PROPERTY_LOOKUP("n", prop)),
                                    REMOVE("n", {label})));
@@ -1024,7 +1067,7 @@ TYPED_TEST(TestPlanner, MultiMatch) {
       MATCH(PATTERN(node_j, edge_e, node_i, edge_f, node_h)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto get_symbol = [&symbol_table](const auto *atom_node) {
     return symbol_table.at(*atom_node->identifier_);
   };
@@ -1035,17 +1078,21 @@ TYPED_TEST(TestPlanner, MultiMatch) {
   ExpectPullRemote right_pull({get_symbol(node_j), get_symbol(edge_e),
                                get_symbol(node_i), get_symbol(edge_f),
                                get_symbol(node_h)});
-  auto right_cart = MakeCheckers(
-      ExpectScanAll(), ExpectDistributedExpand(), ExpectDistributedExpand(),
-      ExpectEdgeUniquenessFilter(), right_pull);
+  auto right_cart = MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
+                                 ExpectDistributedExpand(),
+                                 ExpectEdgeUniquenessFilter(), right_pull);
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectDistributedCartesian(left_cart, right_cart),
                    ExpectProduce()),
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand()),
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
-                   ExpectDistributedExpand(),
-                   ExpectEdgeUniquenessFilter()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+                   ExpectDistributedExpand(), ExpectEdgeUniquenessFilter()));
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MultiMatchSameStart) {
@@ -1059,14 +1106,19 @@ TYPED_TEST(TestPlanner, MultiMatchSameStart) {
   // MATCH clause.
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                                      ExpectProduce(), pull),
                         MakeCheckers(ExpectScanAll(), ExpectDistributedExpand(),
                                      ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchWithReturn) {
@@ -1078,12 +1130,17 @@ TYPED_TEST(TestPlanner, MatchWithReturn) {
   // No accumulation since we only do reads.
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_new)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectProduce(), ExpectProduce(), pull),
       MakeCheckers(ExpectScanAll(), ExpectProduce(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
@@ -1097,21 +1154,26 @@ TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
       WHERE(LESS(PROPERTY_LOOKUP("new", prop), LITERAL(42))), RETURN(as_new)));
   // No accumulation since we only do reads.
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_new)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectProduce(),
                                      ExpectFilter(), ExpectProduce(), pull),
                         MakeCheckers(ExpectScanAll(), ExpectProduce(),
                                      ExpectFilter(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, CreateMultiExpand) {
   // Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
   FakeDbAccessor dba;
-  auto r = dba.EdgeType("r");
-  auto p = dba.EdgeType("p");
+  auto r = "r";
+  auto p = "p";
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
       CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {r}), NODE("m")),
@@ -1136,10 +1198,10 @@ TYPED_TEST(TestPlanner, MatchReturnSum) {
       MATCH(PATTERN(NODE("n"))), RETURN(sum, AS("sum"), n_prop2, AS("group"))));
   auto aggr = ExpectAggregate({sum}, {n_prop2});
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   std::atomic<int64_t> next_plan_id{0};
-  auto distributed_plan =
-      MakeDistributedPlan(planner.plan(), symbol_table, next_plan_id);
+  auto distributed_plan = MakeDistributedPlan(storage, planner.plan(),
+                                              symbol_table, next_plan_id, {});
   auto merge_sum = SUM(IDENT("worker_sum"));
   auto master_aggr = ExpectMasterAggregate({merge_sum}, {n_prop2});
   ExpectPullRemote pull(
@@ -1154,7 +1216,7 @@ TYPED_TEST(TestPlanner, MatchReturnSum) {
 TYPED_TEST(TestPlanner, MatchWithCreate) {
   // Test MATCH (n) WITH n AS a CREATE (a) -[r :r]-> (b)
   FakeDbAccessor dba;
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))), WITH("n", AS("a")),
@@ -1177,13 +1239,18 @@ TYPED_TEST(TestPlanner, MatchReturnSkipLimit) {
                          RETURN(as_n, SKIP(LITERAL(2)), LIMIT(LITERAL(1)))));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected =
       ExpectDistributed(MakeCheckers(ExpectScanAll(), ExpectProduce(), pull,
                                      ExpectSkip(), ExpectLimit()),
                         MakeCheckers(ExpectScanAll(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, CreateWithSkipReturnLimit) {
@@ -1196,13 +1263,18 @@ TYPED_TEST(TestPlanner, CreateWithSkipReturnLimit) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectedDistributedPlan expected{
       MakeCheckers(ExpectDistributedCreateNode(true), ExpectSynchronize(true),
                    ExpectProduce(), ExpectSkip(), ExpectProduce(),
                    ExpectLimit()),
       {}};
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, MatchReturnOrderBy) {
@@ -1215,14 +1287,19 @@ TYPED_TEST(TestPlanner, MatchReturnOrderBy) {
   auto ret = RETURN(as_m, ORDER_BY(PROPERTY_LOOKUP("n", prop)));
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), ret));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemoteOrderBy pull_order_by(
       {symbol_table.at(*as_m), symbol_table.at(*node_n->identifier_)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectProduce(), ExpectOrderBy(),
                    pull_order_by),
       MakeCheckers(ExpectScanAll(), ExpectProduce(), ExpectOrderBy()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
   // Even though last operator pulls and orders by `m` and `n`, we expect only
   // `m` as the output of the query execution.
   EXPECT_THAT(planner.plan().OutputSymbols(symbol_table),
@@ -1234,7 +1311,7 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
   //      WITH n AS new ORDER BY new.prop, r.prop WHERE m.prop < 42
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   AstStorage storage;
   auto ident_n = IDENT("n");
   auto new_prop = PROPERTY_LOOKUP("new", prop);
@@ -1252,12 +1329,17 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
       symbol_table.at(*r_prop->expression_),  // `r` in ORDER BY
       symbol_table.at(*m_prop->expression_),  // `m` in WHERE
   });
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectDistributedCreateNode(true),
                    ExpectDistributedCreateExpand(), ExpectSynchronize(true),
                    ExpectProduce(), ExpectOrderBy(), ExpectFilter()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, ReturnAddSumCountOrderBy) {
@@ -1284,12 +1366,17 @@ TYPED_TEST(TestPlanner, MatchUnwindReturn) {
                    RETURN(as_n, as_x)));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n), symbol_table.at(*as_x)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectUnwind(), ExpectProduce(), pull),
       MakeCheckers(ExpectScanAll(), ExpectUnwind(), ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, ReturnDistinctOrderBySkipLimit) {
@@ -1315,14 +1402,19 @@ TYPED_TEST(TestPlanner, MatchWhereBeforeExpand) {
       WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))), RETURN(as_n)));
   // We expect Filter to come immediately after ScanAll, since it only uses `n`.
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   ExpectPullRemote pull({symbol_table.at(*as_n)});
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectFilter(), ExpectDistributedExpand(),
                    ExpectProduce(), pull),
       MakeCheckers(ExpectScanAll(), ExpectFilter(), ExpectDistributedExpand(),
                    ExpectProduce()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, FunctionAggregationReturn) {
@@ -1348,11 +1440,11 @@ TYPED_TEST(TestPlanner, FunctionWithoutArguments) {
 TYPED_TEST(TestPlanner, MatchBfs) {
   // Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
   FakeDbAccessor dba;
-  auto edge_type = dba.EdgeType("type");
   AstStorage storage;
+  auto edge_type = storage.GetEdgeTypeIx("type");
   auto *bfs = storage.Create<query::EdgeAtom>(
       IDENT("r"), query::EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
-      std::vector<storage::EdgeType>{edge_type});
+      std::vector<EdgeTypeIx>{edge_type});
   bfs->filter_lambda_.inner_edge = IDENT("r");
   bfs->filter_lambda_.inner_node = IDENT("n");
   bfs->filter_lambda_.expression = IDENT("n");
@@ -1433,20 +1525,25 @@ TYPED_TEST(TestPlanner, DistributedMatchCreateReturn) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*ident_m)});
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto expected = ExpectDistributed(
       MakeCheckers(ExpectScanAll(), ExpectDistributedCreateNode(),
                    ExpectSynchronize({symbol_table.at(*ident_m)}),
                    ExpectProduce()),
       MakeCheckers(ExpectScanAll(), ExpectDistributedCreateNode()));
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianCreateExpand) {
   // Test MATCH (a), (b) CREATE (a)-[e:r]->(b) RETURN e
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("r");
+  auto relationship = "r";
   auto *node_a = NODE("a");
   auto *node_b = NODE("b");
   auto *query = QUERY(SINGLE_QUERY(
@@ -1466,8 +1563,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianCreateExpand) {
                    ExpectDistributedCreateExpand(), ExpectSynchronize(false),
                    ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianExpand) {
@@ -1493,8 +1595,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianExpand) {
       MakeCheckers(ExpectScanAll()),
       MakeCheckers(ExpectScanAll(), ExpectDistributedExpand()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianExpandToExisting) {
@@ -1515,8 +1622,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianExpandToExisting) {
                    ExpectDistributedExpand(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianExpandFromExisting) {
@@ -1538,8 +1650,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianExpandFromExisting) {
                    ExpectDistributedExpand(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianFilter) {
@@ -1569,21 +1686,27 @@ TYPED_TEST(TestPlanner, DistributedCartesianFilter) {
       MakeCheckers(ExpectScanAll(), ExpectFilter()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByProperty) {
   // Test MATCH (a), (b :label) WHERE b.prop = a RETURN b
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label_name = "label";
+  auto label = dba.Label(label_name);
   auto prop = dba.Property("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
   auto *node_a = NODE("a");
-  auto *node_b = NODE("b", label);
+  auto *node_b = NODE("b", label_name);
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(node_a), PATTERN(node_b)),
       WHERE(EQ(PROPERTY_LOOKUP("b", prop), IDENT("a"))), RETURN("b")));
@@ -1599,21 +1722,27 @@ TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByProperty) {
       MakeCheckers(ExpectDistributedCartesian(left_cart, right_cart),
                    ExpectFilter(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAllByLabel()));
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByLowerBound) {
   // Test MATCH (a), (b :label) WHERE a < b.prop RETURN b
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label_name = "label";
+  auto label = dba.Label(label_name);
   auto prop = dba.Property("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
   auto *node_a = NODE("a");
-  auto *node_b = NODE("b", label);
+  auto *node_b = NODE("b", label_name);
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(node_a), PATTERN(node_b)),
       WHERE(LESS(IDENT("a"), PROPERTY_LOOKUP("b", prop))), RETURN("b")));
@@ -1629,21 +1758,27 @@ TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByLowerBound) {
       MakeCheckers(ExpectDistributedCartesian(left_cart, right_cart),
                    ExpectFilter(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAllByLabel()));
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByUpperBound) {
   // Test MATCH (a), (b :label) WHERE a > b.prop RETURN b
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label_name = "label";
+  auto label = dba.Label(label_name);
   auto prop = dba.Property("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
   auto *node_a = NODE("a");
-  auto *node_b = NODE("b", label);
+  auto *node_b = NODE("b", label_name);
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(node_a), PATTERN(node_b)),
       WHERE(GREATER(IDENT("a"), PROPERTY_LOOKUP("b", prop))), RETURN("b")));
@@ -1659,8 +1794,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianIndexedScanByUpperBound) {
       MakeCheckers(ExpectDistributedCartesian(left_cart, right_cart),
                    ExpectFilter(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAllByLabel()));
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TEST(TestPlanner, DistributedCartesianIndexedScanByBothBounds) {
@@ -1669,6 +1809,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByBothBounds) {
   FakeDbAccessor dba;
   auto label = dba.Label("label");
   auto prop = dba.Property("prop");
+  storage.GetPropertyIx("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
@@ -1683,7 +1824,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByBothBounds) {
   symbol_table[*upper_expr] = sym_a;
   auto upper_bound = utils::MakeBoundExclusive(upper_expr);
   auto scan_b = std::make_shared<ScanAllByLabelPropertyRange>(
-      scan_a, sym_b, label, prop, lower_bound, upper_bound);
+      scan_a, sym_b, label, prop, "prop", lower_bound, upper_bound);
   auto ident_b = IDENT("b");
   symbol_table[*ident_b] = sym_b;
   auto as_b = NEXPR("b", ident_b);
@@ -1698,7 +1839,12 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByBothBounds) {
       MakeCheckers(ExpectDistributedCartesian(left_cart, right_cart),
                    ExpectFilter(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAllByLabel()));
-  CheckDistributedPlan(*produce, symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, *produce, symbol_table, properties_by_ix,
+                       expected);
 }
 
 TEST(TestPlanner, DistributedCartesianIndexedScanByLowerWithBothBounds) {
@@ -1707,6 +1853,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByLowerWithBothBounds) {
   FakeDbAccessor dba;
   auto label = dba.Label("label");
   auto prop = dba.Property("prop");
+  storage.GetPropertyIx("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
@@ -1720,7 +1867,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByLowerWithBothBounds) {
   symbol_table[*upper_expr] = sym_a;
   auto upper_bound = utils::MakeBoundExclusive(upper_expr);
   auto scan_b = std::make_shared<ScanAllByLabelPropertyRange>(
-      scan_a, sym_b, label, prop, lower_bound, upper_bound);
+      scan_a, sym_b, label, prop, "prop", lower_bound, upper_bound);
   auto ident_b = IDENT("b");
   symbol_table[*ident_b] = sym_b;
   auto as_b = NEXPR("b", ident_b);
@@ -1739,7 +1886,12 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByLowerWithBothBounds) {
       MakeCheckers(ExpectScanAll()),
       MakeCheckers(ExpectScanAllByLabelPropertyRange(
           label, prop, lower_bound, std::experimental::nullopt)));
-  CheckDistributedPlan(*produce, symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, *produce, symbol_table, properties_by_ix,
+                       expected);
 }
 
 TEST(TestPlanner, DistributedCartesianIndexedScanByUpperWithBothBounds) {
@@ -1748,6 +1900,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByUpperWithBothBounds) {
   FakeDbAccessor dba;
   auto label = dba.Label("label");
   auto prop = dba.Property("prop");
+  storage.GetPropertyIx("prop");
   // Set indexes so that lookup by property is preferred.
   dba.SetIndexCount(label, 1024);
   dba.SetIndexCount(label, prop, 0);
@@ -1761,7 +1914,7 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByUpperWithBothBounds) {
   query::Expression *upper_expr = LITERAL(42);
   auto upper_bound = utils::MakeBoundExclusive(upper_expr);
   auto scan_b = std::make_shared<ScanAllByLabelPropertyRange>(
-      scan_a, sym_b, label, prop, lower_bound, upper_bound);
+      scan_a, sym_b, label, prop, "prop", lower_bound, upper_bound);
   auto ident_b = IDENT("b");
   symbol_table[*ident_b] = sym_b;
   auto as_b = NEXPR("b", ident_b);
@@ -1780,7 +1933,12 @@ TEST(TestPlanner, DistributedCartesianIndexedScanByUpperWithBothBounds) {
       MakeCheckers(ExpectScanAll()),
       MakeCheckers(ExpectScanAllByLabelPropertyRange(
           label, prop, std::experimental::nullopt, upper_bound)));
-  CheckDistributedPlan(*produce, symbol_table, expected);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, *produce, symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianProduce) {
@@ -1803,8 +1961,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianProduce) {
       MakeCheckers(ExpectScanAll(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianUnwind) {
@@ -1824,8 +1987,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianUnwind) {
                    ExpectUnwind(), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianMatchCreateNode) {
@@ -1849,8 +2017,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianMatchCreateNode) {
       MakeCheckers(ExpectScanAll(), ExpectDistributedCreateNode()),
       MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianCreateNode) {
@@ -1871,8 +2044,13 @@ TYPED_TEST(TestPlanner, DistributedCartesianCreateNode) {
                    ExpectProduce()),
       MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedOptionalExpand) {
@@ -1894,8 +2072,13 @@ TYPED_TEST(TestPlanner, DistributedOptionalExpand) {
       MakeCheckers(ExpectScanAll(), ExpectDistributedOptional(optional),
                    ExpectProduce()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedOptionalCartesian) {
@@ -1922,20 +2105,26 @@ TYPED_TEST(TestPlanner, DistributedOptionalCartesian) {
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()),
       MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedCartesianTransitiveDependency) {
   // Test MATCH (n:L)-[a]-(m:L)-[b]-(l:L) RETURN l;
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("L");
+  auto label_name = "L";
+  auto label = dba.Label(label_name);
   // Set indexes so that multiple scans and expanding to existing is preferred.
   dba.SetIndexCount(label, 1);
-  auto *node_n = NODE("n", label);
-  auto *node_m = NODE("m", label);
-  auto *node_l = NODE("l", label);
+  auto *node_n = NODE("n", label_name);
+  auto *node_m = NODE("m", label_name);
+  auto *node_l = NODE("l", label_name);
   auto *edge_a = EDGE("a");
   auto *edge_b = EDGE("b");
   auto *query = QUERY(SINGLE_QUERY(
@@ -1961,13 +2150,17 @@ TYPED_TEST(TestPlanner, DistributedCartesianTransitiveDependency) {
                    // This expand depends on the previous one.
                    ExpectDistributedExpand(),
                    // UniquenessFilter depends on both expands.
-                   ExpectEdgeUniquenessFilter(),
-                   ExpectProduce()),
+                   ExpectEdgeUniquenessFilter(), ExpectProduce()),
       MakeCheckers(ExpectScanAllByLabel()),
       MakeCheckers(ExpectScanAllByLabel()),
       MakeCheckers(ExpectScanAllByLabel()));
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TYPED_TEST(TestPlanner, DistributedOptionalScanExpandExisting) {
@@ -1989,8 +2182,13 @@ TYPED_TEST(TestPlanner, DistributedOptionalScanExpandExisting) {
                    ExpectDistributedOptional(optional), ExpectProduce()),
       MakeCheckers(ExpectScanAll()), MakeCheckers(ExpectScanAll()));
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
-  CheckDistributedPlan(planner.plan(), symbol_table, expected);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
+  std::vector<storage::Property> properties_by_ix;
+  for (const auto &prop : storage.properties_) {
+    properties_by_ix.push_back(dba.Property(prop));
+  }
+  CheckDistributedPlan(storage, planner.plan(), symbol_table, properties_by_ix,
+                       expected);
 }
 
 TEST(CapnpSerial, Union) {
diff --git a/tests/unit/query_common.hpp b/tests/unit/query_common.hpp
index 7a75c4e85..5317469d6 100644
--- a/tests/unit/query_common.hpp
+++ b/tests/unit/query_common.hpp
@@ -57,15 +57,15 @@ auto ToMap(const TypedValue &t) {
   return map;
 };
 
-std::string ToString(Expression *expr) {
+std::string ToString(const AstStorage &storage, Expression *expr) {
   std::ostringstream ss;
-  PrintExpression(expr, &ss);
+  PrintExpression(storage, expr, &ss);
   return ss.str();
 }
 
-std::string ToString(NamedExpression *expr) {
+std::string ToString(const AstStorage &storage, NamedExpression *expr) {
   std::ostringstream ss;
-  PrintExpression(expr, &ss);
+  PrintExpression(storage, expr, &ss);
   return ss.str();
 }
 
@@ -121,15 +121,22 @@ auto GetOrderBy(T... exprs) {
 template <class TDbAccessor>
 auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba,
                        const std::string &name, storage::Property property) {
-  return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
-                                        dba.PropertyName(property), property);
+  return storage.Create<PropertyLookup>(
+      storage.Create<Identifier>(name),
+      storage.GetPropertyIx(dba.PropertyName(property)));
 }
 
 template <class TDbAccessor>
 auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba, Expression *expr,
                        storage::Property property) {
-  return storage.Create<PropertyLookup>(expr, dba.PropertyName(property),
-                                        property);
+  return storage.Create<PropertyLookup>(
+      expr, storage.GetPropertyIx(dba.PropertyName(property)));
+}
+
+template <class TDbAccessor>
+auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba, Expression *expr,
+                       const std::string &property) {
+  return storage.Create<PropertyLookup>(expr, storage.GetPropertyIx(property));
 }
 
 template <class TDbAccessor>
@@ -137,15 +144,15 @@ auto GetPropertyLookup(
     AstStorage &storage, TDbAccessor &, const std::string &name,
     const std::pair<std::string, storage::Property> &prop_pair) {
   return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
-                                        prop_pair.first, prop_pair.second);
+                                        storage.GetPropertyIx(prop_pair.first));
 }
 
 template <class TDbAccessor>
 auto GetPropertyLookup(
     AstStorage &storage, TDbAccessor &, Expression *expr,
     const std::pair<std::string, storage::Property> &prop_pair) {
-  return storage.Create<PropertyLookup>(expr, prop_pair.first,
-                                        prop_pair.second);
+  return storage.Create<PropertyLookup>(expr,
+                                        storage.GetPropertyIx(prop_pair.first));
 }
 
 /// Create an EdgeAtom with given name, direction and edge_type.
@@ -153,9 +160,14 @@ auto GetPropertyLookup(
 /// Name is used to create the Identifier which is assigned to the edge.
 auto GetEdge(AstStorage &storage, const std::string &name,
              EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
-             const std::vector<storage::EdgeType> &edge_types = {}) {
+             const std::vector<std::string> &edge_types = {}) {
+  std::vector<EdgeTypeIx> types;
+  types.reserve(edge_types.size());
+  for (const auto &type : edge_types) {
+    types.push_back(storage.GetEdgeTypeIx(type));
+  }
   return storage.Create<EdgeAtom>(storage.Create<Identifier>(name),
-                                  EdgeAtom::Type::SINGLE, dir, edge_types);
+                                  EdgeAtom::Type::SINGLE, dir, types);
 }
 
 /// Create a variable length expansion EdgeAtom with given name, direction and
@@ -165,15 +177,20 @@ auto GetEdge(AstStorage &storage, const std::string &name,
 auto GetEdgeVariable(AstStorage &storage, const std::string &name,
                      EdgeAtom::Type type = EdgeAtom::Type::DEPTH_FIRST,
                      EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
-                     const std::vector<storage::EdgeType> &edge_types = {},
+                     const std::vector<std::string> &edge_types = {},
                      Identifier *flambda_inner_edge = nullptr,
                      Identifier *flambda_inner_node = nullptr,
                      Identifier *wlambda_inner_edge = nullptr,
                      Identifier *wlambda_inner_node = nullptr,
                      Expression *wlambda_expression = nullptr,
                      Identifier *total_weight = nullptr) {
+  std::vector<EdgeTypeIx> types;
+  types.reserve(edge_types.size());
+  for (const auto &type : edge_types) {
+    types.push_back(storage.GetEdgeTypeIx(type));
+  }
   auto r_val = storage.Create<EdgeAtom>(storage.Create<Identifier>(name), type,
-                                        dir, edge_types);
+                                        dir, types);
 
   r_val->filter_lambda_.inner_edge =
       flambda_inner_edge ? flambda_inner_edge
@@ -205,10 +222,10 @@ auto GetEdgeVariable(AstStorage &storage, const std::string &name,
 ///
 /// Name is used to create the Identifier which is assigned to the node.
 auto GetNode(AstStorage &storage, const std::string &name,
-             std::experimental::optional<storage::Label> label =
+             std::experimental::optional<std::string> label =
                  std::experimental::nullopt) {
   auto node = storage.Create<NodeAtom>(storage.Create<Identifier>(name));
-  if (label) node->labels_.emplace_back(*label);
+  if (label) node->labels_.emplace_back(storage.GetLabelIx(*label));
   return node;
 }
 
@@ -444,7 +461,12 @@ auto GetSet(AstStorage &storage, const std::string &name, Expression *expr,
 
 /// Create a set labels clause for given identifier name and labels.
 auto GetSet(AstStorage &storage, const std::string &name,
-            std::vector<storage::Label> labels) {
+            std::vector<std::string> label_names) {
+  std::vector<LabelIx> labels;
+  labels.reserve(label_names.size());
+  for (const auto &label : label_names) {
+    labels.push_back(storage.GetLabelIx(label));
+  }
   return storage.Create<SetLabels>(storage.Create<Identifier>(name), labels);
 }
 
@@ -455,7 +477,12 @@ auto GetRemove(AstStorage &storage, PropertyLookup *prop_lookup) {
 
 /// Create a remove labels clause for given identifier name and labels.
 auto GetRemove(AstStorage &storage, const std::string &name,
-               std::vector<storage::Label> labels) {
+               std::vector<std::string> label_names) {
+  std::vector<LabelIx> labels;
+  labels.reserve(label_names.size());
+  for (const auto &label : label_names) {
+    labels.push_back(storage.GetLabelIx(label));
+  }
   return storage.Create<RemoveLabels>(storage.Create<Identifier>(name), labels);
 }
 
@@ -513,10 +540,9 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
 #define LIST(...)                     \
   storage.Create<query::ListLiteral>( \
       std::vector<query::Expression *>{__VA_ARGS__})
-#define MAP(...)                                                    \
-  storage.Create<query::MapLiteral>(                                \
-      std::unordered_map<std::pair<std::string, storage::Property>, \
-                         query::Expression *>{__VA_ARGS__})
+#define MAP(...)                     \
+  storage.Create<query::MapLiteral>( \
+      std::unordered_map<query::PropertyIx, query::Expression *>{__VA_ARGS__})
 #define PROPERTY_PAIR(property_name) \
   std::make_pair(property_name, dba.Property(property_name))
 #define PROPERTY_LOOKUP(...) \
@@ -557,7 +583,7 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
 #define CREATE_INDEX_ON(label, property)          \
   storage.Create<query::IndexQuery>(              \
       query::IndexQuery::Action::CREATE, (label), \
-      std::vector<storage::Property>{(property)})
+      std::vector<query::PropertyIx>{(property)})
 #define QUERY(...) query::test_common::GetQuery(storage, __VA_ARGS__)
 #define SINGLE_QUERY(...) \
   query::test_common::GetSingleQuery(storage.Create<SingleQuery>(), __VA_ARGS__)
diff --git a/tests/unit/query_cost_estimator.cpp b/tests/unit/query_cost_estimator.cpp
index 62cd97acd..9d9ffde02 100644
--- a/tests/unit/query_cost_estimator.cpp
+++ b/tests/unit/query_cost_estimator.cpp
@@ -62,7 +62,7 @@ class QueryCostEstimator : public ::testing::Test {
   }
 
   auto Cost() {
-    CostEstimator<database::GraphDbAccessor> cost_estimator(*dba, parameters_);
+    CostEstimator<database::GraphDbAccessor> cost_estimator(dba.get(), parameters_);
     last_op_->Accept(cost_estimator);
     return cost_estimator.cost();
   }
@@ -112,7 +112,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyValueConstant) {
   AddVertices(100, 30, 20);
   for (auto const_val : {Literal(12), Parameter(12)}) {
     MakeOp<ScanAllByLabelPropertyValue>(nullptr, NextSymbol(), label, property,
-                                        const_val);
+                                        "property", const_val);
     EXPECT_COST(1 * CostParam::MakeScanAllByLabelPropertyValue);
   }
 }
@@ -121,7 +121,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyValueConstExpr) {
   AddVertices(100, 30, 20);
   for (auto const_val : {Literal(12), Parameter(12)}) {
     MakeOp<ScanAllByLabelPropertyValue>(
-        nullptr, NextSymbol(), label, property,
+        nullptr, NextSymbol(), label, property, "property",
         // once we make expression const-folding this test case will fail
         storage_.Create<UnaryPlusOperator>(const_val));
     EXPECT_COST(20 * CardParam::kFilter *
@@ -133,7 +133,8 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeUpperConstant) {
   AddVertices(100, 30, 20);
   for (auto const_val : {Literal(12), Parameter(12)}) {
     MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
-                                        nullopt, InclusiveBound(const_val));
+                                        "property", nullopt,
+                                        InclusiveBound(const_val));
     // cardinality estimation is exact for very small indexes
     EXPECT_COST(13 * CostParam::MakeScanAllByLabelPropertyRange);
   }
@@ -143,12 +144,14 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeLowerConstant) {
   AddVertices(100, 30, 20);
   for (auto const_val : {Literal(17), Parameter(17)}) {
     MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
-                                        InclusiveBound(const_val), nullopt);
+                                        "property", InclusiveBound(const_val),
+                                        nullopt);
     // cardinality estimation is exact for very small indexes
     EXPECT_COST(3 * CostParam::MakeScanAllByLabelPropertyRange);
   }
 }
 
+
 TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
   AddVertices(100, 30, 20);
   for (auto const_val : {Literal(12), Parameter(12)}) {
@@ -156,7 +159,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
         utils::MakeBoundInclusive(static_cast<Expression *>(
             storage_.Create<UnaryPlusOperator>(const_val))));
     MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
-                                        bound, nullopt);
+                                        "property", bound, nullopt);
     EXPECT_COST(20 * CardParam::kFilter *
                 CostParam::MakeScanAllByLabelPropertyRange);
   }
diff --git a/tests/unit/query_expression_evaluator.cpp b/tests/unit/query_expression_evaluator.cpp
index 115eaf0b6..423f52fbf 100644
--- a/tests/unit/query_expression_evaluator.cpp
+++ b/tests/unit/query_expression_evaluator.cpp
@@ -48,17 +48,24 @@ class ExpressionEvaluatorTest : public ::testing::Test {
     frame[symbol] = value;
     return id;
   }
+
+  template <class TExpression>
+  auto Eval(TExpression *expr) {
+    ctx.properties = NamesToProperties(storage.properties_, dba.get());
+    ctx.labels = NamesToLabels(storage.labels_, dba.get());
+    return expr->Accept(eval);
+  }
 };
 
 TEST_F(ExpressionEvaluatorTest, OrOperator) {
   auto *op =
       storage.Create<OrOperator>(storage.Create<PrimitiveLiteral>(true),
                                  storage.Create<PrimitiveLiteral>(false));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<OrOperator>(storage.Create<PrimitiveLiteral>(true),
                                   storage.Create<PrimitiveLiteral>(true));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), true);
 }
 
@@ -66,11 +73,11 @@ TEST_F(ExpressionEvaluatorTest, XorOperator) {
   auto *op =
       storage.Create<XorOperator>(storage.Create<PrimitiveLiteral>(true),
                                   storage.Create<PrimitiveLiteral>(false));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<XorOperator>(storage.Create<PrimitiveLiteral>(true),
                                    storage.Create<PrimitiveLiteral>(true));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), false);
 }
 
@@ -78,11 +85,11 @@ TEST_F(ExpressionEvaluatorTest, AndOperator) {
   auto *op =
       storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(true),
                                   storage.Create<PrimitiveLiteral>(true));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(false),
                                    storage.Create<PrimitiveLiteral>(true));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), false);
 }
 
@@ -91,7 +98,7 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorShortCircuit) {
     auto *op =
         storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(false),
                                     storage.Create<PrimitiveLiteral>(5));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueBool(), false);
   }
   {
@@ -102,7 +109,7 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorShortCircuit) {
     // raise due to `5`. This differs from neo4j, where they evaluate both
     // sides and return `false` without checking for type of the first
     // expression.
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
 }
 
@@ -112,20 +119,20 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorNull) {
     auto *op = storage.Create<AndOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>(5));
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
   {
     auto *op = storage.Create<AndOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>(true));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     auto *op = storage.Create<AndOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>(false));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     ASSERT_TRUE(value.IsBool());
     EXPECT_EQ(value.ValueBool(), false);
   }
@@ -134,21 +141,21 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorNull) {
 TEST_F(ExpressionEvaluatorTest, AdditionOperator) {
   auto *op = storage.Create<AdditionOperator>(
       storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), 5);
 }
 
 TEST_F(ExpressionEvaluatorTest, SubtractionOperator) {
   auto *op = storage.Create<SubtractionOperator>(
       storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), -1);
 }
 
 TEST_F(ExpressionEvaluatorTest, MultiplicationOperator) {
   auto *op = storage.Create<MultiplicationOperator>(
       storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), 6);
 }
 
@@ -156,14 +163,14 @@ TEST_F(ExpressionEvaluatorTest, DivisionOperator) {
   auto *op =
       storage.Create<DivisionOperator>(storage.Create<PrimitiveLiteral>(50),
                                        storage.Create<PrimitiveLiteral>(10));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), 5);
 }
 
 TEST_F(ExpressionEvaluatorTest, ModOperator) {
   auto *op = storage.Create<ModOperator>(storage.Create<PrimitiveLiteral>(65),
                                          storage.Create<PrimitiveLiteral>(10));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), 5);
 }
 
@@ -171,15 +178,15 @@ TEST_F(ExpressionEvaluatorTest, EqualOperator) {
   auto *op =
       storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(10),
                                     storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), false);
   op = storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(15),
                                      storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), true);
   op = storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(20),
                                      storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), false);
 }
 
@@ -187,30 +194,30 @@ TEST_F(ExpressionEvaluatorTest, NotEqualOperator) {
   auto *op =
       storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(10),
                                        storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(15),
                                         storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), false);
   op = storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(20),
                                         storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), true);
 }
 
 TEST_F(ExpressionEvaluatorTest, LessOperator) {
   auto *op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(10),
                                           storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(15),
                                     storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), false);
   op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(20),
                                     storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), false);
 }
 
@@ -218,15 +225,15 @@ TEST_F(ExpressionEvaluatorTest, GreaterOperator) {
   auto *op =
       storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(10),
                                       storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), false);
   op = storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(15),
                                        storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), false);
   op = storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(20),
                                        storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), true);
 }
 
@@ -234,15 +241,15 @@ TEST_F(ExpressionEvaluatorTest, LessEqualOperator) {
   auto *op =
       storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(10),
                                         storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), true);
   op = storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(15),
                                          storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), true);
   op = storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(20),
                                          storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), false);
 }
 
@@ -250,17 +257,17 @@ TEST_F(ExpressionEvaluatorTest, GreaterEqualOperator) {
   auto *op = storage.Create<GreaterEqualOperator>(
       storage.Create<PrimitiveLiteral>(10),
       storage.Create<PrimitiveLiteral>(15));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), false);
   op = storage.Create<GreaterEqualOperator>(
       storage.Create<PrimitiveLiteral>(15),
       storage.Create<PrimitiveLiteral>(15));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), true);
   op = storage.Create<GreaterEqualOperator>(
       storage.Create<PrimitiveLiteral>(20),
       storage.Create<PrimitiveLiteral>(15));
-  auto val3 = op->Accept(eval);
+  auto val3 = Eval(op);
   ASSERT_EQ(val3.ValueBool(), true);
 }
 
@@ -272,14 +279,14 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
     // Element exists in list.
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>(2), list_literal);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueBool(), true);
   }
   {
     // Element doesn't exist in list.
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>("x"), list_literal);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueBool(), false);
   }
   {
@@ -290,7 +297,7 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
     // Element doesn't exist in list with null element.
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>("x"), list_literal);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
@@ -298,14 +305,14 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>("x"),
         storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     // Null literal.
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null), list_literal);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
@@ -313,7 +320,7 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
     auto *op = storage.Create<InListOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<ListLiteral>(std::vector<Expression *>()));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_FALSE(value.ValueBool());
   }
 }
@@ -327,28 +334,28 @@ TEST_F(ExpressionEvaluatorTest, ListIndexing) {
     // Legal indexing.
     auto *op = storage.Create<SubscriptOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(2));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueInt(), 3);
   }
   {
     // Out of bounds indexing.
     auto *op = storage.Create<SubscriptOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(4));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     // Out of bounds indexing with negative bound.
     auto *op = storage.Create<SubscriptOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(-100));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     // Legal indexing with negative index.
     auto *op = storage.Create<SubscriptOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(-2));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueInt(), 3);
   }
   {
@@ -356,51 +363,48 @@ TEST_F(ExpressionEvaluatorTest, ListIndexing) {
     auto *op = storage.Create<SubscriptOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>(-2));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     // Indexing with incompatible type.
     auto *op = storage.Create<SubscriptOperator>(
         list_literal, storage.Create<PrimitiveLiteral>("bla"));
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
 }
 
 TEST_F(ExpressionEvaluatorTest, MapIndexing) {
-  auto *map_literal = storage.Create<MapLiteral>(
-      std::unordered_map<std::pair<std::string, storage::Property>,
-                         Expression *>{{std::make_pair("a", dba->Property("a")),
-                                        storage.Create<PrimitiveLiteral>(1)},
-                                       {std::make_pair("b", dba->Property("b")),
-                                        storage.Create<PrimitiveLiteral>(2)},
-                                       {std::make_pair("c", dba->Property("c")),
-                                        storage.Create<PrimitiveLiteral>(3)}});
+  auto *map_literal =
+      storage.Create<MapLiteral>(std::unordered_map<PropertyIx, Expression *>{
+          {storage.GetPropertyIx("a"), storage.Create<PrimitiveLiteral>(1)},
+          {storage.GetPropertyIx("b"), storage.Create<PrimitiveLiteral>(2)},
+          {storage.GetPropertyIx("c"), storage.Create<PrimitiveLiteral>(3)}});
   {
     // Legal indexing.
     auto *op = storage.Create<SubscriptOperator>(
         map_literal, storage.Create<PrimitiveLiteral>("b"));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueInt(), 2);
   }
   {
     // Legal indexing, non-existing key.
     auto *op = storage.Create<SubscriptOperator>(
         map_literal, storage.Create<PrimitiveLiteral>("z"));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
   {
     // Wrong key type.
     auto *op = storage.Create<SubscriptOperator>(
         map_literal, storage.Create<PrimitiveLiteral>(42));
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
   {
     // Indexing with Null.
     auto *op = storage.Create<SubscriptOperator>(
         map_literal, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
 }
@@ -419,46 +423,46 @@ TEST_F(ExpressionEvaluatorTest, VertexAndEdgeIndexing) {
     // Legal indexing.
     auto *op1 = storage.Create<SubscriptOperator>(
         vertex_id, storage.Create<PrimitiveLiteral>("prop"));
-    auto value1 = op1->Accept(eval);
+    auto value1 = Eval(op1);
     EXPECT_EQ(value1.ValueInt(), 42);
 
     auto *op2 = storage.Create<SubscriptOperator>(
         edge_id, storage.Create<PrimitiveLiteral>("prop"));
-    auto value2 = op2->Accept(eval);
+    auto value2 = Eval(op2);
     EXPECT_EQ(value2.ValueInt(), 43);
   }
   {
     // Legal indexing, non-existing key.
     auto *op1 = storage.Create<SubscriptOperator>(
         vertex_id, storage.Create<PrimitiveLiteral>("blah"));
-    auto value1 = op1->Accept(eval);
+    auto value1 = Eval(op1);
     EXPECT_TRUE(value1.IsNull());
 
     auto *op2 = storage.Create<SubscriptOperator>(
         edge_id, storage.Create<PrimitiveLiteral>("blah"));
-    auto value2 = op2->Accept(eval);
+    auto value2 = Eval(op2);
     EXPECT_TRUE(value2.IsNull());
   }
   {
     // Wrong key type.
     auto *op1 = storage.Create<SubscriptOperator>(
         vertex_id, storage.Create<PrimitiveLiteral>(1));
-    EXPECT_THROW(op1->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op1), QueryRuntimeException);
 
     auto *op2 = storage.Create<SubscriptOperator>(
         edge_id, storage.Create<PrimitiveLiteral>(1));
-    EXPECT_THROW(op2->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op2), QueryRuntimeException);
   }
   {
     // Indexing with Null.
     auto *op1 = storage.Create<SubscriptOperator>(
         vertex_id, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-    auto value1 = op1->Accept(eval);
+    auto value1 = Eval(op1);
     EXPECT_TRUE(value1.IsNull());
 
     auto *op2 = storage.Create<SubscriptOperator>(
         edge_id, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-    auto value2 = op2->Accept(eval);
+    auto value2 = Eval(op2);
     EXPECT_TRUE(value2.IsNull());
   }
 }
@@ -481,7 +485,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(2),
         storage.Create<PrimitiveLiteral>(4));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre(3, 4));
   }
   {
@@ -489,7 +493,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(2),
         storage.Create<PrimitiveLiteral>(-1));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre(3));
   }
   {
@@ -497,7 +501,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(2),
         storage.Create<PrimitiveLiteral>(-4));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre());
   }
   {
@@ -505,21 +509,21 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(-100),
         storage.Create<PrimitiveLiteral>(10));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre(1, 2, 3, 4));
   }
   {
     // Lower bound undefined.
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, nullptr, storage.Create<PrimitiveLiteral>(3));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre(1, 2, 3));
   }
   {
     // Upper bound undefined.
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(-2), nullptr);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_THAT(extract_ints(value), ElementsAre(3, 4));
   }
   {
@@ -527,21 +531,21 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>("mirko"));
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
   {
     // List of illegal type.
     auto *op = storage.Create<ListSlicingOperator>(
         storage.Create<PrimitiveLiteral>("a"),
         storage.Create<PrimitiveLiteral>(-2), nullptr);
-    EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
+    EXPECT_THROW(Eval(op), QueryRuntimeException);
   }
   {
     // Null value list with undefined upper bound.
     auto *op = storage.Create<ListSlicingOperator>(
         storage.Create<PrimitiveLiteral>(PropertyValue::Null),
         storage.Create<PrimitiveLiteral>(-2), nullptr);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
     ;
   }
@@ -550,7 +554,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
     auto *op = storage.Create<ListSlicingOperator>(
         list_literal, storage.Create<PrimitiveLiteral>(-2),
         storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
     ;
   }
@@ -565,7 +569,7 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
                                       storage.Create<PrimitiveLiteral>(2));
     auto *op = storage.Create<IfOperator>(condition_true, then_expression,
                                           else_expression);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     ASSERT_EQ(value.ValueInt(), 10);
   }
   {
@@ -574,7 +578,7 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
                                       storage.Create<PrimitiveLiteral>(3));
     auto *op = storage.Create<IfOperator>(condition_false, then_expression,
                                           else_expression);
-    auto value = op->Accept(eval);
+    auto value = Eval(op);
     ASSERT_EQ(value.ValueInt(), 20);
   }
   {
@@ -583,39 +587,39 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
                                          storage.Create<PrimitiveLiteral>(3));
     auto *op = storage.Create<IfOperator>(condition_exception, then_expression,
                                           else_expression);
-    ASSERT_THROW(op->Accept(eval), QueryRuntimeException);
+    ASSERT_THROW(Eval(op), QueryRuntimeException);
   }
 }
 
 TEST_F(ExpressionEvaluatorTest, NotOperator) {
   auto *op =
       storage.Create<NotOperator>(storage.Create<PrimitiveLiteral>(false));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueBool(), true);
 }
 
 TEST_F(ExpressionEvaluatorTest, UnaryPlusOperator) {
   auto *op =
       storage.Create<UnaryPlusOperator>(storage.Create<PrimitiveLiteral>(5));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), 5);
 }
 
 TEST_F(ExpressionEvaluatorTest, UnaryMinusOperator) {
   auto *op =
       storage.Create<UnaryMinusOperator>(storage.Create<PrimitiveLiteral>(5));
-  auto value = op->Accept(eval);
+  auto value = Eval(op);
   ASSERT_EQ(value.ValueInt(), -5);
 }
 
 TEST_F(ExpressionEvaluatorTest, IsNullOperator) {
   auto *op =
       storage.Create<IsNullOperator>(storage.Create<PrimitiveLiteral>(1));
-  auto val1 = op->Accept(eval);
+  auto val1 = Eval(op);
   ASSERT_EQ(val1.ValueBool(), false);
   op = storage.Create<IsNullOperator>(
       storage.Create<PrimitiveLiteral>(PropertyValue::Null));
-  auto val2 = op->Accept(eval);
+  auto val2 = Eval(op);
   ASSERT_EQ(val2.ValueBool(), true);
 }
 
@@ -630,26 +634,26 @@ TEST_F(ExpressionEvaluatorTest, LabelsTest) {
   frame[node_symbol] = v1;
   {
     auto *op = storage.Create<LabelsTest>(
-        identifier,
-        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("ANIMAL")});
-    auto value = op->Accept(eval);
+        identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
+                                         storage.GetLabelIx("ANIMAL")});
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueBool(), true);
   }
   {
     auto *op = storage.Create<LabelsTest>(
-        identifier,
-        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
-                                    dba->Label("ANIMAL")});
-    auto value = op->Accept(eval);
+        identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
+                                         storage.GetLabelIx("BAD_DOG"),
+                                         storage.GetLabelIx("ANIMAL")});
+    auto value = Eval(op);
     EXPECT_EQ(value.ValueBool(), false);
   }
   {
     frame[node_symbol] = TypedValue::Null;
     auto *op = storage.Create<LabelsTest>(
-        identifier,
-        std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
-                                    dba->Label("ANIMAL")});
-    auto value = op->Accept(eval);
+        identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
+                                         storage.GetLabelIx("BAD_DOG"),
+                                         storage.GetLabelIx("ANIMAL")});
+    auto value = Eval(op);
     EXPECT_TRUE(value.IsNull());
   }
 }
@@ -660,7 +664,7 @@ TEST_F(ExpressionEvaluatorTest, Aggregation) {
   auto aggr_sym = symbol_table.CreateSymbol("aggr", true);
   symbol_table[*aggr] = aggr_sym;
   frame[aggr_sym] = TypedValue(1);
-  auto value = aggr->Accept(eval);
+  auto value = Eval(aggr);
   EXPECT_EQ(value.ValueInt(), 1);
 }
 
@@ -669,7 +673,7 @@ TEST_F(ExpressionEvaluatorTest, ListLiteral) {
       std::vector<Expression *>{storage.Create<PrimitiveLiteral>(1),
                                 storage.Create<PrimitiveLiteral>("bla"),
                                 storage.Create<PrimitiveLiteral>(true)});
-  TypedValue result = list_literal->Accept(eval);
+  TypedValue result = Eval(list_literal);
   ASSERT_TRUE(result.IsList());
   auto &result_elems = result.ValueList();
   ASSERT_EQ(3, result_elems.size());
@@ -684,7 +688,7 @@ TEST_F(ExpressionEvaluatorTest, ListLiteral) {
 TEST_F(ExpressionEvaluatorTest, ParameterLookup) {
   ctx.parameters.Add(0, 42);
   auto *param_lookup = storage.Create<ParameterLookup>(0);
-  auto value = param_lookup->Accept(eval);
+  auto value = Eval(param_lookup);
   ASSERT_TRUE(value.IsInt());
   EXPECT_EQ(value.ValueInt(), 42);
 }
@@ -697,7 +701,7 @@ TEST_F(ExpressionEvaluatorTest, All) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*all->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = all->Accept(eval);
+  auto value = Eval(all);
   ASSERT_TRUE(value.IsBool());
   EXPECT_FALSE(value.ValueBool());
 }
@@ -707,7 +711,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionAllNullList) {
   auto *all = ALL("x", LITERAL(PropertyValue::Null), WHERE(LITERAL(true)));
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*all->identifier_] = x_sym;
-  auto value = all->Accept(eval);
+  auto value = Eval(all);
   EXPECT_TRUE(value.IsNull());
 }
 
@@ -716,7 +720,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionAllWhereWrongType) {
   auto *all = ALL("x", LIST(LITERAL(1)), WHERE(LITERAL(2)));
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*all->identifier_] = x_sym;
-  EXPECT_THROW(all->Accept(eval), QueryRuntimeException);
+  EXPECT_THROW(Eval(all), QueryRuntimeException);
 }
 
 TEST_F(ExpressionEvaluatorTest, FunctionSingle) {
@@ -727,7 +731,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingle) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*single->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = single->Accept(eval);
+  auto value = Eval(single);
   ASSERT_TRUE(value.IsBool());
   EXPECT_TRUE(value.ValueBool());
 }
@@ -740,7 +744,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingle2) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*single->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = single->Accept(eval);
+  auto value = Eval(single);
   ASSERT_TRUE(value.IsBool());
   EXPECT_FALSE(value.ValueBool());
 }
@@ -751,7 +755,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingleNullList) {
       SINGLE("x", LITERAL(PropertyValue::Null), WHERE(LITERAL(true)));
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*single->identifier_] = x_sym;
-  auto value = single->Accept(eval);
+  auto value = Eval(single);
   EXPECT_TRUE(value.IsNull());
 }
 
@@ -767,7 +771,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionReduce) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*reduce->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = reduce->Accept(eval);
+  auto value = Eval(reduce);
   ASSERT_TRUE(value.IsInt());
   EXPECT_EQ(value.ValueInt(), 3);
 }
@@ -781,7 +785,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtract) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*extract->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = extract->Accept(eval);
+  auto value = Eval(extract);
   EXPECT_TRUE(value.IsList());
   ;
   auto result = value.ValueList();
@@ -798,7 +802,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtractNull) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*extract->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  auto value = extract->Accept(eval);
+  auto value = Eval(extract);
   EXPECT_TRUE(value.IsNull());
 }
 
@@ -809,40 +813,37 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtractExceptions) {
   const auto x_sym = symbol_table.CreateSymbol("x", true);
   symbol_table[*extract->identifier_] = x_sym;
   symbol_table[*ident_x] = x_sym;
-  EXPECT_THROW(extract->Accept(eval), QueryRuntimeException);
+  EXPECT_THROW(Eval(extract), QueryRuntimeException);
 }
 
 TEST_F(ExpressionEvaluatorTest, Coalesce) {
   // coalesce()
-  EXPECT_THROW(COALESCE()->Accept(eval), QueryRuntimeException);
+  EXPECT_THROW(Eval(COALESCE()), QueryRuntimeException);
 
   // coalesce(null, null)
-  EXPECT_TRUE(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null))
-                  ->Accept(eval)
-                  .IsNull());
+  EXPECT_TRUE(
+      Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null)))
+          .IsNull());
 
   // coalesce(null, 2, 3)
-  EXPECT_EQ(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3))
-                ->Accept(eval)
+  EXPECT_EQ(Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3)))
                 .ValueInt(),
             2);
 
   // coalesce(null, 2, assert(false), 3)
-  EXPECT_EQ(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
-                     FN("ASSERT", LITERAL(false)), LITERAL(3))
-                ->Accept(eval)
+  EXPECT_EQ(Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
+                          FN("ASSERT", LITERAL(false)), LITERAL(3)))
                 .ValueInt(),
             2);
 
   // (null, assert(false))
-  EXPECT_THROW(COALESCE(LITERAL(TypedValue::Null), FN("ASSERT", LITERAL(false)))
-                   ->Accept(eval),
-               QueryRuntimeException);
+  EXPECT_THROW(
+      Eval(COALESCE(LITERAL(TypedValue::Null), FN("ASSERT", LITERAL(false)))),
+      QueryRuntimeException);
 
   // coalesce([null, null])
-  EXPECT_FALSE(COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
-                            TypedValue::Null, TypedValue::Null})))
-                   ->Accept(eval)
+  EXPECT_FALSE(Eval(COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
+                        TypedValue::Null, TypedValue::Null}))))
                    .IsNull());
 }
 
@@ -858,8 +859,9 @@ class ExpressionEvaluatorPropertyLookup : public ExpressionEvaluatorTest {
   void SetUp() { symbol_table[*identifier] = symbol; }
 
   auto Value(std::pair<std::string, storage::Property> property) {
-    auto *op = storage.Create<PropertyLookup>(identifier, property);
-    return op->Accept(eval);
+    auto *op = storage.Create<PropertyLookup>(
+        identifier, storage.GetPropertyIx(property.first));
+    return Eval(op);
   }
 };
 
@@ -915,7 +917,7 @@ class FunctionTest : public ExpressionEvaluatorTest {
       const std::string &function_name,
       const std::vector<Expression *> &expressions) {
     auto *op = storage.Create<Function>(function_name, expressions);
-    return op->Accept(eval);
+    return Eval(op);
   }
 
   TypedValue EvaluateFunction(const std::string &function_name,
diff --git a/tests/unit/query_plan.cpp b/tests/unit/query_plan.cpp
index a033c05c2..b92de69e2 100644
--- a/tests/unit/query_plan.cpp
+++ b/tests/unit/query_plan.cpp
@@ -43,9 +43,9 @@ class Planner {
  public:
   template <class TDbAccessor>
   Planner(std::vector<SingleQueryPart> single_query_parts,
-          PlanningContext<TDbAccessor> &context) {
+          PlanningContext<TDbAccessor> context) {
     plan_ = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(single_query_parts,
-                                                            context);
+                                                            &context);
   }
 
   auto &plan() { return *plan_; }
@@ -68,7 +68,7 @@ auto CheckPlan(query::CypherQuery *query, AstStorage &storage,
                TChecker... checker) {
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TPlanner>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TPlanner>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, checker...);
 }
 
@@ -86,7 +86,7 @@ TYPED_TEST(TestPlanner, MatchNodeReturn) {
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce());
 }
 
@@ -99,7 +99,7 @@ TYPED_TEST(TestPlanner, CreateNodeReturn) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc,
             ExpectProduce());
 }
@@ -108,7 +108,7 @@ TYPED_TEST(TestPlanner, CreateExpand) {
   // Test CREATE (n) -[r :rel1]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
       NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
   CheckPlan<TypeParam>(query, storage, ExpectCreateNode(),
@@ -127,7 +127,7 @@ TYPED_TEST(TestPlanner, CreateNodeExpandNode) {
   // Test CREATE (n) -[r :rel]-> (m), (l)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("rel");
+  auto relationship = "rel";
   auto *query = QUERY(SINGLE_QUERY(CREATE(
       PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")),
       PATTERN(NODE("l")))));
@@ -139,7 +139,7 @@ TYPED_TEST(TestPlanner, CreateNamedPattern) {
   // Test CREATE p = (n) -[r :rel]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("rel");
+  auto relationship = "rel";
   auto *query = QUERY(SINGLE_QUERY(CREATE(NAMED_PATTERN(
       "p", NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
   CheckPlan<TypeParam>(query, storage, ExpectCreateNode(), ExpectCreateExpand(),
@@ -150,7 +150,7 @@ TYPED_TEST(TestPlanner, MatchCreateExpand) {
   // Test MATCH (n) CREATE (n) -[r :rel1]-> (m)
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))),
       CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}),
@@ -162,12 +162,12 @@ TYPED_TEST(TestPlanner, MatchLabeledNodes) {
   // Test MATCH (n :label) RETURN n
   AstStorage storage;
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *as_n = NEXPR("n", IDENT("n"));
   auto *query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAllByLabel(),
             ExpectProduce());
 }
@@ -176,14 +176,14 @@ TYPED_TEST(TestPlanner, MatchPathReturn) {
   // Test MATCH (n) -[r :relationship]- (m) RETURN n
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_n = NEXPR("n", IDENT("n"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {relationship}),
                     NODE("m"))),
       RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
             ExpectProduce());
 }
@@ -192,7 +192,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
   // Test MATCH p = (n) -[r :relationship]- (m) RETURN p
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_p = NEXPR("p", IDENT("p"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(NAMED_PATTERN("p", NODE("n"),
@@ -200,7 +200,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
                           NODE("m"))),
       RETURN(as_p)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
             ExpectConstructNamedPath(), ExpectProduce());
 }
@@ -209,7 +209,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
   // Test MATCH p = (n) -[r :relationship]- (m) WHERE 2 = p RETURN p
   AstStorage storage;
   FakeDbAccessor dba;
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto *as_p = NEXPR("p", IDENT("p"));
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(NAMED_PATTERN("p", NODE("n"),
@@ -217,7 +217,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
                           NODE("m"))),
       WHERE(EQ(LITERAL(2), IDENT("p"))), RETURN(as_p)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
             ExpectConstructNamedPath(), ExpectFilter(), ExpectProduce());
 }
@@ -238,7 +238,7 @@ TYPED_TEST(TestPlanner, OptionalMatchNamedPatternReturn) {
   std::vector<Symbol> optional_symbols{get_symbol(pattern), get_symbol(node_n),
                                        get_symbol(edge), get_symbol(node_m)};
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   std::list<BaseOpChecker *> optional{new ExpectScanAll(), new ExpectExpand(),
                                       new ExpectConstructNamedPath()};
   CheckPlan(planner.plan(), symbol_table,
@@ -255,7 +255,7 @@ TYPED_TEST(TestPlanner, MatchWhereReturn) {
       MATCH(PATTERN(NODE("n"))),
       WHERE(LESS(PROPERTY_LOOKUP("n", property), LITERAL(42))), RETURN(as_n)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectFilter(),
             ExpectProduce());
 }
@@ -273,7 +273,7 @@ TYPED_TEST(TestPlanner, MatchNodeSet) {
   AstStorage storage;
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
                                    SET(PROPERTY_LOOKUP("n", prop), LITERAL(42)),
                                    SET("n", IDENT("n")), SET("n", {label})));
@@ -286,7 +286,7 @@ TYPED_TEST(TestPlanner, MatchRemove) {
   AstStorage storage;
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto label = dba.Label("label");
+  auto label = "label";
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
                                    REMOVE(PROPERTY_LOOKUP("n", prop)),
                                    REMOVE("n", {label})));
@@ -351,7 +351,7 @@ TYPED_TEST(TestPlanner, MultiMatch) {
       MATCH(PATTERN(node_j, edge_e, node_i, edge_f, node_h)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // Multiple MATCH clauses form a Cartesian product, so the uniqueness should
   // not cross MATCH boundaries.
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
@@ -370,7 +370,7 @@ TYPED_TEST(TestPlanner, MultiMatchSameStart) {
   // MATCH clause.
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
             ExpectProduce());
 }
@@ -384,7 +384,7 @@ TYPED_TEST(TestPlanner, MatchWithReturn) {
   // No accumulation since we only do reads.
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
             ExpectProduce());
 }
@@ -400,7 +400,7 @@ TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
       WHERE(LESS(PROPERTY_LOOKUP("new", prop), LITERAL(42))), RETURN(as_new)));
   // No accumulation since we only do reads.
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
             ExpectFilter(), ExpectProduce());
 }
@@ -408,8 +408,8 @@ TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
 TYPED_TEST(TestPlanner, CreateMultiExpand) {
   // Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
   FakeDbAccessor dba;
-  auto r = dba.EdgeType("r");
-  auto p = dba.EdgeType("p");
+  auto r = "r";
+  auto p = "p";
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
       CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {r}), NODE("m")),
@@ -446,7 +446,7 @@ TYPED_TEST(TestPlanner, MatchReturnSum) {
       MATCH(PATTERN(NODE("n"))), RETURN(sum, AS("sum"), n_prop2, AS("group"))));
   auto aggr = ExpectAggregate({sum}, {n_prop2});
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), aggr,
             ExpectProduce());
 }
@@ -463,7 +463,7 @@ TYPED_TEST(TestPlanner, CreateWithSum) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
   auto aggr = ExpectAggregate({sum}, {});
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // We expect both the accumulation and aggregation because the part before
   // WITH updates the database.
   CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
@@ -473,7 +473,7 @@ TYPED_TEST(TestPlanner, CreateWithSum) {
 TYPED_TEST(TestPlanner, MatchWithCreate) {
   // Test MATCH (n) WITH n AS a CREATE (a) -[r :r]-> (b)
   FakeDbAccessor dba;
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))), WITH("n", AS("a")),
@@ -492,7 +492,7 @@ TYPED_TEST(TestPlanner, MatchReturnSkipLimit) {
                          RETURN(as_n, SKIP(LITERAL(2)), LIMIT(LITERAL(1)))));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
             ExpectSkip(), ExpectLimit());
 }
@@ -507,7 +507,7 @@ TYPED_TEST(TestPlanner, CreateWithSkipReturnLimit) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // Since we have a write query, we need to have Accumulate. This is a bit
   // different than Neo4j 3.0, which optimizes WITH followed by RETURN as a
   // single RETURN clause and then moves Skip and Limit before Accumulate.
@@ -530,7 +530,7 @@ TYPED_TEST(TestPlanner, CreateReturnSumSkipLimit) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
   auto aggr = ExpectAggregate({sum}, {});
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
             ExpectProduce(), ExpectSkip(), ExpectLimit());
 }
@@ -545,7 +545,7 @@ TYPED_TEST(TestPlanner, MatchReturnOrderBy) {
   auto ret = RETURN(as_m, ORDER_BY(PROPERTY_LOOKUP("n", prop)));
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), ret));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
             ExpectOrderBy());
 }
@@ -555,7 +555,7 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
   //      WITH n AS new ORDER BY new.prop, r.prop WHERE m.prop < 42
   FakeDbAccessor dba;
   auto prop = dba.Property("prop");
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   AstStorage storage;
   auto ident_n = IDENT("n");
   auto new_prop = PROPERTY_LOOKUP("new", prop);
@@ -573,7 +573,7 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
       symbol_table.at(*r_prop->expression_),  // `r` in ORDER BY
       symbol_table.at(*m_prop->expression_),  // `m` in WHERE
   });
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(),
             ExpectCreateExpand(), acc, ExpectProduce(), ExpectOrderBy(),
             ExpectFilter());
@@ -595,7 +595,7 @@ TYPED_TEST(TestPlanner, MatchMerge) {
   //      ON MATCH SET n.prop = 42 ON CREATE SET m = n
   //      RETURN n AS n
   FakeDbAccessor dba;
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   auto prop = dba.Property("prop");
   AstStorage storage;
   auto ident_n = IDENT("n");
@@ -612,7 +612,7 @@ TYPED_TEST(TestPlanner, MatchMerge) {
   auto symbol_table = query::MakeSymbolTable(query);
   // We expect Accumulate after Merge, because it is considered as a write.
   auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(),
             ExpectMerge(on_match, on_create), acc, ExpectProduce());
   for (auto &op : on_match) delete op;
@@ -647,7 +647,7 @@ TYPED_TEST(TestPlanner, MatchUnwindReturn) {
                    RETURN(as_n, as_x)));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectUnwind(),
             ExpectProduce());
 }
@@ -675,7 +675,7 @@ TYPED_TEST(TestPlanner, CreateWithDistinctSumWhereReturn) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto acc = ExpectAccumulate({symbol_table.at(*node_n->identifier_)});
   auto aggr = ExpectAggregate({sum}, {});
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
             ExpectProduce(), ExpectDistinct(), ExpectFilter(), ExpectProduce());
 }
@@ -687,10 +687,10 @@ TYPED_TEST(TestPlanner, MatchCrossReferenceVariable) {
   AstStorage storage;
   auto node_n = NODE("n");
   auto m_prop = PROPERTY_LOOKUP("m", prop.second);
-  node_n->properties_[prop] = m_prop;
+  node_n->properties_[storage.GetPropertyIx(prop.first)] = m_prop;
   auto node_m = NODE("m");
   auto n_prop = PROPERTY_LOOKUP("n", prop.second);
-  node_m->properties_[prop] = n_prop;
+  node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
   auto *query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n), PATTERN(node_m)), RETURN("n")));
   // We expect both ScanAll to come before filters (2 are joined into one),
@@ -710,7 +710,7 @@ TYPED_TEST(TestPlanner, MatchWhereBeforeExpand) {
       WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))), RETURN(as_n)));
   // We expect Filter to come immediately after ScanAll, since it only uses `n`.
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectFilter(),
             ExpectExpand(), ExpectProduce());
 }
@@ -757,7 +757,7 @@ TYPED_TEST(TestPlanner, MatchReturnAsterisk) {
   auto query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))), ret));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
             ExpectProduce());
   std::vector<std::string> output_names;
@@ -778,7 +778,7 @@ TYPED_TEST(TestPlanner, MatchReturnAsteriskSum) {
   ret->body_.all_identifiers = true;
   auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), ret));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto *produce = dynamic_cast<Produce *>(&planner.plan());
   ASSERT_TRUE(produce);
   const auto &named_expressions = produce->named_expressions_;
@@ -802,7 +802,7 @@ TYPED_TEST(TestPlanner, UnwindMergeNodeProperty) {
   AstStorage storage;
   FakeDbAccessor dba;
   auto node_n = NODE("n");
-  node_n->properties_[PROPERTY_PAIR("prop")] = IDENT("i");
+  node_n->properties_[storage.GetPropertyIx("prop")] = IDENT("i");
   auto *query = QUERY(
       SINGLE_QUERY(UNWIND(LIST(LITERAL(1)), AS("i")), MERGE(PATTERN(node_n))));
   std::list<BaseOpChecker *> on_match{new ExpectScanAll(), new ExpectFilter()};
@@ -859,9 +859,9 @@ TYPED_TEST(TestPlanner, MapLiteralAggregationReturn) {
   FakeDbAccessor dba;
   auto sum = SUM(LITERAL(2));
   auto group_by_literal = LITERAL(42);
-  auto *query =
-      QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum}), AS("result"),
-                                group_by_literal, AS("group_by"))));
+  auto *query = QUERY(
+      SINGLE_QUERY(RETURN(MAP({storage.GetPropertyIx("sum"), sum}),
+                          AS("result"), group_by_literal, AS("group_by"))));
   auto aggr = ExpectAggregate({sum}, {group_by_literal});
   CheckPlan<TypeParam>(query, storage, aggr, ExpectProduce());
 }
@@ -926,10 +926,10 @@ TYPED_TEST(TestPlanner, MapWithAggregationAndGroupBy) {
   FakeDbAccessor dba;
   auto sum = SUM(LITERAL(2));
   auto group_by_literal = LITERAL(42);
-  auto *query =
-      QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum},
-                                    {PROPERTY_PAIR("lit"), group_by_literal}),
-                                AS("result"))));
+  auto *query = QUERY(
+      SINGLE_QUERY(RETURN(MAP({storage.GetPropertyIx("sum"), sum},
+                              {storage.GetPropertyIx("lit"), group_by_literal}),
+                          AS("result"))));
   auto aggr = ExpectAggregate({sum}, {group_by_literal});
   CheckPlan<TypeParam>(query, storage, aggr, ExpectProduce());
 }
@@ -943,13 +943,13 @@ TYPED_TEST(TestPlanner, AtomIndexedLabelProperty) {
   auto not_indexed = PROPERTY_PAIR("not_indexed");
   dba.SetIndexCount(label, 1);
   dba.SetIndexCount(label, property.second, 1);
-  auto node = NODE("n", label);
+  auto node = NODE("n", "label");
   auto lit_42 = LITERAL(42);
-  node->properties_[property] = lit_42;
-  node->properties_[not_indexed] = LITERAL(0);
+  node->properties_[storage.GetPropertyIx(property.first)] = lit_42;
+  node->properties_[storage.GetPropertyIx(not_indexed.first)] = LITERAL(0);
   auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table,
             ExpectScanAllByLabelPropertyValue(label, property, lit_42),
             ExpectFilter(), ExpectProduce());
@@ -965,15 +965,16 @@ TYPED_TEST(TestPlanner, AtomPropertyWhereLabelIndexing) {
   dba.SetIndexCount(label, property.second, 0);
   auto node = NODE("n");
   auto lit_42 = LITERAL(42);
-  node->properties_[property] = lit_42;
+  node->properties_[storage.GetPropertyIx(property.first)] = lit_42;
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(node)),
       WHERE(AND(PROPERTY_LOOKUP("n", not_indexed),
                 storage.Create<query::LabelsTest>(
-                    IDENT("n"), std::vector<storage::Label>{label}))),
+                    IDENT("n"),
+                    std::vector<query::LabelIx>{storage.GetLabelIx("label")}))),
       RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table,
             ExpectScanAllByLabelPropertyValue(label, property, lit_42),
             ExpectFilter(), ExpectProduce());
@@ -988,10 +989,10 @@ TYPED_TEST(TestPlanner, WhereIndexedLabelProperty) {
   dba.SetIndexCount(label, property.second, 0);
   auto lit_42 = LITERAL(42);
   auto *query = QUERY(SINGLE_QUERY(
-      MATCH(PATTERN(NODE("n", label))),
+      MATCH(PATTERN(NODE("n", "label"))),
       WHERE(EQ(PROPERTY_LOOKUP("n", property), lit_42)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table,
             ExpectScanAllByLabelPropertyValue(label, property, lit_42),
             ExpectProduce());
@@ -1010,12 +1011,12 @@ TYPED_TEST(TestPlanner, BestPropertyIndexed) {
   dba.SetIndexCount(label, better.second, 0);
   auto lit_42 = LITERAL(42);
   auto *query = QUERY(
-      SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
+      SINGLE_QUERY(MATCH(PATTERN(NODE("n", "label"))),
                    WHERE(AND(EQ(PROPERTY_LOOKUP("n", property), LITERAL(1)),
                              EQ(PROPERTY_LOOKUP("n", better), lit_42))),
                    RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table,
             ExpectScanAllByLabelPropertyValue(label, better, lit_42),
             ExpectFilter(), ExpectProduce());
@@ -1035,12 +1036,12 @@ TYPED_TEST(TestPlanner, MultiPropertyIndexScan) {
   auto lit_1 = LITERAL(1);
   auto lit_2 = LITERAL(2);
   auto *query = QUERY(SINGLE_QUERY(
-      MATCH(PATTERN(NODE("n", label1)), PATTERN(NODE("m", label2))),
+      MATCH(PATTERN(NODE("n", "label1")), PATTERN(NODE("m", "label2"))),
       WHERE(AND(EQ(PROPERTY_LOOKUP("n", prop1), lit_1),
                 EQ(PROPERTY_LOOKUP("m", prop2), lit_2))),
       RETURN("n", "m")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table,
             ExpectScanAllByLabelPropertyValue(label1, prop1, lit_1),
             ExpectScanAllByLabelPropertyValue(label2, prop2, lit_2),
@@ -1062,10 +1063,12 @@ TYPED_TEST(TestPlanner, WhereIndexedLabelPropertyRange) {
                                                        auto upper_bound) {
     // Shadow the first storage, so that the query is created in this one.
     AstStorage storage;
-    auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
+    storage.GetLabelIx("label");
+    storage.GetPropertyIx("property");
+    auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", "label"))),
                                      WHERE(rel_expr), RETURN("n")));
     auto symbol_table = query::MakeSymbolTable(query);
-    auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+    auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
     CheckPlan(planner.plan(), symbol_table,
               ExpectScanAllByLabelPropertyRange(label, property, lower_bound,
                                                 upper_bound),
@@ -1105,11 +1108,11 @@ TYPED_TEST(TestPlanner, UnableToUsePropertyIndex) {
   dba.SetIndexCount(label, property, 0);
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
-      MATCH(PATTERN(NODE("n", label))),
+      MATCH(PATTERN(NODE("n", "label"))),
       WHERE(EQ(PROPERTY_LOOKUP("n", property), PROPERTY_LOOKUP("n", property))),
       RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // We can only get ScanAllByLabelIndex, because we are comparing properties
   // with those on the same node.
   CheckPlan(planner.plan(), symbol_table, ExpectScanAllByLabel(),
@@ -1125,11 +1128,11 @@ TYPED_TEST(TestPlanner, SecondPropertyIndex) {
   AstStorage storage;
   auto n_prop = PROPERTY_LOOKUP("n", property);
   auto m_prop = PROPERTY_LOOKUP("m", property);
-  auto *query = QUERY(
-      SINGLE_QUERY(MATCH(PATTERN(NODE("n", label)), PATTERN(NODE("m", label))),
-                   WHERE(EQ(m_prop, n_prop)), RETURN("n")));
+  auto *query = QUERY(SINGLE_QUERY(
+      MATCH(PATTERN(NODE("n", "label")), PATTERN(NODE("m", "label"))),
+      WHERE(EQ(m_prop, n_prop)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(
       planner.plan(), symbol_table, ExpectScanAllByLabel(),
       // Note: We are scanning for m, therefore property should equal n_prop.
@@ -1171,11 +1174,11 @@ TYPED_TEST(TestPlanner, MatchExpandVariableNoBounds) {
 TYPED_TEST(TestPlanner, MatchExpandVariableInlinedFilter) {
   // Test MATCH (n) -[r :type * {prop: 42}]-> (m) RETURN r
   FakeDbAccessor dba;
-  auto type = dba.EdgeType("type");
+  auto type = "type";
   auto prop = PROPERTY_PAIR("prop");
   AstStorage storage;
   auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH, {type});
-  edge->properties_[prop] = LITERAL(42);
+  edge->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
   CheckPlan<TypeParam>(
@@ -1187,11 +1190,12 @@ TYPED_TEST(TestPlanner, MatchExpandVariableInlinedFilter) {
 TYPED_TEST(TestPlanner, MatchExpandVariableNotInlinedFilter) {
   // Test MATCH (n) -[r :type * {prop: m.prop}]-> (m) RETURN r
   FakeDbAccessor dba;
-  auto type = dba.EdgeType("type");
+  auto type = "type";
   auto prop = PROPERTY_PAIR("prop");
   AstStorage storage;
   auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH, {type});
-  edge->properties_[prop] = EQ(PROPERTY_LOOKUP("m", prop), LITERAL(42));
+  edge->properties_[storage.GetPropertyIx(prop.first)] =
+      EQ(PROPERTY_LOOKUP("m", prop), LITERAL(42));
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
   CheckPlan<TypeParam>(query, storage, ExpectScanAll(), ExpectExpandVariable(),
@@ -1210,7 +1214,7 @@ TYPED_TEST(TestPlanner, MatchExpandVariableTotalWeightSymbol) {
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("*")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto *root = dynamic_cast<Produce *>(&planner.plan());
 
   ASSERT_TRUE(root);
@@ -1245,11 +1249,11 @@ TYPED_TEST(TestPlanner, UnwindMatchVariable) {
 TYPED_TEST(TestPlanner, MatchBfs) {
   // Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
   FakeDbAccessor dba;
-  auto edge_type = dba.EdgeType("type");
   AstStorage storage;
+  auto edge_type = storage.GetEdgeTypeIx("type");
   auto *bfs = storage.Create<query::EdgeAtom>(
       IDENT("r"), query::EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
-      std::vector<storage::EdgeType>{edge_type});
+      std::vector<query::EdgeTypeIx>{edge_type});
   bfs->filter_lambda_.inner_edge = IDENT("r");
   bfs->filter_lambda_.inner_node = IDENT("n");
   bfs->filter_lambda_.expression = IDENT("n");
@@ -1258,7 +1262,7 @@ TYPED_TEST(TestPlanner, MatchBfs) {
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), bfs, NODE("m"))), RETURN(as_r)));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpandBfs(),
             ExpectProduce());
 }
@@ -1266,12 +1270,12 @@ TYPED_TEST(TestPlanner, MatchBfs) {
 TYPED_TEST(TestPlanner, MatchDoubleScanToExpandExisting) {
   // Test MATCH (n) -[r]- (m :label) RETURN r
   FakeDbAccessor dba;
-  auto label = dba.Label("label");
+  auto label = "label";
   AstStorage storage;
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m", label))), RETURN("r")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // We expect 2x ScanAll and then Expand, since we are guessing that is
   // faster (due to low label index vertex count).
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(),
@@ -1288,12 +1292,12 @@ TYPED_TEST(TestPlanner, MatchScanToExpand) {
                     FLAGS_query_vertex_count_to_expand_existing + 1);
   dba.SetIndexCount(label, FLAGS_query_vertex_count_to_expand_existing + 1);
   AstStorage storage;
-  auto node_m = NODE("m", label);
-  node_m->properties_[std::make_pair("property", property)] = LITERAL(1);
+  auto node_m = NODE("m", "label");
+  node_m->properties_[storage.GetPropertyIx("property")] = LITERAL(1);
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), node_m)), RETURN("r")));
   auto symbol_table = query::MakeSymbolTable(query);
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   // We expect 1x ScanAll and then Expand, since we are guessing that
   // is faster (due to high label index vertex count).
   CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
@@ -1327,7 +1331,7 @@ TYPED_TEST(TestPlanner, ReturnAsteriskOmitsLambdaSymbols) {
       QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), ret));
   auto symbol_table = query::MakeSymbolTable(query);
   FakeDbAccessor dba;
-  auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
+  auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
   auto *produce = dynamic_cast<Produce *>(&planner.plan());
   ASSERT_TRUE(produce);
   std::vector<std::string> outputs;
diff --git a/tests/unit/query_plan_accumulate_aggregate.cpp b/tests/unit/query_plan_accumulate_aggregate.cpp
index 7f659a47a..43f63eb72 100644
--- a/tests/unit/query_plan_accumulate_aggregate.cpp
+++ b/tests/unit/query_plan_accumulate_aggregate.cpp
@@ -51,11 +51,11 @@ TEST(QueryPlan, Accumulate) {
     auto n_p = PROPERTY_LOOKUP("n", prop);
     symbol_table[*n_p->expression_] = n.sym_;
     auto set_n_p =
-        std::make_shared<plan::SetProperty>(r_m.op_, n_p, ADD(n_p, one));
+        std::make_shared<plan::SetProperty>(r_m.op_, prop, n_p, ADD(n_p, one));
     auto m_p = PROPERTY_LOOKUP("m", prop);
     symbol_table[*m_p->expression_] = r_m.node_sym_;
     auto set_m_p =
-        std::make_shared<plan::SetProperty>(set_n_p, m_p, ADD(m_p, one));
+        std::make_shared<plan::SetProperty>(set_n_p, prop, m_p, ADD(m_p, one));
 
     std::shared_ptr<LogicalOperator> last_op = set_m_p;
     if (accumulate) {
@@ -68,7 +68,8 @@ TEST(QueryPlan, Accumulate) {
     auto m_p_ne = NEXPR("m.p", m_p);
     symbol_table[*m_p_ne] = symbol_table.CreateSymbol("m_p_ne", true);
     auto produce = MakeProduce(last_op, n_p_ne, m_p_ne);
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    auto results = CollectProduce(*produce, &context);
     std::vector<int> results_data;
     for (const auto &row : results)
       for (const auto &column : row)
@@ -97,7 +98,8 @@ TEST(QueryPlan, AccumulateAdvance) {
     auto accumulate = std::make_shared<Accumulate>(
         create, std::vector<Symbol>{node.symbol}, advance);
     auto match = MakeScanAll(storage, symbol_table, "m", accumulate);
-    EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, *dba, symbol_table));
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_EQ(advance ? 1 : 0, PullAll(*match.op_, &context));
   };
   check(false);
   check(true);
@@ -187,7 +189,8 @@ class QueryPlanAggregateOps : public ::testing::Test {
     auto produce =
         MakeAggregationProduce(n.op_, symbol_table, storage,
                                aggregation_expressions, ops, group_bys, {});
-    return CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    return CollectProduce(*produce, &context);
   }
 };
 
@@ -330,7 +333,8 @@ TEST(QueryPlan, AggregateGroupByValues) {
       MakeAggregationProduce(n.op_, symbol_table, storage, {n_p},
                              {Aggregation::Op::COUNT}, {n_p}, {n.sym_});
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(results.size(), group_by_vals.size() - 2);
   TypedValue::unordered_set result_group_bys;
   for (const auto &row : results) {
@@ -378,7 +382,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
                                         {Aggregation::Op::COUNT},
                                         {n_p1, n_p2, n_p3}, {n.sym_});
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 2 * 3 * 5);
 }
 
@@ -394,7 +399,8 @@ TEST(QueryPlan, AggregateNoInput) {
 
   auto produce = MakeAggregationProduce(nullptr, symbol_table, storage, {two},
                                         {Aggregation::Op::COUNT}, {}, {});
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(1, results.size());
   EXPECT_EQ(1, results[0].size());
   EXPECT_EQ(TypedValue::Type::Int, results[0][0].type());
@@ -427,7 +433,8 @@ TEST(QueryPlan, AggregateCountEdgeCases) {
   auto count = [&]() {
     auto produce = MakeAggregationProduce(n.op_, symbol_table, storage, {n_p},
                                           {Aggregation::Op::COUNT}, {}, {});
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    auto results = CollectProduce(*produce, &context);
     if (results.size() == 0) return -1L;
     EXPECT_EQ(1, results.size());
     EXPECT_EQ(1, results[0].size());
@@ -487,7 +494,8 @@ TEST(QueryPlan, AggregateFirstValueTypes) {
   auto aggregate = [&](Expression *expression, Aggregation::Op aggr_op) {
     auto produce = MakeAggregationProduce(n.op_, symbol_table, storage,
                                           {expression}, {aggr_op}, {}, {});
-    CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    CollectProduce(*produce, &context);
   };
 
   // everything except for COUNT and COLLECT fails on a Vertex
@@ -545,7 +553,8 @@ TEST(QueryPlan, AggregateTypes) {
   auto aggregate = [&](Expression *expression, Aggregation::Op aggr_op) {
     auto produce = MakeAggregationProduce(n.op_, symbol_table, storage,
                                           {expression}, {aggr_op}, {}, {});
-    CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    CollectProduce(*produce, &context);
   };
 
   // everything except for COUNT and COLLECT fails on a Vertex
@@ -602,7 +611,8 @@ TEST(QueryPlan, Unwind) {
   symbol_table[*y_ne] = symbol_table.CreateSymbol("y_ne", true);
   auto produce = MakeProduce(unwind_1, x_ne, y_ne);
 
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(4, results.size());
   const std::vector<int> expected_x_card{3, 3, 3, 1};
   auto expected_x_card_it = expected_x_card.begin();
diff --git a/tests/unit/query_plan_bag_semantics.cpp b/tests/unit/query_plan_bag_semantics.cpp
index eac724e4c..b763e557e 100644
--- a/tests/unit/query_plan_bag_semantics.cpp
+++ b/tests/unit/query_plan_bag_semantics.cpp
@@ -31,23 +31,24 @@ TEST(QueryPlan, Skip) {
   auto n = MakeScanAll(storage, symbol_table, "n1");
   auto skip = std::make_shared<plan::Skip>(n.op_, LITERAL(2));
 
-  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(0, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(0, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(0, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(1, PullAll(*skip, &context));
 
   for (int i = 0; i < 10; ++i) dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(11, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(11, PullAll(*skip, &context));
 }
 
 TEST(QueryPlan, Limit) {
@@ -60,23 +61,24 @@ TEST(QueryPlan, Limit) {
   auto n = MakeScanAll(storage, symbol_table, "n1");
   auto skip = std::make_shared<plan::Limit>(n.op_, LITERAL(2));
 
-  EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(0, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(1, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(2, PullAll(*skip, &context));
 
   dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(2, PullAll(*skip, &context));
 
   for (int i = 0; i < 10; ++i) dba->InsertVertex();
   dba->AdvanceCommand();
-  EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
+  EXPECT_EQ(2, PullAll(*skip, &context));
 }
 
 TEST(QueryPlan, CreateLimit) {
@@ -98,7 +100,8 @@ TEST(QueryPlan, CreateLimit) {
   auto c = std::make_shared<CreateNode>(n.op_, m);
   auto skip = std::make_shared<plan::Limit>(c, LITERAL(1));
 
-  EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*skip, &context));
   dba->AdvanceCommand();
   EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
 }
@@ -156,7 +159,8 @@ TEST(QueryPlan, OrderBy) {
     auto n_p_ne = NEXPR("n.p", n_p);
     symbol_table[*n_p_ne] = symbol_table.CreateSymbol("n.p", true);
     auto produce = MakeProduce(order_by, n_p_ne);
-    auto results = CollectProduce(produce.get(), symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    auto results = CollectProduce(*produce, &context);
     ASSERT_EQ(values.size(), results.size());
     for (int j = 0; j < results.size(); ++j)
       EXPECT_TRUE(TypedValue::BoolEqual{}(results[j][0], values[j]));
@@ -210,7 +214,8 @@ TEST(QueryPlan, OrderByMultiple) {
   auto n_p2_ne = NEXPR("n.p2", n_p2);
   symbol_table[*n_p2_ne] = symbol_table.CreateSymbol("n.p2", true);
   auto produce = MakeProduce(order_by, n_p1_ne, n_p2_ne);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(N * N, results.size());
   for (int j = 0; j < N * N; ++j) {
     ASSERT_EQ(results[j][0].type(), TypedValue::Type::Int);
@@ -261,6 +266,7 @@ TEST(QueryPlan, OrderByExceptions) {
     auto order_by = std::make_shared<plan::OrderBy>(
         n.op_, std::vector<SortItem>{{Ordering::ASC, n_p}},
         std::vector<Symbol>{});
-    EXPECT_THROW(PullAll(order_by, dba, symbol_table), QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    EXPECT_THROW(PullAll(*order_by, &context), QueryRuntimeException);
   }
 }
diff --git a/tests/unit/query_plan_checker.hpp b/tests/unit/query_plan_checker.hpp
index 7c0ca8641..ecb228efd 100644
--- a/tests/unit/query_plan_checker.hpp
+++ b/tests/unit/query_plan_checker.hpp
@@ -341,10 +341,10 @@ std::list<std::unique_ptr<BaseOpChecker>> MakeCheckers(T arg, Rest &&... rest) {
 }
 
 template <class TPlanner, class TDbAccessor>
-TPlanner MakePlanner(const TDbAccessor &dba, AstStorage &storage,
+TPlanner MakePlanner(TDbAccessor *dba, AstStorage &storage,
                      SymbolTable &symbol_table, CypherQuery *query) {
   auto planning_context =
-      MakePlanningContext(storage, symbol_table, query, dba);
+      MakePlanningContext(&storage, &symbol_table, query, dba);
   auto query_parts = CollectQueryParts(symbol_table, storage, query);
   auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
   return TPlanner(single_query_parts, planning_context);
diff --git a/tests/unit/query_plan_common.hpp b/tests/unit/query_plan_common.hpp
index 33df9dbd1..325cac519 100644
--- a/tests/unit/query_plan_common.hpp
+++ b/tests/unit/query_plan_common.hpp
@@ -17,26 +17,33 @@ using namespace query::plan;
 
 using Bound = ScanAllByLabelPropertyRange::Bound;
 
+Context MakeContext(const AstStorage &storage, const SymbolTable &symbol_table,
+                    database::GraphDbAccessor *dba) {
+  Context context(*dba);
+  context.symbol_table_ = symbol_table;
+  context.evaluation_context_.properties =
+      NamesToProperties(storage.properties_, dba);
+  context.evaluation_context_.labels = NamesToLabels(storage.labels_, dba);
+  return context;
+}
+
 /** Helper function that collects all the results from the given Produce. */
-std::vector<std::vector<TypedValue>> CollectProduce(
-    Produce *produce, SymbolTable &symbol_table,
-    database::GraphDbAccessor &db_accessor) {
-  Frame frame(symbol_table.max_position());
+std::vector<std::vector<TypedValue>> CollectProduce(const Produce &produce,
+                                                    Context *context) {
+  Frame frame(context->symbol_table_.max_position());
 
   // top level node in the operator tree is a produce (return)
   // so stream out results
 
   // collect the symbols from the return clause
   std::vector<Symbol> symbols;
-  for (auto named_expression : produce->named_expressions_)
-    symbols.emplace_back(symbol_table[*named_expression]);
+  for (auto named_expression : produce.named_expressions_)
+    symbols.emplace_back(context->symbol_table_[*named_expression]);
 
-  Context context(db_accessor);
-  context.symbol_table_ = symbol_table;
   // stream out results
-  auto cursor = produce->MakeCursor(db_accessor);
+  auto cursor = produce.MakeCursor(context->db_accessor_);
   std::vector<std::vector<TypedValue>> results;
-  while (cursor->Pull(frame, context)) {
+  while (cursor->Pull(frame, *context)) {
     std::vector<TypedValue> values;
     for (auto &symbol : symbols) values.emplace_back(frame[symbol]);
     results.emplace_back(values);
@@ -45,14 +52,11 @@ std::vector<std::vector<TypedValue>> CollectProduce(
   return results;
 }
 
-int PullAll(std::shared_ptr<LogicalOperator> logical_op,
-            database::GraphDbAccessor &db, SymbolTable &symbol_table) {
-  Frame frame(symbol_table.max_position());
-  auto cursor = logical_op->MakeCursor(db);
+int PullAll(const LogicalOperator &logical_op, Context *context) {
+  Frame frame(context->symbol_table_.max_position());
+  auto cursor = logical_op.MakeCursor(context->db_accessor_);
   int count = 0;
-  Context context(db);
-  context.symbol_table_ = symbol_table;
-  while (cursor->Pull(frame, context)) count++;
+  while (cursor->Pull(frame, *context)) count++;
   return count;
 }
 
@@ -114,6 +118,7 @@ ScanAllTuple MakeScanAllByLabel(
 ScanAllTuple MakeScanAllByLabelPropertyRange(
     AstStorage &storage, SymbolTable &symbol_table, std::string identifier,
     storage::Label label, storage::Property property,
+    const std::string &property_name,
     std::experimental::optional<Bound> lower_bound,
     std::experimental::optional<Bound> upper_bound,
     std::shared_ptr<LogicalOperator> input = {nullptr},
@@ -122,7 +127,8 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
   auto symbol = symbol_table.CreateSymbol(identifier, true);
   symbol_table[*node->identifier_] = symbol;
   auto logical_op = std::make_shared<ScanAllByLabelPropertyRange>(
-      input, symbol, label, property, lower_bound, upper_bound, graph_view);
+      input, symbol, label, property, property_name, lower_bound, upper_bound,
+      graph_view);
   return ScanAllTuple{node, logical_op, symbol};
 }
 
@@ -134,14 +140,15 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
  */
 ScanAllTuple MakeScanAllByLabelPropertyValue(
     AstStorage &storage, SymbolTable &symbol_table, std::string identifier,
-    storage::Label label, storage::Property property, Expression *value,
+    storage::Label label, storage::Property property,
+    const std::string &property_name, Expression *value,
     std::shared_ptr<LogicalOperator> input = {nullptr},
     GraphView graph_view = GraphView::OLD) {
   auto node = NODE(identifier);
   auto symbol = symbol_table.CreateSymbol(identifier, true);
   symbol_table[*node->identifier_] = symbol;
   auto logical_op = std::make_shared<ScanAllByLabelPropertyValue>(
-      input, symbol, label, property, value, graph_view);
+      input, symbol, label, property, property_name, value, graph_view);
   return ScanAllTuple{node, logical_op, symbol};
 }
 
diff --git a/tests/unit/query_plan_create_set_remove_delete.cpp b/tests/unit/query_plan_create_set_remove_delete.cpp
index ea95c5f37..63d7d6941 100644
--- a/tests/unit/query_plan_create_set_remove_delete.cpp
+++ b/tests/unit/query_plan_create_set_remove_delete.cpp
@@ -33,7 +33,8 @@ TEST(QueryPlan, CreateNodeWithAttributes) {
   node.properties.emplace_back(property.second, LITERAL(42));
 
   auto create = std::make_shared<CreateNode>(nullptr, node);
-  PullAll(create, dba, symbol_table);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  PullAll(*create, &context);
   dba.AdvanceCommand();
 
   // count the number of vertices
@@ -79,7 +80,8 @@ TEST(QueryPlan, CreateReturn) {
   symbol_table[*named_expr_n->expression_] = node.symbol;
 
   auto produce = MakeProduce(create, named_expr_n, named_expr_n_p);
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(1, results.size());
   EXPECT_EQ(2, results[0].size());
   EXPECT_EQ(TypedValue::Type::Vertex, results[0][0].type());
@@ -130,7 +132,8 @@ TEST(QueryPlan, CreateExpand) {
     auto create_op = std::make_shared<CreateNode>(nullptr, n);
     auto create_expand =
         std::make_shared<CreateExpand>(m, r, create_op, n.symbol, cycle);
-    PullAll(create_expand, dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    PullAll(*create_expand, &context);
     dba.AdvanceCommand();
 
     EXPECT_EQ(CountIterable(dba.Vertices(false)) - before_v,
@@ -185,7 +188,8 @@ TEST(QueryPlan, MatchCreateNode) {
   auto create_node = std::make_shared<CreateNode>(n_scan_all.op_, m);
 
   EXPECT_EQ(CountIterable(dba->Vertices(false)), 3);
-  PullAll(create_node, *dba, symbol_table);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  PullAll(*create_node, &context);
   dba->AdvanceCommand();
   EXPECT_EQ(CountIterable(dba->Vertices(false)), 6);
 }
@@ -227,7 +231,8 @@ TEST(QueryPlan, MatchCreateExpand) {
 
     auto create_expand = std::make_shared<CreateExpand>(m, r, n_scan_all.op_,
                                                         n_scan_all.sym_, cycle);
-    PullAll(create_expand, *dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    PullAll(*create_expand, &context);
     dba->AdvanceCommand();
 
     EXPECT_EQ(CountIterable(dba->Vertices(false)) - before_v,
@@ -266,7 +271,8 @@ TEST(QueryPlan, Delete) {
     symbol_table[*n_get] = n.sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, false);
-    EXPECT_THROW(PullAll(delete_op, *dba, symbol_table), QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_THROW(PullAll(*delete_op, &context), QueryRuntimeException);
     dba->AdvanceCommand();
     EXPECT_EQ(4, CountIterable(dba->Vertices(false)));
     EXPECT_EQ(6, CountIterable(dba->Edges(false)));
@@ -280,8 +286,7 @@ TEST(QueryPlan, Delete) {
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, true);
     Frame frame(symbol_table.max_position());
-    Context context(*dba);
-    context.symbol_table_ = symbol_table;
+    Context context = MakeContext(storage, symbol_table, dba.get());
     delete_op->MakeCursor(*dba)->Pull(frame, context);
     dba->AdvanceCommand();
     EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
@@ -298,7 +303,8 @@ TEST(QueryPlan, Delete) {
     symbol_table[*r_get] = r_m.edge_sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         r_m.op_, std::vector<Expression *>{r_get}, false);
-    PullAll(delete_op, *dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    PullAll(*delete_op, &context);
     dba->AdvanceCommand();
     EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
     EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@@ -311,7 +317,8 @@ TEST(QueryPlan, Delete) {
     symbol_table[*n_get] = n.sym_;
     auto delete_op = std::make_shared<plan::Delete>(
         n.op_, std::vector<Expression *>{n_get}, false);
-    PullAll(delete_op, *dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    PullAll(*delete_op, &context);
     dba->AdvanceCommand();
     EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
     EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@@ -359,7 +366,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
 
     auto delete_op = std::make_shared<plan::Delete>(
         r_m.op_, std::vector<Expression *>{n_get, r_get, m_get}, detach);
-    EXPECT_EQ(2, PullAll(delete_op, *dba, symbol_table));
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_EQ(2, PullAll(*delete_op, &context));
     dba->AdvanceCommand();
     EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
     EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@@ -395,14 +403,14 @@ TEST(QueryPlan, DeleteReturn) {
   auto delete_op = std::make_shared<plan::Delete>(
       n.op_, std::vector<Expression *>{n_get}, true);
 
-  auto prop_lookup =
-      storage.Create<PropertyLookup>(storage.Create<Identifier>("n"), prop);
+  auto prop_lookup = PROPERTY_LOOKUP("n", prop);
   symbol_table[*prop_lookup->expression_] = n.sym_;
   auto n_p = storage.Create<NamedExpression>("n", prop_lookup);
   symbol_table[*n_p] = symbol_table.CreateSymbol("bla", true);
   auto produce = MakeProduce(delete_op, n_p);
 
-  auto results = CollectProduce(produce.get(), symbol_table, dba);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(4, results.size());
   dba.AdvanceCommand();
   EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
@@ -418,7 +426,8 @@ TEST(QueryPlan, DeleteNull) {
   auto once = std::make_shared<Once>();
   auto delete_op = std::make_shared<plan::Delete>(
       once, std::vector<Expression *>{LITERAL(TypedValue::Null)}, false);
-  EXPECT_EQ(1, PullAll(delete_op, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*delete_op, &context));
 }
 
 TEST(QueryPlan, DeleteAdvance) {
@@ -446,7 +455,8 @@ TEST(QueryPlan, DeleteAdvance) {
       n.op_, std::vector<Expression *>{n_get}, false);
   auto advance = std::make_shared<Accumulate>(
       delete_op, std::vector<Symbol>{n.sym_}, true);
-  EXPECT_THROW(PullAll(advance, *dba, symbol_table), ReconstructionException);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_THROW(PullAll(*advance, &context), ReconstructionException);
 }
 
 TEST(QueryPlan, SetProperty) {
@@ -481,12 +491,15 @@ TEST(QueryPlan, SetProperty) {
 
   auto n_p = PROPERTY_LOOKUP("n", prop1);
   symbol_table[*n_p->expression_] = n.sym_;
-  auto set_n_p = std::make_shared<plan::SetProperty>(r_m.op_, n_p, literal);
+  auto set_n_p =
+      std::make_shared<plan::SetProperty>(r_m.op_, prop1, n_p, literal);
 
   auto r_p = PROPERTY_LOOKUP("r", prop1);
   symbol_table[*r_p->expression_] = r_m.edge_sym_;
-  auto set_r_p = std::make_shared<plan::SetProperty>(set_n_p, r_p, literal);
-  EXPECT_EQ(2, PullAll(set_r_p, dba, symbol_table));
+  auto set_r_p =
+      std::make_shared<plan::SetProperty>(set_n_p, prop1, r_p, literal);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(2, PullAll(*set_r_p, &context));
   dba.AdvanceCommand();
 
   EXPECT_EQ(CountIterable(dba.Edges(false)), 2);
@@ -539,7 +552,8 @@ TEST(QueryPlan, SetProperties) {
         std::make_shared<plan::SetProperties>(r_m.op_, n.sym_, r_ident, op);
     auto set_m_to_r = std::make_shared<plan::SetProperties>(
         set_r_to_n, r_m.edge_sym_, m_ident, op);
-    EXPECT_EQ(1, PullAll(set_m_to_r, *dba, symbol_table));
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_EQ(1, PullAll(*set_m_to_r, &context));
     dba->AdvanceCommand();
 
     EXPECT_EQ(CountIterable(dba->Edges(false)), 1);
@@ -589,7 +603,8 @@ TEST(QueryPlan, SetLabels) {
   auto n = MakeScanAll(storage, symbol_table, "n");
   auto label_set = std::make_shared<plan::SetLabels>(
       n.op_, n.sym_, std::vector<storage::Label>{label2, label3});
-  EXPECT_EQ(2, PullAll(label_set, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(2, PullAll(*label_set, &context));
 
   for (VertexAccessor vertex : dba->Vertices(false)) {
     vertex.SwitchNew();
@@ -634,12 +649,13 @@ TEST(QueryPlan, RemoveProperty) {
 
   auto n_p = PROPERTY_LOOKUP("n", prop1);
   symbol_table[*n_p->expression_] = n.sym_;
-  auto set_n_p = std::make_shared<plan::RemoveProperty>(r_m.op_, n_p);
+  auto set_n_p = std::make_shared<plan::RemoveProperty>(r_m.op_, prop1, n_p);
 
   auto r_p = PROPERTY_LOOKUP("r", prop1);
   symbol_table[*r_p->expression_] = r_m.edge_sym_;
-  auto set_r_p = std::make_shared<plan::RemoveProperty>(set_n_p, r_p);
-  EXPECT_EQ(2, PullAll(set_r_p, dba, symbol_table));
+  auto set_r_p = std::make_shared<plan::RemoveProperty>(set_n_p, prop1, r_p);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(2, PullAll(*set_r_p, &context));
   dba.AdvanceCommand();
 
   EXPECT_EQ(CountIterable(dba.Edges(false)), 2);
@@ -675,7 +691,8 @@ TEST(QueryPlan, RemoveLabels) {
   auto n = MakeScanAll(storage, symbol_table, "n");
   auto label_remove = std::make_shared<plan::RemoveLabels>(
       n.op_, n.sym_, std::vector<storage::Label>{label1, label2});
-  EXPECT_EQ(2, PullAll(label_remove, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(2, PullAll(*label_remove, &context));
 
   for (VertexAccessor vertex : dba->Vertices(false)) {
     vertex.SwitchNew();
@@ -706,20 +723,23 @@ TEST(QueryPlan, NodeFilterSet) {
   SymbolTable symbol_table;
   // MATCH (n {prop: 42}) -[r]- (m)
   auto scan_all = MakeScanAll(storage, symbol_table, "n");
-  scan_all.node_->properties_[prop] = LITERAL(42);
+  scan_all.node_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
   auto expand =
       MakeExpand(storage, symbol_table, scan_all.op_, scan_all.sym_, "r",
                  EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
   auto *filter_expr =
-      EQ(storage.Create<PropertyLookup>(scan_all.node_->identifier_, prop),
+      EQ(storage.Create<PropertyLookup>(scan_all.node_->identifier_,
+                                        storage.GetPropertyIx(prop.first)),
          LITERAL(42));
   auto node_filter = std::make_shared<Filter>(expand.op_, filter_expr);
   // SET n.prop = n.prop + 1
   auto set_prop = PROPERTY_LOOKUP("n", prop);
   symbol_table[*set_prop->expression_] = scan_all.sym_;
   auto add = ADD(set_prop, LITERAL(1));
-  auto set = std::make_shared<plan::SetProperty>(node_filter, set_prop, add);
-  EXPECT_EQ(2, PullAll(set, dba, symbol_table));
+  auto set = std::make_shared<plan::SetProperty>(node_filter, prop.second,
+                                                 set_prop, add);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(2, PullAll(*set, &context));
   dba.AdvanceCommand();
   v1.Reconstruct();
   auto prop_eq = v1.PropsAt(prop.second) == TypedValue(42 + 2);
@@ -747,7 +767,7 @@ TEST(QueryPlan, FilterRemove) {
   SymbolTable symbol_table;
   // MATCH (n) -[r]- (m) WHERE n.prop < 43
   auto scan_all = MakeScanAll(storage, symbol_table, "n");
-  scan_all.node_->properties_[prop] = LITERAL(42);
+  scan_all.node_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
   auto expand =
       MakeExpand(storage, symbol_table, scan_all.op_, scan_all.sym_, "r",
                  EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
@@ -758,8 +778,10 @@ TEST(QueryPlan, FilterRemove) {
   // REMOVE n.prop
   auto rem_prop = PROPERTY_LOOKUP("n", prop);
   symbol_table[*rem_prop->expression_] = scan_all.sym_;
-  auto rem = std::make_shared<plan::RemoveProperty>(filter, rem_prop);
-  EXPECT_EQ(2, PullAll(rem, dba, symbol_table));
+  auto rem =
+      std::make_shared<plan::RemoveProperty>(filter, prop.second, rem_prop);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(2, PullAll(*rem, &context));
   dba.AdvanceCommand();
   v1.Reconstruct();
   EXPECT_EQ(v1.PropsAt(prop.second).type(), PropertyValue::Type::Null);
@@ -782,7 +804,8 @@ TEST(QueryPlan, SetRemove) {
       scan_all.op_, scan_all.sym_, std::vector<storage::Label>{label1, label2});
   auto rem = std::make_shared<plan::RemoveLabels>(
       set, scan_all.sym_, std::vector<storage::Label>{label1, label2});
-  EXPECT_EQ(1, PullAll(rem, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*rem, &context));
   dba->AdvanceCommand();
   v.Reconstruct();
   EXPECT_FALSE(v.has_label(label1));
@@ -817,16 +840,18 @@ TEST(QueryPlan, Merge) {
                  EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
   auto m_p = PROPERTY_LOOKUP("m", prop);
   symbol_table[*m_p->expression_] = r_m.node_sym_;
-  auto m_set = std::make_shared<plan::SetProperty>(r_m.op_, m_p, LITERAL(1));
+  auto m_set = std::make_shared<plan::SetProperty>(r_m.op_, prop.second, m_p,
+                                                   LITERAL(1));
 
   // merge_create branch
   auto n_p = PROPERTY_LOOKUP("n", prop);
   symbol_table[*n_p->expression_] = n.sym_;
-  auto n_set = std::make_shared<plan::SetProperty>(std::make_shared<Once>(),
-                                                   n_p, LITERAL(2));
+  auto n_set = std::make_shared<plan::SetProperty>(
+      std::make_shared<Once>(), prop.second, n_p, LITERAL(2));
 
   auto merge = std::make_shared<plan::Merge>(n.op_, m_set, n_set);
-  ASSERT_EQ(3, PullAll(merge, dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, &dba);
+  ASSERT_EQ(3, PullAll(*merge, &context));
   dba.AdvanceCommand();
   v1.Reconstruct();
   v2.Reconstruct();
@@ -854,7 +879,8 @@ TEST(QueryPlan, MergeNoInput) {
   auto merge = std::make_shared<plan::Merge>(nullptr, create, create);
 
   EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
-  EXPECT_EQ(1, PullAll(merge, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*merge, &context));
   dba->AdvanceCommand();
   EXPECT_EQ(1, CountIterable(dba->Vertices(false)));
 }
@@ -869,10 +895,12 @@ TEST(QueryPlan, SetPropertyOnNull) {
   auto prop = PROPERTY_PAIR("property");
   auto null = LITERAL(TypedValue::Null);
   auto literal = LITERAL(42);
-  auto n_prop = storage.Create<PropertyLookup>(null, prop);
+  auto n_prop = PROPERTY_LOOKUP(null, prop);
   auto once = std::make_shared<Once>();
-  auto set_op = std::make_shared<plan::SetProperty>(once, n_prop, literal);
-  EXPECT_EQ(1, PullAll(set_op, dba, symbol_table));
+  auto set_op =
+      std::make_shared<plan::SetProperty>(once, prop.second, n_prop, literal);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(1, PullAll(*set_op, &context));
 }
 
 TEST(QueryPlan, SetPropertiesOnNull) {
@@ -889,7 +917,8 @@ TEST(QueryPlan, SetPropertiesOnNull) {
   auto set_op = std::make_shared<plan::SetProperties>(
       optional, n.sym_, n_ident, plan::SetProperties::Op::REPLACE);
   EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
-  EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*set_op, &context));
 }
 
 TEST(QueryPlan, SetLabelsOnNull) {
@@ -907,7 +936,8 @@ TEST(QueryPlan, SetLabelsOnNull) {
   auto set_op = std::make_shared<plan::SetLabels>(
       optional, n.sym_, std::vector<storage::Label>{label});
   EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
-  EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*set_op, &context));
 }
 
 TEST(QueryPlan, RemovePropertyOnNull) {
@@ -919,10 +949,12 @@ TEST(QueryPlan, RemovePropertyOnNull) {
   SymbolTable symbol_table;
   auto prop = PROPERTY_PAIR("property");
   auto null = LITERAL(TypedValue::Null);
-  auto n_prop = storage.Create<PropertyLookup>(null, prop);
+  auto n_prop = PROPERTY_LOOKUP(null, prop);
   auto once = std::make_shared<Once>();
-  auto remove_op = std::make_shared<plan::RemoveProperty>(once, n_prop);
-  EXPECT_EQ(1, PullAll(remove_op, dba, symbol_table));
+  auto remove_op =
+      std::make_shared<plan::RemoveProperty>(once, prop.second, n_prop);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(1, PullAll(*remove_op, &context));
 }
 
 TEST(QueryPlan, RemoveLabelsOnNull) {
@@ -940,7 +972,8 @@ TEST(QueryPlan, RemoveLabelsOnNull) {
   auto remove_op = std::make_shared<plan::RemoveLabels>(
       optional, n.sym_, std::vector<storage::Label>{label});
   EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
-  EXPECT_EQ(1, PullAll(remove_op, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*remove_op, &context));
 }
 
 TEST(QueryPlan, DeleteSetProperty) {
@@ -962,9 +995,10 @@ TEST(QueryPlan, DeleteSetProperty) {
   auto prop = PROPERTY_PAIR("property");
   auto n_prop = PROPERTY_LOOKUP("n", prop);
   symbol_table[*n_prop->expression_] = n.sym_;
-  auto set_op =
-      std::make_shared<plan::SetProperty>(delete_op, n_prop, LITERAL(42));
-  EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
+  auto set_op = std::make_shared<plan::SetProperty>(delete_op, prop.second,
+                                                    n_prop, LITERAL(42));
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
 }
 
 TEST(QueryPlan, DeleteSetPropertiesFromMap) {
@@ -986,16 +1020,16 @@ TEST(QueryPlan, DeleteSetPropertiesFromMap) {
   auto prop = PROPERTY_PAIR("property");
   auto n_prop = PROPERTY_LOOKUP("n", prop);
   symbol_table[*n_prop->expression_] = n.sym_;
-  std::unordered_map<std::pair<std::string, storage::Property>, Expression *>
-      prop_map;
-  prop_map.emplace(prop, LITERAL(42));
+  std::unordered_map<PropertyIx, Expression *> prop_map;
+  prop_map.emplace(storage.GetPropertyIx(prop.first), LITERAL(42));
   auto *rhs = storage.Create<MapLiteral>(prop_map);
   symbol_table[*rhs] = n.sym_;
   for (auto op_type :
        {plan::SetProperties::Op::REPLACE, plan::SetProperties::Op::UPDATE}) {
     auto set_op =
         std::make_shared<plan::SetProperties>(delete_op, n.sym_, rhs, op_type);
-    EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
   }
 }
 
@@ -1027,7 +1061,8 @@ TEST(QueryPlan, DeleteSetPropertiesFromVertex) {
        {plan::SetProperties::Op::REPLACE, plan::SetProperties::Op::UPDATE}) {
     auto set_op =
         std::make_shared<plan::SetProperties>(delete_op, n.sym_, rhs, op_type);
-    EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
   }
 }
 
@@ -1048,7 +1083,8 @@ TEST(QueryPlan, DeleteRemoveLabels) {
       n.op_, std::vector<Expression *>{n_get}, false);
   std::vector<storage::Label> labels{dba->Label("label")};
   auto rem_op = std::make_shared<plan::RemoveLabels>(delete_op, n.sym_, labels);
-  EXPECT_THROW(PullAll(rem_op, *dba, symbol_table), QueryRuntimeException);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_THROW(PullAll(*rem_op, &context), QueryRuntimeException);
 }
 
 TEST(QueryPlan, DeleteRemoveProperty) {
@@ -1070,6 +1106,8 @@ TEST(QueryPlan, DeleteRemoveProperty) {
   auto prop = PROPERTY_PAIR("property");
   auto n_prop = PROPERTY_LOOKUP("n", prop);
   symbol_table[*n_prop->expression_] = n.sym_;
-  auto rem_op = std::make_shared<plan::RemoveProperty>(delete_op, n_prop);
-  EXPECT_THROW(PullAll(rem_op, dba, symbol_table), QueryRuntimeException);
+  auto rem_op =
+      std::make_shared<plan::RemoveProperty>(delete_op, prop.second, n_prop);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_THROW(PullAll(*rem_op, &context), QueryRuntimeException);
 }
diff --git a/tests/unit/query_plan_match_filter_return.cpp b/tests/unit/query_plan_match_filter_return.cpp
index b40bac54a..c3a922811 100644
--- a/tests/unit/query_plan_match_filter_return.cpp
+++ b/tests/unit/query_plan_match_filter_return.cpp
@@ -38,7 +38,8 @@ class MatchReturnFixture : public testing::Test {
   template <typename TResult>
   std::vector<TResult> Results(std::shared_ptr<Produce> &op) {
     std::vector<TResult> res;
-    for (const auto &row : CollectProduce(op.get(), symbol_table, *dba_))
+    Context context = MakeContext(storage, symbol_table, dba_.get());
+    for (const auto &row : CollectProduce(*op, &context))
       res.emplace_back(row[0].Value<TResult>());
     return res;
   }
@@ -56,7 +57,8 @@ TEST_F(MatchReturnFixture, MatchReturn) {
     symbol_table[*output->expression_] = scan_all.sym_;
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
-    return PullAll(produce, *dba_, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba_.get());
+    return PullAll(*produce, &context);
   };
 
   EXPECT_EQ(2, test_pull_count(GraphView::NEW));
@@ -111,8 +113,8 @@ TEST(QueryPlan, MatchReturnCartesian) {
   symbol_table[*return_m] =
       symbol_table.CreateSymbol("named_expression_2", true);
   auto produce = MakeProduce(m.op_, return_n, return_m);
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 4);
   // ensure the result ordering is OK:
   // "n" from the results is the same for the first two rows, while "m" isn't
@@ -138,7 +140,8 @@ TEST(QueryPlan, StandaloneReturn) {
   auto produce = MakeProduce(std::shared_ptr<LogicalOperator>(nullptr), output);
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
 
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 1);
   EXPECT_EQ(results[0].size(), 1);
   EXPECT_EQ(results[0][0].Value<int64_t>(), 42);
@@ -176,8 +179,8 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
 
   // make a scan all
   auto n = MakeScanAll(storage, symbol_table, "n");
-  n.node_->labels_.emplace_back(label);
-  n.node_->properties_[property] = LITERAL(42);
+  n.node_->labels_.emplace_back(storage.GetLabelIx(dba.LabelName(label)));
+  n.node_->properties_[storage.GetPropertyIx(property.first)] = LITERAL(42);
 
   // node filtering
   auto *filter_expr =
@@ -191,14 +194,15 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   auto produce = MakeProduce(node_filter, output);
 
-  EXPECT_EQ(1, PullAll(produce, dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(1, PullAll(*produce, &context));
 
   //  test that filtering works with old records
   v4.Reconstruct();
   v4.add_label(label);
-  EXPECT_EQ(1, PullAll(produce, dba, symbol_table));
+  EXPECT_EQ(1, PullAll(*produce, &context));
   dba.AdvanceCommand();
-  EXPECT_EQ(2, PullAll(produce, dba, symbol_table));
+  EXPECT_EQ(2, PullAll(*produce, &context));
 }
 
 TEST(QueryPlan, NodeFilterMultipleLabels) {
@@ -231,8 +235,8 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
 
   // make a scan all
   auto n = MakeScanAll(storage, symbol_table, "n");
-  n.node_->labels_.emplace_back(label1);
-  n.node_->labels_.emplace_back(label2);
+  n.node_->labels_.emplace_back(storage.GetLabelIx(dba->LabelName(label1)));
+  n.node_->labels_.emplace_back(storage.GetLabelIx(dba->LabelName(label2)));
 
   // node filtering
   auto *filter_expr =
@@ -247,7 +251,8 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   symbol_table[*output->expression_] = n.sym_;
 
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 2);
 }
 
@@ -286,7 +291,8 @@ TEST(QueryPlan, Cartesian) {
 
   auto produce = MakeProduce(cartesian_op, return_n, return_m);
 
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 9);
   for (int i = 0; i < 3; ++i) {
     for (int j = 0; j < 3; ++j) {
@@ -320,8 +326,8 @@ TEST(QueryPlan, CartesianEmptySet) {
       std::make_shared<Cartesian>(n.op_, left_symbols, m.op_, right_symbols);
 
   auto produce = MakeProduce(cartesian_op, return_n, return_m);
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 0);
 }
 
@@ -368,8 +374,8 @@ TEST(QueryPlan, CartesianThreeWay) {
                                                     l.op_, l_symbols);
 
   auto produce = MakeProduce(cartesian_op_2, return_n, return_m, return_l);
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 27);
   int id = 0;
   for (int i = 0; i < 3; ++i) {
@@ -419,8 +425,8 @@ TEST_F(ExpandFixture, Expand) {
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
     auto produce = MakeProduce(r_m.op_, output);
-
-    return PullAll(produce, *dba_, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba_.get());
+    return PullAll(*produce, &context);
   };
 
   // test that expand works well for both old and new graph state
@@ -456,7 +462,8 @@ TEST_F(ExpandFixture, ExpandPath) {
   auto produce = MakeProduce(path, output);
 
   std::vector<query::Path> expected_paths{{v1, r2, v3}, {v1, r1, v2}};
-  auto results = CollectProduce(produce.get(), symbol_table, *dba_);
+  Context context = MakeContext(storage, symbol_table, dba_.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(results.size(), 2);
   std::vector<query::Path> results_paths;
   for (const auto &result : results)
@@ -546,9 +553,10 @@ class QueryPlanExpandVariable : public testing::Test {
       bool is_reverse = false) {
     auto n_from = MakeScanAll(storage, symbol_table, node_from, input_op);
     auto filter_op = std::make_shared<Filter>(
-        n_from.op_, storage.Create<query::LabelsTest>(
-                        n_from.node_->identifier_,
-                        std::vector<storage::Label>{labels[layer]}));
+        n_from.op_,
+        storage.Create<query::LabelsTest>(
+            n_from.node_->identifier_, std::vector<LabelIx>{storage.GetLabelIx(
+                                           dba_->LabelName(labels[layer]))}));
 
     auto n_to = NODE(node_to);
     auto n_to_sym = symbol_table.CreateSymbol(node_to, true);
@@ -592,8 +600,7 @@ class QueryPlanExpandVariable : public testing::Test {
   auto GetResults(std::shared_ptr<LogicalOperator> input_op, Symbol symbol) {
     Frame frame(symbol_table.max_position());
     auto cursor = input_op->MakeCursor(*dba_);
-    Context context(*dba_);
-    context.symbol_table_ = symbol_table;
+    Context context = MakeContext(storage, symbol_table, dba_.get());
     std::vector<TResult> results;
     while (cursor->Pull(frame, context))
       results.emplace_back(frame[symbol].Value<TResult>());
@@ -710,8 +717,8 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessSingleAndVariableExpansion) {
     if (add_uniqueness_check) {
       auto last_symbol = symbols.back();
       symbols.pop_back();
-      last_op = std::make_shared<EdgeUniquenessFilter>(
-          last_op, last_symbol, symbols);
+      last_op =
+          std::make_shared<EdgeUniquenessFilter>(last_op, last_symbol, symbols);
     }
 
     return GetEdgeListSizes(last_op, var_length_sym);
@@ -741,8 +748,8 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessTwoVariableExpansions) {
         AddMatch<ExpandVariable>(first, "n2", layer, direction, {}, lower,
                                  upper, e2, "m2", GraphView::OLD);
     if (add_uniqueness_check) {
-      last_op = std::make_shared<EdgeUniquenessFilter>(
-          last_op, e2, std::vector<Symbol>{e1});
+      last_op = std::make_shared<EdgeUniquenessFilter>(last_op, e2,
+                                                       std::vector<Symbol>{e1});
     }
 
     return GetEdgeListSizes(last_op, e2);
@@ -929,8 +936,7 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
     Frame frame(symbol_table.max_position());
     auto cursor = last_op->MakeCursor(dba);
     std::vector<ResultType> results;
-    Context context(dba);
-    context.symbol_table_ = symbol_table;
+    Context context = MakeContext(storage, symbol_table, &dba);
     while (cursor->Pull(frame, context)) {
       results.push_back(ResultType{std::vector<EdgeAccessor>(),
                                    frame[node_sym].Value<VertexAccessor>(),
@@ -1231,8 +1237,8 @@ TEST(QueryPlan, ExpandOptional) {
   symbol_table[*m_ne->expression_] = r_m.node_sym_;
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(optional, n_ne, r_ne, m_ne);
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(4, results.size());
   int v1_is_n_count = 0;
   for (auto &row : results) {
@@ -1268,8 +1274,8 @@ TEST(QueryPlan, OptionalMatchEmptyDB) {
   auto optional = std::make_shared<plan::Optional>(nullptr, n.op_,
                                                    std::vector<Symbol>{n.sym_});
   auto produce = MakeProduce(optional, n_ne);
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(1, results.size());
   EXPECT_EQ(results[0][0].type(), TypedValue::Type::Null);
 }
@@ -1298,7 +1304,8 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
   symbol_table[*m_ne->expression_] = r_m.node_sym_;
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(r_m.op_, m_ne);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(0, results.size());
 }
 
@@ -1317,8 +1324,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
   SymbolTable symbol_table;
   // OPTIONAL MATCH (n :missing)
   auto n = MakeScanAll(storage, symbol_table, "n");
-  auto label_missing = dba->Label("missing");
-  n.node_->labels_.emplace_back(label_missing);
+  auto label_missing = "missing";
+  n.node_->labels_.emplace_back(storage.GetLabelIx(label_missing));
 
   auto *filter_expr =
       storage.Create<LabelsTest>(n.node_->identifier_, n.node_->labels_);
@@ -1347,7 +1354,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
   symbol_table[*m_ne->expression_] = m.sym_;
   symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
   auto produce = MakeProduce(expand, m_ne);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(0, results.size());
 }
 
@@ -1383,8 +1391,8 @@ TEST(QueryPlan, ExpandExistingNode) {
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
     auto produce = MakeProduce(r_n.op_, output);
-
-    auto results = CollectProduce(produce.get(), symbol_table, *dba);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    auto results = CollectProduce(*produce, &context);
     EXPECT_EQ(results.size(), expected_result_count);
   };
 
@@ -1409,7 +1417,8 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
   auto r_ =
       MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
                  EdgeAtom::Direction::BOTH, {}, "_", false, GraphView::OLD);
-  EXPECT_EQ(1, PullAll(r_.op_, *dba, symbol_table));
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(1, PullAll(*r_.op_, &context));
 }
 
 TEST(QueryPlan, EdgeFilter) {
@@ -1458,8 +1467,9 @@ TEST(QueryPlan, EdgeFilter) {
     auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
                           EdgeAtom::Direction::OUT, {edge_type}, "m", false,
                           GraphView::OLD);
-    r_m.edge_->edge_types_.push_back(edge_type);
-    r_m.edge_->properties_[prop] = LITERAL(42);
+    r_m.edge_->edge_types_.push_back(
+        storage.GetEdgeTypeIx(dba.EdgeTypeName(edge_type)));
+    r_m.edge_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
     auto *filter_expr =
         EQ(PROPERTY_LOOKUP(r_m.edge_->identifier_, prop), LITERAL(42));
     auto edge_filter = std::make_shared<Filter>(r_m.op_, filter_expr);
@@ -1470,8 +1480,8 @@ TEST(QueryPlan, EdgeFilter) {
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
     auto produce = MakeProduce(edge_filter, output);
-
-    return PullAll(produce, dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    return PullAll(*produce, &context);
   };
 
   EXPECT_EQ(1, test_filter());
@@ -1512,8 +1522,8 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) {
   // fill up the symbol table
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   symbol_table[*output->expression_] = r_m.node_sym_;
-
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 2);
 }
 
@@ -1533,8 +1543,7 @@ TEST(QueryPlan, Filter) {
   SymbolTable symbol_table;
 
   auto n = MakeScanAll(storage, symbol_table, "n");
-  auto e =
-      storage.Create<PropertyLookup>(storage.Create<Identifier>("n"), property);
+  auto e = PROPERTY_LOOKUP("n", property);
   symbol_table[*e->expression_] = n.sym_;
   auto f = std::make_shared<Filter>(n.op_, e);
 
@@ -1543,8 +1552,8 @@ TEST(QueryPlan, Filter) {
   symbol_table[*output->expression_] = n.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
   auto produce = MakeProduce(f, output);
-
-  EXPECT_EQ(CollectProduce(produce.get(), symbol_table, dba).size(), 2);
+  Context context = MakeContext(storage, symbol_table, &dba);
+  EXPECT_EQ(CollectProduce(*produce, &context).size(), 2);
 }
 
 TEST(QueryPlan, EdgeUniquenessFilter) {
@@ -1575,7 +1584,8 @@ TEST(QueryPlan, EdgeUniquenessFilter) {
     if (edge_uniqueness)
       last_op = std::make_shared<EdgeUniquenessFilter>(
           last_op, r2_n3.edge_sym_, std::vector<Symbol>{r1_n2.edge_sym_});
-    return PullAll(last_op, *dba, symbol_table);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    return PullAll(*last_op, &context);
   };
 
   EXPECT_EQ(2, check_expand_results(false));
@@ -1607,8 +1617,8 @@ TEST(QueryPlan, Distinct) {
     auto x_ne = NEXPR("x", x_expr);
     symbol_table[*x_ne] = symbol_table.CreateSymbol("x_ne", true);
     auto produce = MakeProduce(distinct, x_ne);
-
-    auto results = CollectProduce(produce.get(), symbol_table, *dba);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    auto results = CollectProduce(*produce, &context);
     ASSERT_EQ(output.size(), results.size());
     auto output_it = output.begin();
     for (const auto &row : results) {
@@ -1647,7 +1657,8 @@ TEST(QueryPlan, ScanAllByLabel) {
   auto produce = MakeProduce(scan_all_by_label.op_, output);
   symbol_table[*output->expression_] = scan_all_by_label.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(results.size(), 1);
   auto result_row = results[0];
   ASSERT_EQ(result_row.size(), 1);
@@ -1685,14 +1696,15 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
     AstStorage storage;
     SymbolTable symbol_table;
     auto scan_all = MakeScanAllByLabelPropertyRange(
-        storage, symbol_table, "n", label, prop,
+        storage, symbol_table, "n", label, prop, "prop",
         Bound{LITERAL(lower), lower_type}, Bound{LITERAL(upper), upper_type});
     // RETURN n
     auto output = NEXPR("n", IDENT("n"));
     auto produce = MakeProduce(scan_all.op_, output);
     symbol_table[*output->expression_] = scan_all.sym_;
     symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-    auto results = CollectProduce(produce.get(), symbol_table, *dba);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    auto results = CollectProduce(*produce, &context);
     ASSERT_EQ(results.size(), expected.size());
     for (size_t i = 0; i < expected.size(); i++) {
       TypedValue equal =
@@ -1746,14 +1758,15 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
   // MATCH (n :label {prop: 42})
   AstStorage storage;
   SymbolTable symbol_table;
-  auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n",
-                                                  label, prop, LITERAL(42));
+  auto scan_all = MakeScanAllByLabelPropertyValue(
+      storage, symbol_table, "n", label, prop, "prop", LITERAL(42));
   // RETURN n
   auto output = NEXPR("n", IDENT("n"));
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   ASSERT_EQ(results.size(), 1);
   const auto &row = results[0];
   ASSERT_EQ(row.size(), 1);
@@ -1786,9 +1799,9 @@ TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
   auto *ident_m = IDENT("m");
   symbol_table[*ident_m] = scan_all.sym_;
   auto scan_index = MakeScanAllByLabelPropertyValue(
-      storage, symbol_table, "n", label, prop, ident_m, scan_all.op_);
-  EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
-               QueryRuntimeException);
+      storage, symbol_table, "n", label, prop, "prop", ident_m, scan_all.op_);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
 }
 
 TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
@@ -1816,28 +1829,29 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
   {
     // Lower bound isn't property value
     auto scan_index = MakeScanAllByLabelPropertyRange(
-        storage, symbol_table, "n", label, prop,
+        storage, symbol_table, "n", label, prop, "prop",
         Bound{ident_m, Bound::Type::INCLUSIVE}, std::experimental::nullopt,
         scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
-                 QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
   }
   {
     // Upper bound isn't property value
     auto scan_index = MakeScanAllByLabelPropertyRange(
-        storage, symbol_table, "n", label, prop, std::experimental::nullopt,
-        Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
-                 QueryRuntimeException);
+        storage, symbol_table, "n", label, prop, "prop",
+        std::experimental::nullopt, Bound{ident_m, Bound::Type::INCLUSIVE},
+        scan_all.op_);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
   }
   {
     // Both bounds aren't property value
     auto scan_index = MakeScanAllByLabelPropertyRange(
-        storage, symbol_table, "n", label, prop,
+        storage, symbol_table, "n", label, prop, "prop",
         Bound{ident_m, Bound::Type::INCLUSIVE},
         Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
-    EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
-                 QueryRuntimeException);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
   }
 }
 
@@ -1863,14 +1877,16 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
   // MATCH (n :label {prop: 42})
   AstStorage storage;
   SymbolTable symbol_table;
-  auto scan_all = MakeScanAllByLabelPropertyValue(
-      storage, symbol_table, "n", label, prop, LITERAL(TypedValue::Null));
+  auto scan_all =
+      MakeScanAllByLabelPropertyValue(storage, symbol_table, "n", label, prop,
+                                      "prop", LITERAL(TypedValue::Null));
   // RETURN n
   auto output = NEXPR("n", IDENT("n"));
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 0);
 }
 
@@ -1897,7 +1913,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
   AstStorage storage;
   SymbolTable symbol_table;
   auto scan_all = MakeScanAllByLabelPropertyRange(
-      storage, symbol_table, "n", label, prop,
+      storage, symbol_table, "n", label, prop, "prop",
       Bound{LITERAL(TypedValue::Null), Bound::Type::INCLUSIVE},
       Bound{LITERAL(TypedValue::Null), Bound::Type::EXCLUSIVE});
   // RETURN n
@@ -1905,7 +1921,8 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
   auto produce = MakeProduce(scan_all.op_, output);
   symbol_table[*output->expression_] = scan_all.sym_;
   symbol_table[*output] = symbol_table.CreateSymbol("n", true);
-  auto results = CollectProduce(produce.get(), symbol_table, *dba);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  auto results = CollectProduce(*produce, &context);
   EXPECT_EQ(results.size(), 0);
 }
 
@@ -1935,10 +1952,11 @@ TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
   symbol_table[*x_expr] = x;
 
   // MATCH (n :label {prop: x})
-  auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n",
-                                                  label, prop, x_expr, unwind);
+  auto scan_all = MakeScanAllByLabelPropertyValue(
+      storage, symbol_table, "n", label, prop, "prop", x_expr, unwind);
 
-  EXPECT_EQ(PullAll(scan_all.op_, *dba, symbol_table), 1);
+  Context context = MakeContext(storage, symbol_table, dba.get());
+  EXPECT_EQ(PullAll(*scan_all.op_, &context), 1);
 }
 
 TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
@@ -1972,13 +1990,14 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
     SymbolTable symbol_table;
     auto dba = db.Access();
     auto scan_all_by_label_property_value = MakeScanAllByLabelPropertyValue(
-        storage, symbol_table, "n", label, prop, LITERAL(prop_value));
+        storage, symbol_table, "n", label, prop, "prop", LITERAL(prop_value));
     auto output = NEXPR("n", IDENT("n"));
     auto produce = MakeProduce(scan_all_by_label_property_value.op_, output);
     symbol_table[*output->expression_] = scan_all_by_label_property_value.sym_;
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
-    EXPECT_EQ(PullAll(produce, *dba, symbol_table), prop_count);
+    Context context = MakeContext(storage, symbol_table, dba.get());
+    EXPECT_EQ(PullAll(*produce, &context), prop_count);
   };
 
   // Make sure there are `vertex_count` results when using scan all
@@ -1988,8 +2007,7 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
     auto dba_ptr = db.Access();
     auto &dba = *dba_ptr;
     auto scan_all = MakeScanAll(storage, symbol_table, "n");
-    auto e = storage.Create<PropertyLookup>(storage.Create<Identifier>("n"),
-                                            std::make_pair("prop", prop));
+    auto e = PROPERTY_LOOKUP("n", std::make_pair("prop", prop));
     symbol_table[*e->expression_] = scan_all.sym_;
     auto filter =
         std::make_shared<Filter>(scan_all.op_, EQ(e, LITERAL(prop_value)));
@@ -1998,7 +2016,8 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
     symbol_table[*output->expression_] = scan_all.sym_;
     symbol_table[*output] =
         symbol_table.CreateSymbol("named_expression_1", true);
-    EXPECT_EQ(PullAll(produce, dba, symbol_table), prop_count);
+    Context context = MakeContext(storage, symbol_table, &dba);
+    EXPECT_EQ(PullAll(*produce, &context), prop_count);
   };
 
   count_with_index(prop_value1, vertex_prop_count);
diff --git a/tests/unit/query_pretty_print.cpp b/tests/unit/query_pretty_print.cpp
index 2b56f24d6..6bd0ee29c 100644
--- a/tests/unit/query_pretty_print.cpp
+++ b/tests/unit/query_pretty_print.cpp
@@ -28,112 +28,120 @@ struct ExpressionPrettyPrinterTest : public ::testing::Test {
 
 TEST_F(ExpressionPrettyPrinterTest, Literals) {
   // 1
-  EXPECT_EQ(ToString(LITERAL(1)), "1");
+  EXPECT_EQ(ToString(storage, LITERAL(1)), "1");
 
   // "hello"
-  EXPECT_EQ(ToString(LITERAL("hello")), "\"hello\"");
+  EXPECT_EQ(ToString(storage, LITERAL("hello")), "\"hello\"");
 
   // null
-  EXPECT_EQ(ToString(LITERAL(TypedValue::Null)), "null");
+  EXPECT_EQ(ToString(storage, LITERAL(TypedValue::Null)), "null");
 
   // true
-  EXPECT_EQ(ToString(LITERAL(true)), "true");
+  EXPECT_EQ(ToString(storage, LITERAL(true)), "true");
 
   // false
-  EXPECT_EQ(ToString(LITERAL(false)), "false");
+  EXPECT_EQ(ToString(storage, LITERAL(false)), "false");
 
   // [1 null "hello"]
-  EXPECT_EQ(ToString(LITERAL(
-                (std::vector<PropertyValue>{1, PropertyValue::Null, "hello"}))),
+  EXPECT_EQ(ToString(storage, LITERAL((std::vector<PropertyValue>{
+                                  1, PropertyValue::Null, "hello"}))),
             "[1, null, \"hello\"]");
 
   // {hello: 1, there: 2}
-  EXPECT_EQ(ToString(LITERAL((std::map<std::string, PropertyValue>{
-                {"hello", 1}, {"there", 2}}))),
+  EXPECT_EQ(ToString(storage, LITERAL((std::map<std::string, PropertyValue>{
+                                  {"hello", 1}, {"there", 2}}))),
             "{\"hello\": 1, \"there\": 2}");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, UnaryOperators) {
   // not(false)
-  EXPECT_EQ(ToString(NOT(LITERAL(false))), "(Not false)");
+  EXPECT_EQ(ToString(storage, NOT(LITERAL(false))), "(Not false)");
 
   // +1
-  EXPECT_EQ(ToString(UPLUS(LITERAL(1))), "(+ 1)");
+  EXPECT_EQ(ToString(storage, UPLUS(LITERAL(1))), "(+ 1)");
 
   // -1
-  EXPECT_EQ(ToString(UMINUS(LITERAL(1))), "(- 1)");
+  EXPECT_EQ(ToString(storage, UMINUS(LITERAL(1))), "(- 1)");
 
   // null IS NULL
-  EXPECT_EQ(ToString(IS_NULL(LITERAL(TypedValue::Null))), "(IsNull null)");
+  EXPECT_EQ(ToString(storage, IS_NULL(LITERAL(TypedValue::Null))),
+            "(IsNull null)");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, BinaryOperators) {
   // and(null, 5)
-  EXPECT_EQ(ToString(AND(LITERAL(TypedValue::Null), LITERAL(5))),
+  EXPECT_EQ(ToString(storage, AND(LITERAL(TypedValue::Null), LITERAL(5))),
             "(And null 5)");
 
   // or(5, {hello: "there"}["hello"])
-  EXPECT_EQ(
-      ToString(OR(LITERAL(5),
-                  PROPERTY_LOOKUP(MAP(std::make_pair(PROPERTY_PAIR("hello"),
-                                                     LITERAL("there"))),
-                                  PROPERTY_PAIR("hello")))),
-      "(Or 5 (PropertyLookup {\"hello\": \"there\"} \"hello\"))");
+  EXPECT_EQ(ToString(storage,
+                     OR(LITERAL(5),
+                        PROPERTY_LOOKUP(
+                            MAP(std::make_pair(storage.GetPropertyIx("hello"),
+                                               LITERAL("there"))),
+                            "hello"))),
+            "(Or 5 (PropertyLookup {\"hello\": \"there\"} \"hello\"))");
 
   // and(coalesce(null, 1), {hello: "there"})
-  EXPECT_EQ(ToString(AND(
-                COALESCE(LITERAL(TypedValue::Null), LITERAL(1)),
-                MAP(std::make_pair(PROPERTY_PAIR("hello"), LITERAL("there"))))),
-            "(And (Coalesce [null, 1]) {\"hello\": \"there\"})");
+  EXPECT_EQ(
+      ToString(storage, AND(COALESCE(LITERAL(TypedValue::Null), LITERAL(1)),
+                            MAP(std::make_pair(storage.GetPropertyIx("hello"),
+                                               LITERAL("there"))))),
+      "(And (Coalesce [null, 1]) {\"hello\": \"there\"})");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, Coalesce) {
   // coalesce()
-  EXPECT_EQ(ToString(COALESCE()), "(Coalesce [])");
+  EXPECT_EQ(ToString(storage, COALESCE()), "(Coalesce [])");
 
   // coalesce(null, null)
-  EXPECT_EQ(
-      ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null))),
-      "(Coalesce [null, null])");
+  EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null),
+                                       LITERAL(TypedValue::Null))),
+            "(Coalesce [null, null])");
 
   // coalesce(null, 2, 3)
-  EXPECT_EQ(
-      ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3))),
-      "(Coalesce [null, 2, 3])");
+  EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
+                                       LITERAL(3))),
+            "(Coalesce [null, 2, 3])");
 
   // coalesce(null, 2, assert(false), 3)
-  EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
-                              FN("ASSERT", LITERAL(false)), LITERAL(3))),
-            "(Coalesce [null, 2, (Function \"ASSERT\" [false]), 3])");
+  EXPECT_EQ(
+      ToString(storage, COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
+                                 FN("ASSERT", LITERAL(false)), LITERAL(3))),
+      "(Coalesce [null, 2, (Function \"ASSERT\" [false]), 3])");
 
   // coalesce(null, assert(false))
-  EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue::Null),
-                              FN("ASSERT", LITERAL(false)))),
+  EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null),
+                                       FN("ASSERT", LITERAL(false)))),
             "(Coalesce [null, (Function \"ASSERT\" [false])])");
 
   // coalesce([null, null])
-  EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue(
-                std::vector<TypedValue>{TypedValue::Null, TypedValue::Null})))),
-            "(Coalesce [[null, null]])");
+  EXPECT_EQ(
+      ToString(storage, COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
+                            TypedValue::Null, TypedValue::Null})))),
+      "(Coalesce [[null, null]])");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, ParameterLookup) {
   // and($hello, $there)
-  EXPECT_EQ(ToString(AND(PARAMETER_LOOKUP(1), PARAMETER_LOOKUP(2))),
+  EXPECT_EQ(ToString(storage, AND(PARAMETER_LOOKUP(1), PARAMETER_LOOKUP(2))),
             "(And (ParameterLookup 1) (ParameterLookup 2))");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, PropertyLookup) {
   // {hello: "there"}["hello"]
-  EXPECT_EQ(ToString(PROPERTY_LOOKUP(
-                MAP(std::make_pair(PROPERTY_PAIR("hello"), LITERAL("there"))),
-                PROPERTY_PAIR("hello"))),
-            "(PropertyLookup {\"hello\": \"there\"} \"hello\")");
+  EXPECT_EQ(
+      ToString(storage, PROPERTY_LOOKUP(
+                            MAP(std::make_pair(storage.GetPropertyIx("hello"),
+                                               LITERAL("there"))),
+                            "hello")),
+      "(PropertyLookup {\"hello\": \"there\"} \"hello\")");
 }
 
 TEST_F(ExpressionPrettyPrinterTest, NamedExpression) {
   // n AS 1
-  EXPECT_EQ(ToString(NEXPR("n", LITERAL(1))), "(NamedExpression \"n\" 1)");
+  EXPECT_EQ(ToString(storage, NEXPR("n", LITERAL(1))),
+            "(NamedExpression \"n\" 1)");
 }
 
 }  // namespace
diff --git a/tests/unit/query_required_privileges.cpp b/tests/unit/query_required_privileges.cpp
index 83624ee06..54ce27703 100644
--- a/tests/unit/query_required_privileges.cpp
+++ b/tests/unit/query_required_privileges.cpp
@@ -10,10 +10,10 @@ using namespace query;
 
 class FakeDbAccessor {};
 
-storage::EdgeType EDGE_TYPE(0);
-storage::Label LABEL_0(0);
-storage::Label LABEL_1(1);
-storage::Property PROP_0(0);
+const std::string EDGE_TYPE = "0";
+const std::string LABEL_0 = "label0";
+const std::string LABEL_1 = "label1";
+const std::string PROP_0 = "prop0";
 
 using ::testing::UnorderedElementsAre;
 
@@ -65,7 +65,8 @@ TEST_F(TestPrivilegeExtractor, MatchNodeSetLabels) {
 TEST_F(TestPrivilegeExtractor, MatchNodeSetProperty) {
   auto *query = QUERY(
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
-                   SET(PROPERTY_LOOKUP("n", {"prop", PROP_0}), LITERAL(42))));
+                   SET(PROPERTY_LOOKUP(storage.Create<Identifier>("n"), PROP_0),
+                       LITERAL(42))));
   EXPECT_THAT(GetRequiredPrivileges(query),
               UnorderedElementsAre(AuthQuery::Privilege::MATCH,
                                    AuthQuery::Privilege::SET));
@@ -88,16 +89,17 @@ TEST_F(TestPrivilegeExtractor, MatchNodeRemoveLabels) {
 }
 
 TEST_F(TestPrivilegeExtractor, MatchNodeRemoveProperty) {
-  auto *query =
-      QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
-                         REMOVE(PROPERTY_LOOKUP("n", {"prop", PROP_0}))));
+  auto *query = QUERY(SINGLE_QUERY(
+      MATCH(PATTERN(NODE("n"))),
+      REMOVE(PROPERTY_LOOKUP(storage.Create<Identifier>("n"), PROP_0))));
   EXPECT_THAT(GetRequiredPrivileges(query),
               UnorderedElementsAre(AuthQuery::Privilege::MATCH,
                                    AuthQuery::Privilege::REMOVE));
 }
 
 TEST_F(TestPrivilegeExtractor, CreateIndex) {
-  auto *query = CREATE_INDEX_ON(LABEL_0, PROP_0);
+  auto *query = CREATE_INDEX_ON(storage.GetLabelIx(LABEL_0),
+                                storage.GetPropertyIx(PROP_0));
   EXPECT_THAT(GetRequiredPrivileges(query),
               UnorderedElementsAre(AuthQuery::Privilege::INDEX));
 }
diff --git a/tests/unit/query_semantic.cpp b/tests/unit/query_semantic.cpp
index 9e9e57a38..433155d10 100644
--- a/tests/unit/query_semantic.cpp
+++ b/tests/unit/query_semantic.cpp
@@ -79,7 +79,7 @@ TEST_F(TestSymbolGenerator, MatchNodeUnboundReturn) {
 TEST_F(TestSymbolGenerator, CreatePropertyUnbound) {
   // AST with unbound variable in create: CREATE ({prop: x})
   auto node = NODE("anon");
-  node->properties_[PROPERTY_PAIR("prop")] = IDENT("x");
+  node->properties_[storage.GetPropertyIx("prop")] = IDENT("x");
   auto query_ast = QUERY(SINGLE_QUERY(CREATE(PATTERN(node))));
   EXPECT_THROW(query::MakeSymbolTable(query_ast), UnboundVariableError);
 }
@@ -130,7 +130,7 @@ TEST_F(TestSymbolGenerator, MatchCreateRedeclareNode) {
 TEST_F(TestSymbolGenerator, MatchCreateRedeclareEdge) {
   // AST with redeclaring a match edge variable in create:
   // MATCH (n) -[r]- (m) CREATE (n) -[r :relationship]-> (l)
-  auto relationship = dba.EdgeType("relationship");
+  auto relationship = "relationship";
   auto query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
       CREATE(PATTERN(NODE("n"),
@@ -160,10 +160,10 @@ TEST_F(TestSymbolGenerator, MatchCreateTypeMismatch) {
 TEST_F(TestSymbolGenerator, CreateMultipleEdgeType) {
   // Multiple edge relationship are not allowed when creating edges.
   // CREATE (n) -[r :rel1 | :rel2]-> (m)
-  auto rel1 = dba.EdgeType("rel1");
-  auto rel2 = dba.EdgeType("rel2");
+  auto rel1 = "rel1";
+  auto rel2 = "rel2";
   auto edge = EDGE("r", EdgeAtom::Direction::OUT, {rel1});
-  edge->edge_types_.emplace_back(rel2);
+  edge->edge_types_.emplace_back(storage.GetEdgeTypeIx(rel2));
   auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"), edge, NODE("m")))));
   EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
 }
@@ -171,7 +171,7 @@ TEST_F(TestSymbolGenerator, CreateMultipleEdgeType) {
 TEST_F(TestSymbolGenerator, CreateBidirectionalEdge) {
   // Bidirectional relationships are not allowed when creating edges.
   // CREATE (n) -[r :rel1]- (m)
-  auto rel1 = dba.EdgeType("rel1");
+  auto rel1 = "rel1";
   auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
       NODE("n"), EDGE("r", EdgeAtom::Direction::BOTH, {rel1}), NODE("m")))));
   EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
@@ -270,8 +270,8 @@ TEST_F(TestSymbolGenerator, MatchWithWhereUnbound) {
 
 TEST_F(TestSymbolGenerator, CreateMultiExpand) {
   // Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
-  auto r_type = dba.EdgeType("r");
-  auto p_type = dba.EdgeType("p");
+  auto r_type = "r";
+  auto p_type = "p";
   auto node_n1 = NODE("n");
   auto edge_r = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
   auto node_m = NODE("m");
@@ -303,8 +303,8 @@ TEST_F(TestSymbolGenerator, CreateMultiExpand) {
 
 TEST_F(TestSymbolGenerator, MatchCreateExpandLabel) {
   // Test MATCH (n) CREATE (m) -[r :r]-> (n:label)
-  auto r_type = dba.EdgeType("r");
-  auto label = dba.Label("label");
+  auto r_type = "r";
+  auto label = "label";
   auto query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))),
       CREATE(PATTERN(NODE("m"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}),
@@ -314,9 +314,9 @@ TEST_F(TestSymbolGenerator, MatchCreateExpandLabel) {
 
 TEST_F(TestSymbolGenerator, CreateExpandProperty) {
   // Test CREATE (n) -[r :r]-> (n {prop: 42})
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   auto n_prop = NODE("n");
-  n_prop->properties_[PROPERTY_PAIR("prop")] = LITERAL(42);
+  n_prop->properties_[storage.GetPropertyIx("prop")] = LITERAL(42);
   auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
       NODE("n"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}), n_prop))));
   EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
@@ -365,7 +365,7 @@ TEST_F(TestSymbolGenerator, MatchPropCreateNodeProp) {
   auto node_n = NODE("n");
   auto node_m = NODE("m");
   auto n_prop = PROPERTY_LOOKUP("n", prop.second);
-  node_m->properties_[prop] = n_prop;
+  node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
   auto query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), CREATE(PATTERN(node_m))));
   auto symbol_table = query::MakeSymbolTable(query);
@@ -379,7 +379,7 @@ TEST_F(TestSymbolGenerator, MatchPropCreateNodeProp) {
 
 TEST_F(TestSymbolGenerator, CreateNodeEdge) {
   // Test CREATE (n), (n) -[r :r]-> (n)
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   auto node_1 = NODE("n");
   auto node_2 = NODE("n");
   auto edge = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
@@ -397,7 +397,7 @@ TEST_F(TestSymbolGenerator, CreateNodeEdge) {
 
 TEST_F(TestSymbolGenerator, MatchWithCreate) {
   // Test MATCH (n) WITH n AS m CREATE (m) -[r :r]-> (m)
-  auto r_type = dba.EdgeType("r");
+  auto r_type = "r";
   auto node_1 = NODE("n");
   auto node_2 = NODE("m");
   auto edge = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
@@ -512,7 +512,7 @@ TEST_F(TestSymbolGenerator, MergeVariableError) {
 
 TEST_F(TestSymbolGenerator, MergeVariableErrorEdge) {
   // Test MATCH (n) -[r]- (m) MERGE (a) -[r :rel]- (b)
-  auto rel = dba.EdgeType("rel");
+  auto rel = "rel";
   auto query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
       MERGE(PATTERN(NODE("a"), EDGE("r", EdgeAtom::Direction::BOTH, {rel}),
@@ -531,7 +531,7 @@ TEST_F(TestSymbolGenerator, MergeEdgeWithoutType) {
 TEST_F(TestSymbolGenerator, MergeOnMatchOnCreate) {
   // Test MATCH (n) MERGE (n) -[r :rel]- (m) ON MATCH SET n.prop = 42
   //      ON CREATE SET m.prop = 42 RETURN r AS r
-  auto rel = dba.EdgeType("rel");
+  auto rel = "rel";
   auto prop = dba.Property("prop");
   auto match_n = NODE("n");
   auto merge_n = NODE("n");
@@ -600,10 +600,10 @@ TEST_F(TestSymbolGenerator, MatchCrossReferenceVariable) {
   auto prop = PROPERTY_PAIR("prop");
   auto node_n = NODE("n");
   auto m_prop = PROPERTY_LOOKUP("m", prop.second);
-  node_n->properties_[prop] = m_prop;
+  node_n->properties_[storage.GetPropertyIx(prop.first)] = m_prop;
   auto node_m = NODE("m");
   auto n_prop = PROPERTY_LOOKUP("n", prop.second);
-  node_m->properties_[prop] = n_prop;
+  node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
   auto ident_n = IDENT("n");
   auto as_n = AS("n");
   auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n), PATTERN(node_m)),
@@ -661,8 +661,8 @@ TEST_F(TestSymbolGenerator, MatchReturnAsteriskNoUserVariables) {
 
 TEST_F(TestSymbolGenerator, MatchMergeExpandLabel) {
   // Test MATCH (n) MERGE (m) -[r :r]-> (n:label)
-  auto r_type = dba.EdgeType("r");
-  auto label = dba.Label("label");
+  auto r_type = "r";
+  auto label = "label";
   auto query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"))),
       MERGE(PATTERN(NODE("m"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}),
@@ -675,7 +675,7 @@ TEST_F(TestSymbolGenerator, MatchEdgeWithIdentifierInProperty) {
   auto prop = PROPERTY_PAIR("prop");
   auto edge = EDGE("r");
   auto n_prop = PROPERTY_LOOKUP("n", prop.second);
-  edge->properties_[prop] = n_prop;
+  edge->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
   auto node_n = NODE("n");
   auto query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n, edge, NODE("m"))), RETURN("r")));
@@ -768,7 +768,7 @@ TEST_F(TestSymbolGenerator, MatchPropertySameIdentifier) {
   auto prop = PROPERTY_PAIR("prop");
   auto node_n = NODE("n");
   auto n_prop = PROPERTY_LOOKUP("n", prop.second);
-  node_n->properties_[prop] = n_prop;
+  node_n->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
   auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), RETURN("n")));
   auto symbol_table = query::MakeSymbolTable(query);
   auto n = symbol_table.at(*node_n->identifier_);
@@ -885,7 +885,7 @@ TEST_F(TestSymbolGenerator, MatchBfsReturn) {
   auto *n_prop = PROPERTY_LOOKUP("n", prop);
   auto *bfs = storage.Create<EdgeAtom>(
       IDENT("r"), EdgeAtom::Type::BREADTH_FIRST, EdgeAtom::Direction::OUT,
-      std::vector<storage::EdgeType>{});
+      std::vector<EdgeTypeIx>{});
   bfs->filter_lambda_.inner_edge = IDENT("r");
   bfs->filter_lambda_.inner_node = IDENT("n");
   bfs->filter_lambda_.expression = r_prop;
@@ -991,7 +991,7 @@ TEST_F(TestSymbolGenerator, MatchWShortestReturn) {
   auto *r_filter = PROPERTY_LOOKUP("r", filter);
   auto *shortest = storage.Create<EdgeAtom>(
       IDENT("r"), EdgeAtom::Type::WEIGHTED_SHORTEST_PATH,
-      EdgeAtom::Direction::OUT, std::vector<storage::EdgeType>{});
+      EdgeAtom::Direction::OUT, std::vector<EdgeTypeIx>{});
   {
     shortest->weight_lambda_.inner_edge = IDENT("r");
     shortest->weight_lambda_.inner_node = IDENT("n");
diff --git a/tests/unit/query_variable_start_planner.cpp b/tests/unit/query_variable_start_planner.cpp
index aa5bcd1ec..536d2a8ff 100644
--- a/tests/unit/query_variable_start_planner.cpp
+++ b/tests/unit/query_variable_start_planner.cpp
@@ -58,21 +58,22 @@ void AssertRows(const std::vector<std::vector<TypedValue>> &datum,
 
 void CheckPlansProduce(
     size_t expected_plan_count, query::CypherQuery *query, AstStorage &storage,
-    database::GraphDbAccessor &dba,
+    database::GraphDbAccessor *dba,
     std::function<void(const std::vector<std::vector<TypedValue>> &)> check) {
   auto symbol_table = query::MakeSymbolTable(query);
   auto planning_context =
-      MakePlanningContext(storage, symbol_table, query, dba);
+      MakePlanningContext(&storage, &symbol_table, query, dba);
   auto query_parts = CollectQueryParts(symbol_table, storage, query);
   EXPECT_TRUE(query_parts.query_parts.size() > 0);
   auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
   auto plans = MakeLogicalPlanForSingleQuery<VariableStartPlanner>(
-      single_query_parts, planning_context);
+      single_query_parts, &planning_context);
   EXPECT_EQ(std::distance(plans.begin(), plans.end()), expected_plan_count);
   for (const auto &plan : plans) {
     auto *produce = dynamic_cast<Produce *>(plan.get());
     ASSERT_TRUE(produce);
-    auto results = CollectProduce(produce, symbol_table, dba);
+    Context context = MakeContext(storage, symbol_table, dba);
+    auto results = CollectProduce(*produce, &context);
     check(results);
   }
 }
@@ -91,7 +92,7 @@ TEST(TestVariableStartPlanner, MatchReturn) {
       MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))),
       RETURN("n")));
   // We have 2 nodes `n` and `m` from which we could start, so expect 2 plans.
-  CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
     // We expect to produce only a single (v1) node.
     AssertRows(results, {{v1}});
   });
@@ -115,7 +116,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
                       EDGE("e", Direction::OUT), NODE("l"))),
         RETURN("n")));
     // We have 3 nodes: `n`, `m` and `l` from which we could start.
-    CheckPlansProduce(3, query, storage, *dba, [&](const auto &results) {
+    CheckPlansProduce(3, query, storage, dba.get(), [&](const auto &results) {
       // We expect to produce only a single (v1) node.
       AssertRows(results, {{v1}});
     });
@@ -127,7 +128,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
         MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m")),
               PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))),
         RETURN("n")));
-    CheckPlansProduce(3, query, storage, *dba, [&](const auto &results) {
+    CheckPlansProduce(3, query, storage, dba.get(), [&](const auto &results) {
       AssertRows(results, {{v1}});
     });
   }
@@ -151,7 +152,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
       RETURN("n", "l")));
   // We have 2 nodes `n` and `m` from which we could start the MATCH, and 2
   // nodes for OPTIONAL MATCH. This should produce 2 * 2 plans.
-  CheckPlansProduce(4, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(4, query, storage, dba.get(), [&](const auto &results) {
     // We expect to produce 2 rows:
     //   * (v1), (v3)
     //   * (v2), null
@@ -165,7 +166,8 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
   // Graph (v1) -[:r]-> (v2)
   auto v1 = dba->InsertVertex();
   auto v2 = dba->InsertVertex();
-  auto r_type = dba->EdgeType("r");
+  auto r_type_name = "r";
+  auto r_type = dba->EdgeType(r_type_name);
   dba->InsertEdge(v1, v2, r_type);
   dba->AdvanceCommand();
   // Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l)
@@ -174,11 +176,12 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
   auto *query = QUERY(SINGLE_QUERY(
       MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))),
       OPTIONAL_MATCH(PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))),
-      MERGE(PATTERN(NODE("u"), EDGE("q", Direction::OUT, {r_type}), NODE("v"))),
+      MERGE(PATTERN(NODE("u"), EDGE("q", Direction::OUT, {r_type_name}),
+                    NODE("v"))),
       RETURN("n", "m", "l", "u", "v")));
   // Since MATCH, OPTIONAL MATCH and MERGE each have 2 nodes from which we can
   // start, we generate 2 * 2 * 2 plans.
-  CheckPlansProduce(8, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(8, query, storage, dba.get(), [&](const auto &results) {
     // We expect to produce a single row: (v1), (v2), null, (v1), (v2)
     AssertRows(results, {{v1, v2, TypedValue::Null, v1, v2}});
   });
@@ -201,7 +204,7 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
       RETURN("n", "m", "l")));
   // We can start from 2 nodes in each match. Since WITH separates query parts,
   // we expect to get 2 plans for each, which totals 2 * 2.
-  CheckPlansProduce(4, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(4, query, storage, dba.get(), [&](const auto &results) {
     // We expect to produce a single row: (v1), (v1), (v2)
     AssertRows(results, {{v1, v1, v2}});
   });
@@ -226,7 +229,7 @@ TEST(TestVariableStartPlanner, MatchVariableExpand) {
   TypedValue r1_list(std::vector<TypedValue>{r1});         // [r1]
   TypedValue r2_list(std::vector<TypedValue>{r2});         // [r2]
   TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2});  // [r1, r2]
-  CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
     AssertRows(results, {{r1_list}, {r2_list}, {r1_r2_list}});
   });
 }
@@ -255,7 +258,7 @@ TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
   // We expect to get a single column with the following rows:
   TypedValue r1_list(std::vector<TypedValue>{r1});  // [r1] (v1 -[*..1]-> v2)
   TypedValue r2_list(std::vector<TypedValue>{r2});  // [r2] (v2 -[*..2]-> v3)
-  CheckPlansProduce(2, query, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(2, query, storage, &dba, [&](const auto &results) {
     AssertRows(results, {{r1_list}, {r2_list}});
   });
 }
@@ -276,13 +279,13 @@ TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
   AstStorage storage;
   auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH);
   auto node_n = NODE("n");
-  node_n->properties_[std::make_pair("id", id)] = LITERAL(1);
+  node_n->properties_[storage.GetPropertyIx("id")] = LITERAL(1);
   auto *query =
       QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n, edge, NODE("m"))), RETURN("r")));
   // We expect to get a single column with the following rows:
   TypedValue r1_list(std::vector<TypedValue>{r1});         // [r1]
   TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2});  // [r1, r2]
-  CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
+  CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
     AssertRows(results, {{r1_list}, {r1_r2_list}});
   });
 }
@@ -306,7 +309,7 @@ TEST(TestVariableStartPlanner, MatchBfs) {
   AstStorage storage;
   auto *bfs = storage.Create<query::EdgeAtom>(
       IDENT("r"), EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
-      std::vector<storage::EdgeType>{});
+      std::vector<query::EdgeTypeIx>{});
   bfs->filter_lambda_.inner_edge = IDENT("r");
   bfs->filter_lambda_.inner_node = IDENT("n");
   bfs->filter_lambda_.expression = NEQ(PROPERTY_LOOKUP("n", id), LITERAL(3));
@@ -315,7 +318,7 @@ TEST(TestVariableStartPlanner, MatchBfs) {
       SINGLE_QUERY(MATCH(PATTERN(NODE("n"), bfs, NODE("m"))), RETURN("r")));
   // We expect to get a single column with the following rows:
   TypedValue r1_list(std::vector<TypedValue>{r1});  // [r1]
-  CheckPlansProduce(2, query, storage, dba, [&](const auto &results) {
+  CheckPlansProduce(2, query, storage, &dba, [&](const auto &results) {
     AssertRows(results, {{r1_list}});
   });
 }