diff --git a/src/query/plan/distributed_ops.cpp b/src/query/plan/distributed_ops.cpp
index 36e9e7e1b..8ad094a36 100644
--- a/src/query/plan/distributed_ops.cpp
+++ b/src/query/plan/distributed_ops.cpp
@@ -115,8 +115,8 @@ DistributedExpand::DistributedExpand(
     GraphView graph_view)
     : input_(input ? input : std::make_shared<Once>()),
       input_symbol_(input_symbol),
-      common_{node_symbol, edge_symbol,   direction,
-              edge_types,  existing_node, graph_view} {}
+      common_{node_symbol, edge_symbol, direction, edge_types, existing_node},
+      graph_view_(graph_view) {}
 
 DistributedExpand::DistributedExpand(
     const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
@@ -139,12 +139,11 @@ DistributedExpandBfs::DistributedExpandBfs(
     const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
     Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Direction direction,
     const std::vector<storage::EdgeType> &edge_types, bool existing_node,
-    GraphView graph_view, Expression *lower_bound, Expression *upper_bound,
+    Expression *lower_bound, Expression *upper_bound,
     const ExpansionLambda &filter_lambda)
     : input_(input ? input : std::make_shared<Once>()),
       input_symbol_(input_symbol),
-      common_{node_symbol, edge_symbol,   direction,
-              edge_types,  existing_node, graph_view},
+      common_{node_symbol, edge_symbol, direction, edge_types, existing_node},
       lower_bound_(lower_bound),
       upper_bound_(upper_bound),
       filter_lambda_(filter_lambda) {}
@@ -979,7 +978,7 @@ class DistributedExpandCursor : public query::plan::Cursor {
 
       ExpectType(self_->input_symbol_, vertex_value, TypedValue::Type::Vertex);
       auto &vertex = vertex_value.Value<VertexAccessor>();
-      SwitchAccessor(vertex, self_->common_.graph_view);
+      SwitchAccessor(vertex, self_->graph_view_);
 
       auto direction = self_->common_.direction;
       if (direction == EdgeAtom::Direction::IN ||
@@ -1058,10 +1057,7 @@ class DistributedExpandBfsCursor : public query::plan::Cursor {
  public:
   DistributedExpandBfsCursor(const DistributedExpandBfs &self,
                              database::GraphDbAccessor &db)
-      : self_(self), db_(db), input_cursor_(self_.input()->MakeCursor(db)) {
-    CHECK(self_.common_.graph_view == GraphView::OLD)
-        << "ExpandVariable should only be planned with GraphView::OLD";
-  }
+      : self_(self), db_(db), input_cursor_(self_.input()->MakeCursor(db)) {}
 
   void InitSubcursors(database::GraphDbAccessor *dba,
                       const query::SymbolTable &symbol_table,
@@ -1090,7 +1086,7 @@ class DistributedExpandBfsCursor : public query::plan::Cursor {
     // Evaluator for the filtering condition and expansion depth.
     ExpressionEvaluator evaluator(
         &frame, context.symbol_table, context.evaluation_context,
-        context.db_accessor, self_.common_.graph_view);
+        context.db_accessor, GraphView::OLD);
 
     while (true) {
       if (context.db_accessor->should_abort()) throw HintedAbortError();
diff --git a/src/query/plan/distributed_ops.lcp b/src/query/plan/distributed_ops.lcp
index a240137ac..d7a49c88e 100644
--- a/src/query/plan/distributed_ops.lcp
+++ b/src/query/plan/distributed_ops.lcp
@@ -230,7 +230,17 @@ by having only one result from each worker.")
           :capnp-save #'save-operator-pointer
           :capnp-load #'load-operator-pointer)
    (input-symbol "Symbol" :scope :public)
-   (common "ExpandCommon" :scope :public))
+   (common "ExpandCommon" :scope :public)
+   (graph-view "GraphView" :scope :public
+               :capnp-init nil
+               :capnp-save (lcp:capnp-save-enum "::query::capnp::GraphView"
+                                                "GraphView"
+                                                '(old new))
+               :capnp-load (lcp:capnp-load-enum "::query::capnp::GraphView"
+                                                "GraphView"
+                                                '(old new))
+               :documentation
+               "State from which the input node should get expanded."))
   (:documentation "Distributed version of Expand operator")
   (:public
    #>cpp
@@ -306,8 +316,8 @@ by having only one result from each worker.")
                         Symbol input_symbol, Symbol node_symbol,
                         Symbol edge_symbol, EdgeAtom::Direction direction,
                         const std::vector<storage::EdgeType> &edge_types,
-                        bool existing_node, GraphView graph_view,
-                        Expression *lower_bound, Expression *upper_bound,
+                        bool existing_node, Expression *lower_bound,
+                        Expression *upper_bound,
                         const ExpansionLambda &filter_lambda);
    DistributedExpandBfs(const std::shared_ptr<LogicalOperator> &input,
                         Symbol input_symbol, const ExpandCommon &common,
diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp
index cdd80abcb..311a3e88a 100644
--- a/src/query/plan/operator.cpp
+++ b/src/query/plan/operator.cpp
@@ -454,8 +454,8 @@ Expand::Expand(const std::shared_ptr<LogicalOperator> &input,
                bool existing_node, GraphView graph_view)
     : input_(input ? input : std::make_shared<Once>()),
       input_symbol_(input_symbol),
-      common_{node_symbol, edge_symbol,   direction,
-              edge_types,  existing_node, graph_view} {}
+      common_{node_symbol, edge_symbol, direction, edge_types, existing_node},
+      graph_view_(graph_view) {}
 
 ACCEPT_WITH_INPUT(Expand)
 
@@ -548,7 +548,7 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame, ExecutionContext &context) {
 
     ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
     auto &vertex = vertex_value.Value<VertexAccessor>();
-    SwitchAccessor(vertex, self_.common_.graph_view);
+    SwitchAccessor(vertex, self_.graph_view_);
 
     auto direction = self_.common_.direction;
     if (direction == EdgeAtom::Direction::IN ||
@@ -601,11 +601,10 @@ ExpandVariable::ExpandVariable(
     Expression *lower_bound, Expression *upper_bound, bool existing_node,
     ExpansionLambda filter_lambda,
     std::experimental::optional<ExpansionLambda> weight_lambda,
-    std::experimental::optional<Symbol> total_weight, GraphView graph_view)
+    std::experimental::optional<Symbol> total_weight)
     : input_(input ? input : std::make_shared<Once>()),
       input_symbol_(input_symbol),
-      common_{node_symbol, edge_symbol,   direction,
-              edge_types,  existing_node, graph_view},
+      common_{node_symbol, edge_symbol, direction, edge_types, existing_node},
       type_(type),
       is_reverse_(is_reverse),
       lower_bound_(lower_bound),
@@ -687,9 +686,9 @@ class ExpandVariableCursor : public Cursor {
   bool Pull(Frame &frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("ExpandVariable");
 
-    ExpressionEvaluator evaluator(
-        &frame, context.symbol_table, context.evaluation_context,
-        context.db_accessor, self_.common_.graph_view);
+    ExpressionEvaluator evaluator(&frame, context.symbol_table,
+                                  context.evaluation_context,
+                                  context.db_accessor, GraphView::OLD);
     while (true) {
       if (Expand(frame, context)) return true;
 
@@ -764,14 +763,12 @@ class ExpandVariableCursor : public Cursor {
 
       ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
       auto &vertex = vertex_value.Value<VertexAccessor>();
-      SwitchAccessor(vertex, self_.common_.graph_view);
+      SwitchAccessor(vertex, GraphView::OLD);
 
       // Evaluate the upper and lower bounds.
       ExpressionEvaluator evaluator(&frame, context.symbol_table,
-
                                     context.evaluation_context,
-                                    context.db_accessor,
-                                    self_.common_.graph_view);
+                                    context.db_accessor, GraphView::OLD);
       auto calc_bound = [&evaluator](auto &bound) {
         auto value = EvaluateInt(&evaluator, bound, "Variable expansion bound");
         if (value < 0)
@@ -785,7 +782,7 @@ class ExpandVariableCursor : public Cursor {
                                         : std::numeric_limits<int64_t>::max();
 
       if (upper_bound_ > 0) {
-        SwitchAccessor(vertex, self_.common_.graph_view);
+        SwitchAccessor(vertex, GraphView::OLD);
         edges_.emplace_back(ExpandFromVertex(vertex, self_.common_.direction,
                                              self_.common_.edge_types));
         edges_it_.emplace_back(edges_.back().begin());
@@ -830,9 +827,9 @@ class ExpandVariableCursor : public Cursor {
    * vertex and another Pull from the input cursor should be performed.
    */
   bool Expand(Frame &frame, ExecutionContext &context) {
-    ExpressionEvaluator evaluator(
-        &frame, context.symbol_table, context.evaluation_context,
-        context.db_accessor, self_.common_.graph_view);
+    ExpressionEvaluator evaluator(&frame, context.symbol_table,
+                                  context.evaluation_context,
+                                  context.db_accessor, GraphView::OLD);
     // Some expansions might not be valid due to edge uniqueness and
     // existing_node criterions, so expand in a loop until either the input
     // vertex is exhausted or a valid variable-length expansion is available.
@@ -902,7 +899,7 @@ class ExpandVariableCursor : public Cursor {
       // we are doing depth-first search, so place the current
       // edge's expansions onto the stack, if we should continue to expand
       if (upper_bound_ > static_cast<int64_t>(edges_.size())) {
-        SwitchAccessor(current_vertex, self_.common_.graph_view);
+        SwitchAccessor(current_vertex, GraphView::OLD);
         edges_.emplace_back(ExpandFromVertex(
             current_vertex, self_.common_.direction, self_.common_.edge_types));
         edges_it_.emplace_back(edges_.back().begin());
@@ -922,8 +919,6 @@ class STShortestPathCursor : public query::plan::Cursor {
   STShortestPathCursor(const ExpandVariable &self,
                        database::GraphDbAccessor &dba)
       : self_(self), input_cursor_(self_.input()->MakeCursor(dba)) {
-    CHECK(self_.common_.graph_view == GraphView::OLD)
-        << "ExpandVariable should only be planned with GraphView::OLD";
     CHECK(self_.common_.existing_node)
         << "s-t shortest path algorithm should only "
            "be used when `existing_node` flag is "
@@ -1157,8 +1152,6 @@ class SingleSourceShortestPathCursor : public query::plan::Cursor {
   SingleSourceShortestPathCursor(const ExpandVariable &self,
                                  database::GraphDbAccessor &db)
       : self_(self), input_cursor_(self_.input()->MakeCursor(db)) {
-    CHECK(self_.common_.graph_view == GraphView::OLD)
-        << "ExpandVariable should only be planned with GraphView::OLD";
     CHECK(!self_.common_.existing_node)
         << "Single source shortest path algorithm "
            "should not be used when `existing_node` "
@@ -1325,7 +1318,7 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
 
     ExpressionEvaluator evaluator(
         &frame, context.symbol_table, context.evaluation_context,
-        context.db_accessor, self_.common_.graph_view);
+        context.db_accessor, GraphView::OLD);
     auto create_state = [this](VertexAccessor vertex, int depth) {
       return std::make_pair(vertex, upper_bound_set_ ? depth : 0);
     };
@@ -1336,8 +1329,8 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
     auto expand_pair = [this, &evaluator, &frame, &create_state](
                            EdgeAccessor edge, VertexAccessor vertex,
                            double weight, int depth) {
-      SwitchAccessor(edge, self_.common_.graph_view);
-      SwitchAccessor(vertex, self_.common_.graph_view);
+      SwitchAccessor(edge, GraphView::OLD);
+      SwitchAccessor(vertex, GraphView::OLD);
 
       if (self_.filter_lambda_.expression) {
         frame[self_.filter_lambda_.inner_edge_symbol] = edge;
@@ -1400,7 +1393,7 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
           // Skip expansion for such nodes.
           if (node.IsNull()) continue;
         }
-        SwitchAccessor(vertex, self_.common_.graph_view);
+        SwitchAccessor(vertex, GraphView::OLD);
         if (self_.upper_bound_) {
           upper_bound_ =
               EvaluateInt(&evaluator, self_.upper_bound_,
@@ -2179,9 +2172,8 @@ std::vector<Symbol> EdgeUniquenessFilter::ModifiedSymbols(
   return input_->ModifiedSymbols(table);
 }
 
-EdgeUniquenessFilter::EdgeUniquenessFilterCursor::
-    EdgeUniquenessFilterCursor(const EdgeUniquenessFilter &self,
-                                 database::GraphDbAccessor &db)
+EdgeUniquenessFilter::EdgeUniquenessFilterCursor::EdgeUniquenessFilterCursor(
+    const EdgeUniquenessFilter &self, database::GraphDbAccessor &db)
     : self_(self), input_cursor_(self.input_->MakeCursor(db)) {}
 
 namespace {
diff --git a/src/query/plan/operator.lcp b/src/query/plan/operator.lcp
index 999c3509f..3240dc6db 100644
--- a/src/query/plan/operator.lcp
+++ b/src/query/plan/operator.lcp
@@ -924,7 +924,17 @@ the given types are valid."
                :capnp-load (lcp:capnp-load-vector "::storage::capnp::EdgeType"
                                                   "storage::EdgeType"))
    (existing-node :bool :documentation "If the given node atom refer to a symbol
-that has already been expanded and should be just validated in the frame.")
+that has already been expanded and should be just validated in the frame."))
+  (:serialize (:slk) (:capnp)))
+
+(lcp:define-class expand (logical-operator)
+  ((input "std::shared_ptr<LogicalOperator>" :scope :public
+          :slk-save #'slk-save-operator-pointer
+          :slk-load #'slk-load-operator-pointer
+          :capnp-save #'save-operator-pointer
+          :capnp-load #'load-operator-pointer)
+   (input-symbol "Symbol" :scope :public)
+   (common "ExpandCommon" :scope :public)
    (graph-view "GraphView" :scope :public
                :capnp-init nil
                :capnp-save (lcp:capnp-save-enum "::query::capnp::GraphView"
@@ -935,16 +945,6 @@ that has already been expanded and should be just validated in the frame.")
                                                 '(old new))
                :documentation
                "State from which the input node should get expanded."))
-  (:serialize (:slk) (:capnp)))
-
-(lcp:define-class expand (logical-operator)
-  ((input "std::shared_ptr<LogicalOperator>" :scope :public
-          :slk-save #'slk-save-operator-pointer
-          :slk-load #'slk-load-operator-pointer
-          :capnp-save #'save-operator-pointer
-          :capnp-load #'load-operator-pointer)
-   (input-symbol "Symbol" :scope :public)
-   (common "ExpandCommon" :scope :public))
   (:documentation
    "Expansion operator. For a node existing in the frame it
 expands one edge and one node and places them on the frame.
@@ -1195,8 +1195,7 @@ pulled.")
                    Expression *upper_bound, bool existing_node,
                    ExpansionLambda filter_lambda,
                    std::experimental::optional<ExpansionLambda> weight_lambda,
-                   std::experimental::optional<Symbol> total_weight,
-                   GraphView graph_view);
+                   std::experimental::optional<Symbol> total_weight);
 
    bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
    std::unique_ptr<Cursor> MakeCursor(
diff --git a/src/query/plan/rule_based_planner.hpp b/src/query/plan/rule_based_planner.hpp
index 3a84f282e..219143950 100644
--- a/src/query/plan/rule_based_planner.hpp
+++ b/src/query/plan/rule_based_planner.hpp
@@ -594,12 +594,13 @@ class RuleBasedPlanner {
           }
 
           // TODO: Pass weight lambda.
+          CHECK(match_context.graph_view == GraphView::OLD)
+              << "ExpandVariable should only be planned with GraphView::OLD";
           last_op = std::make_unique<ExpandVariable>(
               std::move(last_op), node1_symbol, node_symbol, edge_symbol,
               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);
+              existing_node, filter_lambda, weight_lambda, total_weight);
         } else {
           if (!existing_node) {
             // Try to get better behaviour by creating an indexed scan and then
diff --git a/tests/unit/bfs_distributed.cpp b/tests/unit/bfs_distributed.cpp
index 4ca5b140c..7bee86a6c 100644
--- a/tests/unit/bfs_distributed.cpp
+++ b/tests/unit/bfs_distributed.cpp
@@ -30,7 +30,7 @@ class DistributedDb : public Database {
       const ExpansionLambda &filter_lambda) override {
     return std::make_unique<DistributedExpandBfs>(
         input, source_sym, sink_sym, edge_sym, direction, edge_types,
-        existing_node, GraphView::OLD, lower_bound, upper_bound, filter_lambda);
+        existing_node, lower_bound, upper_bound, filter_lambda);
   }
 
   std::pair<std::vector<storage::VertexAddress>,
diff --git a/tests/unit/bfs_single_node.cpp b/tests/unit/bfs_single_node.cpp
index a766deb26..71b29640f 100644
--- a/tests/unit/bfs_single_node.cpp
+++ b/tests/unit/bfs_single_node.cpp
@@ -26,8 +26,7 @@ class SingleNodeDb : public Database {
     return std::make_unique<ExpandVariable>(
         input, source_sym, sink_sym, edge_sym, EdgeAtom::Type::BREADTH_FIRST,
         direction, edge_types, false, lower_bound, upper_bound, existing_node,
-        filter_lambda, std::experimental::nullopt, std::experimental::nullopt,
-        GraphView::OLD);
+        filter_lambda, std::experimental::nullopt, std::experimental::nullopt);
   }
 
   std::pair<std::vector<VertexAddress>, std::vector<EdgeAddress>> BuildGraph(
diff --git a/tests/unit/query_cost_estimator.cpp b/tests/unit/query_cost_estimator.cpp
index 9d9ffde02..eb3fa5535 100644
--- a/tests/unit/query_cost_estimator.cpp
+++ b/tests/unit/query_cost_estimator.cpp
@@ -178,7 +178,7 @@ TEST_F(QueryCostEstimator, ExpandVariable) {
       EdgeAtom::Type::DEPTH_FIRST, EdgeAtom::Direction::IN,
       std::vector<storage::EdgeType>{}, false, nullptr, nullptr, false,
       ExpansionLambda{NextSymbol(), NextSymbol(), nullptr},
-      std::experimental::nullopt, std::experimental::nullopt, GraphView::OLD);
+      std::experimental::nullopt, std::experimental::nullopt);
   EXPECT_COST(CardParam::kExpandVariable * CostParam::kExpandVariable);
 }
 
diff --git a/tests/unit/query_plan_match_filter_return.cpp b/tests/unit/query_plan_match_filter_return.cpp
index a7e1be030..67e0e92db 100644
--- a/tests/unit/query_plan_match_filter_return.cpp
+++ b/tests/unit/query_plan_match_filter_return.cpp
@@ -567,6 +567,8 @@ class QueryPlanExpandVariable : public testing::Test {
       auto convert = [this](std::experimental::optional<size_t> bound) {
         return bound ? LITERAL(static_cast<int64_t>(bound.value())) : nullptr;
       };
+      CHECK(graph_view == GraphView::OLD)
+          << "ExpandVariable should only be planned with GraphView::OLD";
 
       return std::make_shared<ExpandVariable>(
           filter_op, n_from.sym_, n_to_sym, edge_sym,
@@ -575,7 +577,7 @@ class QueryPlanExpandVariable : public testing::Test {
           ExpansionLambda{symbol_table.CreateSymbol("inner_edge", false),
                           symbol_table.CreateSymbol("inner_node", false),
                           nullptr},
-          std::experimental::nullopt, std::experimental::nullopt, graph_view);
+          std::experimental::nullopt, std::experimental::nullopt);
     } else
       return std::make_shared<Expand>(filter_op, n_from.sym_, n_to_sym,
                                       edge_sym, direction, edge_types, false,
@@ -761,39 +763,6 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessTwoVariableExpansions) {
             (map_int{{2, 5 * 8}}));
 }
 
-TEST_F(QueryPlanExpandVariable, GraphState) {
-  auto test_expand = [&](GraphView graph_view,
-                         const std::vector<storage::EdgeType> &edge_types) {
-    auto e = Edge("r", EdgeAtom::Direction::OUT);
-    return GetEdgeListSizes(
-        AddMatch<ExpandVariable>(nullptr, "n", 0, EdgeAtom::Direction::OUT,
-                                 edge_types, 2, 2, e, "m", graph_view),
-        e);
-  };
-
-  auto new_edge_type = dba_->EdgeType("some_type");
-  // add two vertices branching out from the second layer
-  for (VertexAccessor &vertex : dba_->Vertices(true))
-    if (vertex.has_label(labels[1])) {
-      auto new_vertex = dba_->InsertVertex();
-      dba_->InsertEdge(vertex, new_vertex, new_edge_type);
-    }
-  ASSERT_EQ(CountIterable(dba_->Vertices(false)), 6);
-  ASSERT_EQ(CountIterable(dba_->Vertices(true)), 8);
-
-  EXPECT_EQ(test_expand(GraphView::OLD, {}), (map_int{{2, 8}}));
-  EXPECT_EQ(test_expand(GraphView::OLD, {new_edge_type}), (map_int{}));
-  EXPECT_EQ(test_expand(GraphView::NEW, {}), (map_int{{2, 12}}));
-  EXPECT_EQ(test_expand(GraphView::NEW, {edge_type}), (map_int{{2, 8}}));
-  EXPECT_EQ(test_expand(GraphView::NEW, {new_edge_type}), (map_int{}));
-  dba_->AdvanceCommand();
-  for (const auto graph_view : {GraphView::OLD, GraphView::NEW}) {
-    EXPECT_EQ(test_expand(graph_view, {}), (map_int{{2, 12}}));
-    EXPECT_EQ(test_expand(graph_view, {edge_type}), (map_int{{2, 8}}));
-    EXPECT_EQ(test_expand(graph_view, {new_edge_type}), (map_int{}));
-  }
-}
-
 TEST_F(QueryPlanExpandVariable, NamedPath) {
   auto e = Edge("r", EdgeAtom::Direction::OUT);
   auto expand =
@@ -900,7 +869,7 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
   // vertex)
   auto ExpandWShortest(EdgeAtom::Direction direction,
                        std::experimental::optional<int> max_depth,
-                       Expression *where, GraphView graph_view = GraphView::OLD,
+                       Expression *where,
                        std::experimental::optional<int> node_id = 0,
                        ScanAllTuple *existing_node_input = nullptr) {
     // scan the nodes optionally filtering on property value
@@ -931,7 +900,7 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
         ExpansionLambda{filter_edge, filter_node, where},
         ExpansionLambda{weight_edge, weight_node,
                         PROPERTY_LOOKUP(ident_e, prop)},
-        total_weight, graph_view);
+        total_weight);
 
     Frame frame(symbol_table.max_position());
     auto cursor = last_op->MakeCursor(dba);
@@ -1064,27 +1033,6 @@ TEST_F(QueryPlanExpandWeightedShortestPath, Where) {
   }
 }
 
-TEST_F(QueryPlanExpandWeightedShortestPath, GraphState) {
-  auto ExpandSize = [this](GraphView graph_view) {
-    return ExpandWShortest(EdgeAtom::Direction::BOTH, 1000, LITERAL(true),
-                           graph_view)
-        .size();
-  };
-  EXPECT_EQ(ExpandSize(GraphView::OLD), 4);
-  EXPECT_EQ(ExpandSize(GraphView::NEW), 4);
-  auto new_vertex = dba.InsertVertex();
-  new_vertex.PropsSet(prop.second, 5);
-  auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
-  edge.PropsSet(prop.second, 2);
-  EXPECT_EQ(CountIterable(dba.Vertices(false)), 5);
-  EXPECT_EQ(CountIterable(dba.Vertices(true)), 6);
-  EXPECT_EQ(ExpandSize(GraphView::OLD), 4);
-  EXPECT_EQ(ExpandSize(GraphView::NEW), 5);
-  dba.AdvanceCommand();
-  EXPECT_EQ(ExpandSize(GraphView::OLD), 5);
-  EXPECT_EQ(ExpandSize(GraphView::NEW), 5);
-}
-
 TEST_F(QueryPlanExpandWeightedShortestPath, ExistingNode) {
   auto ExpandPreceeding =
       [this](std::experimental::optional<int> preceeding_node_id) {
@@ -1101,7 +1049,7 @@ TEST_F(QueryPlanExpandWeightedShortestPath, ExistingNode) {
         }
 
         return ExpandWShortest(EdgeAtom::Direction::OUT, 1000, LITERAL(true),
-                               GraphView::OLD, std::experimental::nullopt, &n0);
+                               std::experimental::nullopt, &n0);
       };
 
   EXPECT_EQ(ExpandPreceeding(std::experimental::nullopt).size(), 20);
@@ -1126,27 +1074,6 @@ TEST_F(QueryPlanExpandWeightedShortestPath, UpperBound) {
     EXPECT_EQ(GetProp(results[3].vertex), 4);
     EXPECT_EQ(results[3].total_weight, 9);
   }
-  {
-    auto new_vertex = dba.InsertVertex();
-    new_vertex.PropsSet(prop.second, 5);
-    auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
-    edge.PropsSet(prop.second, 2);
-
-    auto results = ExpandWShortest(EdgeAtom::Direction::BOTH, 3, LITERAL(true),
-                                   GraphView::NEW);
-
-    ASSERT_EQ(results.size(), 5);
-    EXPECT_EQ(GetProp(results[0].vertex), 2);
-    EXPECT_EQ(results[0].total_weight, 3);
-    EXPECT_EQ(GetProp(results[1].vertex), 1);
-    EXPECT_EQ(results[1].total_weight, 5);
-    EXPECT_EQ(GetProp(results[2].vertex), 3);
-    EXPECT_EQ(results[2].total_weight, 6);
-    EXPECT_EQ(GetProp(results[3].vertex), 4);
-    EXPECT_EQ(results[3].total_weight, 9);
-    EXPECT_EQ(GetProp(results[4].vertex), 5);
-    EXPECT_EQ(results[4].total_weight, 12);
-  }
   {
     auto results = ExpandWShortest(EdgeAtom::Direction::BOTH, 2, LITERAL(true));
     ASSERT_EQ(results.size(), 4);
@@ -1169,33 +1096,52 @@ TEST_F(QueryPlanExpandWeightedShortestPath, UpperBound) {
     EXPECT_EQ(GetProp(results[2].vertex), 4);
     EXPECT_EQ(results[2].total_weight, 12);
   }
+  {
+    auto new_vertex = dba.InsertVertex();
+    new_vertex.PropsSet(prop.second, 5);
+    auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
+    edge.PropsSet(prop.second, 2);
+    dba.AdvanceCommand();
+
+    auto results = ExpandWShortest(EdgeAtom::Direction::BOTH, 3, LITERAL(true));
+
+    ASSERT_EQ(results.size(), 5);
+    EXPECT_EQ(GetProp(results[0].vertex), 2);
+    EXPECT_EQ(results[0].total_weight, 3);
+    EXPECT_EQ(GetProp(results[1].vertex), 1);
+    EXPECT_EQ(results[1].total_weight, 5);
+    EXPECT_EQ(GetProp(results[2].vertex), 3);
+    EXPECT_EQ(results[2].total_weight, 6);
+    EXPECT_EQ(GetProp(results[3].vertex), 4);
+    EXPECT_EQ(results[3].total_weight, 9);
+    EXPECT_EQ(GetProp(results[4].vertex), 5);
+    EXPECT_EQ(results[4].total_weight, 12);
+  }
 }
 
-TEST_F(QueryPlanExpandWeightedShortestPath, Exceptions) {
-  {
-    auto new_vertex = dba.InsertVertex();
-    new_vertex.PropsSet(prop.second, 5);
-    auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
-    edge.PropsSet(prop.second, "not a number");
-    EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, 1000, LITERAL(true),
-                                 GraphView::NEW),
-                 QueryRuntimeException);
-  }
-  {
-    auto new_vertex = dba.InsertVertex();
-    new_vertex.PropsSet(prop.second, 5);
-    auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
-    edge.PropsSet(prop.second, -10);  // negative weight
-    EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, 1000, LITERAL(true),
-                                 GraphView::NEW),
-                 QueryRuntimeException);
-  }
-  {
-    // negative upper bound
-    EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, -1, LITERAL(true),
-                                 GraphView::NEW),
-                 QueryRuntimeException);
-  }
+TEST_F(QueryPlanExpandWeightedShortestPath, NonNumericWeight) {
+  auto new_vertex = dba.InsertVertex();
+  new_vertex.PropsSet(prop.second, 5);
+  auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
+  edge.PropsSet(prop.second, "not a number");
+  dba.AdvanceCommand();
+  EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, 1000, LITERAL(true)),
+               QueryRuntimeException);
+}
+
+TEST_F(QueryPlanExpandWeightedShortestPath, NegativeWeight) {
+  auto new_vertex = dba.InsertVertex();
+  new_vertex.PropsSet(prop.second, 5);
+  auto edge = dba.InsertEdge(v[4], new_vertex, edge_type);
+  edge.PropsSet(prop.second, -10);  // negative weight
+  dba.AdvanceCommand();
+  EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, 1000, LITERAL(true)),
+               QueryRuntimeException);
+}
+
+TEST_F(QueryPlanExpandWeightedShortestPath, NegativeUpperBound) {
+  EXPECT_THROW(ExpandWShortest(EdgeAtom::Direction::BOTH, -1, LITERAL(true)),
+               QueryRuntimeException);
 }
 
 TEST(QueryPlan, ExpandOptional) {