diff --git a/src/query/common.hpp b/src/query/common.hpp
new file mode 100644
index 000000000..fac6bef85
--- /dev/null
+++ b/src/query/common.hpp
@@ -0,0 +1,16 @@
+#pragma once
+
+namespace query {
+
+/**
+ * Indicates that some part of query execution should
+ * see the OLD graph state (the latest state before the
+ * current transaction+command), or NEW (state as
+ * changed by the current transaction+command).
+ *
+ * Also some part of query execution could leave
+ * the graph state AS_IS, that is as it was left
+ * by some previous part of execution.
+ */
+enum class GraphView { AS_IS, OLD, NEW };
+}
diff --git a/src/query/interpret/eval.hpp b/src/query/interpret/eval.hpp
index 3b305232f..b2135e214 100644
--- a/src/query/interpret/eval.hpp
+++ b/src/query/interpret/eval.hpp
@@ -4,6 +4,7 @@
 #include <vector>
 
 #include "database/graph_db_accessor.hpp"
+#include "query/common.hpp"
 #include "query/frontend/ast/ast.hpp"
 #include "query/frontend/semantic/symbol_table.hpp"
 #include "query/interpret/frame.hpp"
@@ -16,26 +17,12 @@ namespace query {
 class ExpressionEvaluator : public TreeVisitorBase {
  public:
   ExpressionEvaluator(Frame &frame, const SymbolTable &symbol_table,
-                      GraphDbAccessor &db_accessor)
-      : frame_(frame), symbol_table_(symbol_table), db_accessor_(db_accessor) {}
-
-  /**
-   * When evaluting @c RecordAccessor, use @c SwitchNew to get the new
-   * data, as modified during the current command.
-   */
-  auto &SwitchNew() {
-    use_new_ = true;
-    return *this;
-  };
-
-  /**
-   * When evaluting @c RecordAccessor, use @c SwitchOld to get the old
-   * data, before the modification done by the current command.
-   */
-  auto &SwitchOld() {
-    use_new_ = false;
-    return *this;
-  };
+                      GraphDbAccessor &db_accessor,
+                      GraphView graph_view = GraphView::AS_IS)
+      : frame_(frame),
+        symbol_table_(symbol_table),
+        db_accessor_(db_accessor),
+        graph_view_(graph_view) {}
 
   /**
    * Removes and returns the last value from the result stack.
@@ -159,30 +146,45 @@ class ExpressionEvaluator : public TreeVisitorBase {
   // If the given TypedValue contains accessors, switch them to New or Old,
   // depending on use_new_ flag.
   void SwitchAccessors(TypedValue &value) {
+    if (graph_view_ == GraphView::AS_IS) return;
     switch (value.type()) {
       case TypedValue::Type::Vertex: {
         auto &vertex = value.Value<VertexAccessor>();
-        if (use_new_)
-          vertex.SwitchNew();
-        else
-          vertex.SwitchOld();
+        switch (graph_view_) {
+          case GraphView::NEW:
+            vertex.SwitchNew();
+            break;
+          case GraphView::OLD:
+            vertex.SwitchOld();
+            break;
+          default:
+            permanent_fail("Unhandled GraphView enum");
+        }
         break;
       }
       case TypedValue::Type::Edge: {
         auto &edge = value.Value<EdgeAccessor>();
-        if (use_new_)
-          edge.SwitchNew();
-        else
-          edge.SwitchOld();
+        switch (graph_view_) {
+          case GraphView::NEW:
+            edge.SwitchNew();
+            break;
+          case GraphView::OLD:
+            edge.SwitchOld();
+            break;
+          default:
+            permanent_fail("Unhandled GraphView enum");
+        }
         break;
       }
       case TypedValue::Type::List: {
         auto &list = value.Value<std::vector<TypedValue>>();
         for (auto &list_value : list) SwitchAccessors(list_value);
+        break;
       }
       case TypedValue::Type::Map: {
         auto &map = value.Value<std::map<std::string, TypedValue>>();
         for (auto &kv : map) SwitchAccessors(kv.second);
+        break;
       }
       default:
         break;
@@ -191,11 +193,9 @@ class ExpressionEvaluator : public TreeVisitorBase {
 
   Frame &frame_;
   const SymbolTable &symbol_table_;
-  std::list<TypedValue> result_stack_;
-  // If true, use SwitchNew on evaluated record accessors. This should be
-  // done only in expressions which may return one. E.g. identifier, list
-  // indexing.
-  bool use_new_ = false;
   GraphDbAccessor &db_accessor_;
+  // which switching approach should be used when evaluating
+  const GraphView graph_view_;
+  std::list<TypedValue> result_stack_;
 };
 }
diff --git a/src/query/plan/operator.cpp b/src/query/plan/operator.cpp
index b90409b11..0b1bacce2 100644
--- a/src/query/plan/operator.cpp
+++ b/src/query/plan/operator.cpp
@@ -70,10 +70,9 @@ void CreateNode::CreateNodeCursor::Create(Frame &frame,
   auto new_node = db_.insert_vertex();
   for (auto label : self_.node_atom_->labels_) new_node.add_label(label);
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Evaluator should use the latest accessors, as modified in this query, when
   // setting properties on new nodes.
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   for (auto &kv : self_.node_atom_->properties_) {
     kv.second->Accept(evaluator);
     new_node.PropsSet(kv.first, evaluator.PopBack());
@@ -108,12 +107,11 @@ bool CreateExpand::CreateExpandCursor::Pull(Frame &frame,
   TypedValue &vertex_value = frame[self_.input_symbol_];
   auto &v1 = vertex_value.Value<VertexAccessor>();
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Similarly to CreateNode, newly created edges and nodes should use the
   // latest accesors.
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   // E.g. we pickup new properties: `CREATE (n {p: 42}) -[:r {ep: n.p}]-> ()`
   v1.SwitchNew();
-  evaluator.SwitchNew();
 
   // get the destination vertex (possibly an existing node)
   auto &v2 = OtherVertex(frame, symbol_table, evaluator);
@@ -170,9 +168,14 @@ void CreateExpand::CreateExpandCursor::CreateEdge(
 }
 
 ScanAll::ScanAll(const NodeAtom *node_atom,
-                 const std::shared_ptr<LogicalOperator> &input)
-    : node_atom_(node_atom), input_(input ? input : std::make_shared<Once>()) {}
-
+                 const std::shared_ptr<LogicalOperator> &input,
+                 GraphView graph_view)
+    : node_atom_(node_atom),
+      input_(input ? input : std::make_shared<Once>()),
+      graph_view_(graph_view) {
+  permanent_assert(graph_view != GraphView::AS_IS,
+                   "ScanAll must have explicitly defined GraphView")
+}
 ACCEPT_WITH_INPUT(ScanAll)
 
 std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) {
@@ -182,6 +185,8 @@ std::unique_ptr<Cursor> ScanAll::MakeCursor(GraphDbAccessor &db) {
 ScanAll::ScanAllCursor::ScanAllCursor(const ScanAll &self, GraphDbAccessor &db)
     : self_(self),
       input_cursor_(self.input_->MakeCursor(db)),
+      // TODO change to db.vertices(self.switch_ == GraphView::NEW)
+      // once this GraphDbAccessor API is available
       vertices_(db.vertices()),
       vertices_it_(vertices_.end()) {}
 
@@ -207,13 +212,15 @@ void ScanAll::ScanAllCursor::Reset() {
 
 Expand::Expand(const NodeAtom *node_atom, const EdgeAtom *edge_atom,
                const std::shared_ptr<LogicalOperator> &input,
-               Symbol input_symbol, bool existing_node, bool existing_edge)
+               Symbol input_symbol, bool existing_node, bool existing_edge,
+               GraphView graph_view)
     : node_atom_(node_atom),
       edge_atom_(edge_atom),
       input_(input),
       input_symbol_(input_symbol),
       existing_node_(existing_node),
-      existing_edge_(existing_edge) {}
+      existing_edge_(existing_edge),
+      graph_view_(graph_view) {}
 
 ACCEPT_WITH_INPUT(Expand)
 
@@ -275,11 +282,17 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame,
 
   TypedValue &vertex_value = frame[self_.input_symbol_];
   auto &vertex = vertex_value.Value<VertexAccessor>();
-  // We don't want newly created edges, so switch to old. If we included new
-  // edges, e.g. created by CreateExpand operator, the behaviour would be wrong
-  // and may cause infinite loops by continually creating edges and traversing
-  // them.
-  vertex.SwitchOld();
+  // switch the expansion origin vertex to the desired state
+  switch (self_.graph_view_) {
+    case GraphView::NEW:
+      vertex.SwitchNew();
+      break;
+    case GraphView::OLD:
+      vertex.SwitchOld();
+      break;
+    case GraphView::AS_IS:
+      break;
+  }
 
   auto direction = self_.edge_atom_->direction_;
   if (direction == EdgeAtom::Direction::LEFT ||
@@ -377,9 +390,9 @@ bool NodeFilter::NodeFilterCursor::VertexPasses(
   for (auto label : self_.node_atom_->labels_)
     if (!vertex.has_label(label)) return false;
 
-  ExpressionEvaluator expression_evaluator(frame, symbol_table, db_);
+  ExpressionEvaluator expression_evaluator(frame, symbol_table, db_,
+                                           GraphView::OLD);
   // We don't want newly set properties to affect filtering.
-  expression_evaluator.SwitchOld();
   for (auto prop_pair : self_.node_atom_->properties_) {
     prop_pair.second->Accept(expression_evaluator);
     TypedValue comparison_result =
@@ -427,9 +440,9 @@ bool EdgeFilter::EdgeFilterCursor::EdgePasses(const EdgeAccessor &edge,
                                    [type](auto t) { return t == type; }))
     return false;
 
-  ExpressionEvaluator expression_evaluator(frame, symbol_table, db_);
   // We don't want newly set properties to affect filtering.
-  expression_evaluator.SwitchOld();
+  ExpressionEvaluator expression_evaluator(frame, symbol_table, db_,
+                                           GraphView::OLD);
   for (auto prop_pair : self_.edge_atom_->properties_) {
     prop_pair.second->Accept(expression_evaluator);
     TypedValue comparison_result =
@@ -454,10 +467,9 @@ Filter::FilterCursor::FilterCursor(const Filter &self, GraphDbAccessor &db)
     : self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
 
 bool Filter::FilterCursor::Pull(Frame &frame, const SymbolTable &symbol_table) {
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Like all filters, newly set values should not affect filtering of old nodes
   // and edges.
-  evaluator.SwitchOld();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::OLD);
   while (input_cursor_->Pull(frame, symbol_table)) {
     self_.expression_->Accept(evaluator);
     TypedValue result = evaluator.PopBack();
@@ -490,9 +502,8 @@ Produce::ProduceCursor::ProduceCursor(const Produce &self, GraphDbAccessor &db)
 bool Produce::ProduceCursor::Pull(Frame &frame,
                                   const SymbolTable &symbol_table) {
   if (input_cursor_->Pull(frame, symbol_table)) {
-    ExpressionEvaluator evaluator(frame, symbol_table, db_);
     // Produce should always yield the latest results.
-    evaluator.SwitchNew();
+    ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
     for (auto named_expr : self_.named_expressions_)
       named_expr->Accept(evaluator);
     return true;
@@ -518,10 +529,9 @@ Delete::DeleteCursor::DeleteCursor(const Delete &self, GraphDbAccessor &db)
 bool Delete::DeleteCursor::Pull(Frame &frame, const SymbolTable &symbol_table) {
   if (!input_cursor_->Pull(frame, symbol_table)) return false;
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Delete should get the latest information, this way it is also possible to
   // delete newly added nodes and edges.
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   // collect expressions results so edges can get deleted before vertices
   // this is necessary because an edge that gets deleted could block vertex
   // deletion
@@ -547,7 +557,7 @@ bool Delete::DeleteCursor::Pull(Frame &frame, const SymbolTable &symbol_table) {
         if (self_.detach_)
           db_.detach_remove_vertex(va);
         else if (!db_.remove_vertex(va))
-          throw query::QueryRuntimeException(
+          throw QueryRuntimeException(
               "Failed to remove vertex because of it's existing "
               "connections. Consider using DETACH DELETE.");
         break;
@@ -582,9 +592,8 @@ bool SetProperty::SetPropertyCursor::Pull(Frame &frame,
                                           const SymbolTable &symbol_table) {
   if (!input_cursor_->Pull(frame, symbol_table)) return false;
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Set, just like Create needs to see the latest changes.
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   self_.lhs_->expression_->Accept(evaluator);
   TypedValue lhs = evaluator.PopBack();
   self_.rhs_->Accept(evaluator);
@@ -631,9 +640,8 @@ bool SetProperties::SetPropertiesCursor::Pull(Frame &frame,
 
   TypedValue &lhs = frame[self_.input_symbol_];
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Set, just like Create needs to see the latest changes.
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   self_.rhs_->Accept(evaluator);
   TypedValue rhs = evaluator.PopBack();
 
@@ -738,9 +746,8 @@ bool RemoveProperty::RemovePropertyCursor::Pull(
     Frame &frame, const SymbolTable &symbol_table) {
   if (!input_cursor_->Pull(frame, symbol_table)) return false;
 
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
   // Remove, just like Delete needs to see the latest changes.
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   self_.lhs_->expression_->Accept(evaluator);
   TypedValue lhs = evaluator.PopBack();
 
@@ -972,8 +979,7 @@ bool Aggregate::AggregateCursor::Pull(Frame &frame,
 
 void Aggregate::AggregateCursor::ProcessAll(Frame &frame,
                                             const SymbolTable &symbol_table) {
-  ExpressionEvaluator evaluator(frame, symbol_table, db_);
-  evaluator.SwitchNew();
+  ExpressionEvaluator evaluator(frame, symbol_table, db_, GraphView::NEW);
   while (input_cursor_->Pull(frame, symbol_table))
     ProcessOne(frame, symbol_table, evaluator);
 
diff --git a/src/query/plan/operator.hpp b/src/query/plan/operator.hpp
index 856cfd210..72e085a06 100644
--- a/src/query/plan/operator.hpp
+++ b/src/query/plan/operator.hpp
@@ -11,6 +11,7 @@
 
 #include "database/graph_db_accessor.hpp"
 #include "database/graph_db_datatypes.hpp"
+#include "query/common.hpp"
 #include "query/frontend/semantic/symbol_table.hpp"
 #include "utils/hashing/fnv.hpp"
 #include "utils/visitor/visitable.hpp"
@@ -249,17 +250,23 @@ class CreateExpand : public LogicalOperator {
  * It accepts an optional input. If provided then this op scans all the nodes
  * currently in the database for each successful Pull from it's input, thereby
  * producing a cartesian product of input Pulls and database elements.
+ *
+ * ScanAll can either iterate over the previous graph state (state before
+ * the current transacton+command) or over current state. This is controlled
+ * with a constructor argument.
  */
 class ScanAll : public LogicalOperator {
  public:
   ScanAll(const NodeAtom *node_atom,
-          const std::shared_ptr<LogicalOperator> &input);
+          const std::shared_ptr<LogicalOperator> &input,
+          GraphView graph_view = GraphView::OLD);
   void Accept(LogicalOperatorVisitor &visitor) override;
   std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
 
  private:
   const NodeAtom *node_atom_ = nullptr;
   const std::shared_ptr<LogicalOperator> input_;
+  const GraphView graph_view_;
 
   class ScanAllCursor : public Cursor {
    public:
@@ -305,6 +312,10 @@ class Expand : public LogicalOperator {
    * Pulled node/edge, and should only be checked for equalities
    * during expansion.
    *
+   * Expansion can be done from old or new state of the vertex
+   * the expansion originates from. This is controlled with a
+   * constructor argument.
+   *
    * @param node_atom Describes the node to be expanded. Only the
    *    identifier is used, labels and properties are ignored.
    * @param edge_atom Describes the edge to be expanded. Identifier
@@ -318,7 +329,8 @@ class Expand : public LogicalOperator {
    */
   Expand(const NodeAtom *node_atom, const EdgeAtom *edge_atom,
          const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
-         bool existing_node, bool existing_edge);
+         bool existing_node, bool existing_edge,
+         GraphView graph_view = GraphView::AS_IS);
   void Accept(LogicalOperatorVisitor &visitor) override;
   std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
 
@@ -338,6 +350,9 @@ class Expand : public LogicalOperator {
   const bool existing_node_;
   const bool existing_edge_;
 
+  // from which state the input node should get expanded
+  const GraphView graph_view_;
+
   class ExpandCursor : public Cursor {
    public:
     ExpandCursor(const Expand &self, GraphDbAccessor &db);
diff --git a/src/storage/record_accessor.cpp b/src/storage/record_accessor.cpp
index 791edefa2..e712eea7b 100644
--- a/src/storage/record_accessor.cpp
+++ b/src/storage/record_accessor.cpp
@@ -69,12 +69,7 @@ RecordAccessor<TRecord> &RecordAccessor<TRecord>::SwitchNew() {
 
 template <typename TRecord>
 RecordAccessor<TRecord> &RecordAccessor<TRecord>::SwitchOld() {
-  // if this whole record is new (new version-list) then we don't
-  // have a valid old_ version. in such a situation SwitchOld
-  // is not a legal function call
-  debug_assert(old_ != nullptr,
-               "RecordAccessor.old_ is nullptr and SwitchOld called");
-  current_ = old_;
+  current_ = old_ ? old_ : new_;
   return *this;
 }
 
diff --git a/src/storage/record_accessor.hpp b/src/storage/record_accessor.hpp
index 9b02b478d..00b31c26a 100644
--- a/src/storage/record_accessor.hpp
+++ b/src/storage/record_accessor.hpp
@@ -137,18 +137,19 @@ class RecordAccessor {
 
   /*
    * Switches this record accessor to use the latest
-   * version (visible to the current transaction+command).
+   * version visible to the current transaction+command.
+   * Possibly the one that was created by this transaction+command.
    *
    * @return A reference to this.
    */
   RecordAccessor<TRecord> &SwitchNew();
 
   /**
-   * Switches this record accessor to use the old
-   * (not updated) version visible to the current transaction+command.
-   *
-   * It is not legal to call this function on a Vertex/Edge that
-   * was created by the current transaction+command.
+   * Attempts to switch this accessor to use the
+   * latest version not updated by the current transaction+command.
+   * If that is not possible (vertex/edge was created
+   * by the current transaction/command), it does nothing
+   * (current remains pointing to the new version).
    *
    * @return A reference to this.
    */
diff --git a/tests/unit/query_plan_common.hpp b/tests/unit/query_plan_common.hpp
index 696111ef4..49e21edeb 100644
--- a/tests/unit/query_plan_common.hpp
+++ b/tests/unit/query_plan_common.hpp
@@ -7,8 +7,11 @@
 #include <memory>
 #include <vector>
 
+#include "communication/result_stream_faker.hpp"
+#include "query/common.hpp"
 #include "query/frontend/semantic/symbol_table.hpp"
 #include "query/interpret/frame.hpp"
+#include "query/plan/operator.hpp"
 
 #include "query_common.hpp"
 
@@ -86,9 +89,10 @@ struct ScanAllTuple {
  */
 ScanAllTuple MakeScanAll(AstTreeStorage &storage, SymbolTable &symbol_table,
                          const std::string &identifier,
-                         std::shared_ptr<LogicalOperator> input = {nullptr}) {
+                         std::shared_ptr<LogicalOperator> input = {nullptr},
+                         GraphView graph_view = GraphView::OLD) {
   auto node = NODE(identifier);
-  auto logical_op = std::make_shared<ScanAll>(node, input);
+  auto logical_op = std::make_shared<ScanAll>(node, input, graph_view);
   auto symbol = symbol_table.CreateSymbol(identifier);
   symbol_table[*node->identifier_] = symbol;
   //  return std::make_tuple(node, logical_op, symbol);
@@ -106,8 +110,9 @@ struct ExpandTuple {
 ExpandTuple MakeExpand(AstTreeStorage &storage, SymbolTable &symbol_table,
                        std::shared_ptr<LogicalOperator> input,
                        Symbol input_symbol, const std::string &edge_identifier,
-                       EdgeAtom::Direction direction, bool edge_cycle,
-                       const std::string &node_identifier, bool node_cycle) {
+                       EdgeAtom::Direction direction, bool existing_edge,
+                       const std::string &node_identifier, bool existing_node,
+                       GraphView graph_view = GraphView::AS_IS) {
   auto edge = EDGE(edge_identifier, direction);
   auto edge_sym = symbol_table.CreateSymbol(edge_identifier);
   symbol_table[*edge->identifier_] = edge_sym;
@@ -117,7 +122,7 @@ ExpandTuple MakeExpand(AstTreeStorage &storage, SymbolTable &symbol_table,
   symbol_table[*node->identifier_] = node_sym;
 
   auto op = std::make_shared<Expand>(node, edge, input, input_symbol,
-                                     node_cycle, edge_cycle);
+                                     existing_node, existing_edge, graph_view);
 
   return ExpandTuple{edge, edge_sym, node, node_sym, op};
 }
diff --git a/tests/unit/query_plan_match_filter_return.cpp b/tests/unit/query_plan_match_filter_return.cpp
index b05639e08..b2dc3034c 100644
--- a/tests/unit/query_plan_match_filter_return.cpp
+++ b/tests/unit/query_plan_match_filter_return.cpp
@@ -33,14 +33,24 @@ TEST(QueryPlan, MatchReturn) {
   AstTreeStorage storage;
   SymbolTable symbol_table;
 
-  auto scan_all = MakeScanAll(storage, symbol_table, "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("named_expression_1");
+  auto test_pull_count = [&](GraphView graph_view) {
+    auto scan_all =
+        MakeScanAll(storage, symbol_table, "n", nullptr, graph_view);
+    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("named_expression_1");
+    return PullAll(produce, *dba, symbol_table);
+  };
 
-  ResultStreamFaker result = CollectProduce(produce, symbol_table, *dba);
-  EXPECT_EQ(result.GetResults().size(), 2);
+  EXPECT_EQ(2, test_pull_count(GraphView::NEW));
+  EXPECT_EQ(2, test_pull_count(GraphView::OLD));
+  dba->insert_vertex();
+  // TODO uncomment once the functionality is implemented
+  // EXPECT_EQ(3, test_pull_count(GraphView::NEW));
+  EXPECT_EQ(2, test_pull_count(GraphView::OLD));
+  dba->advance_command();
+  EXPECT_EQ(3, test_pull_count(GraphView::OLD));
 }
 
 TEST(QueryPlan, MatchReturnCartesian) {
@@ -112,10 +122,11 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
   dba->insert_vertex();
   // test all combination of (label | no_label) * (no_prop | wrong_prop |
   // right_prop)
-  // only v1 will have the right labels
+  // only v1-v3 will have the right labels
   v1.add_label(label);
   v2.add_label(label);
   v3.add_label(label);
+  // v1 and v4 will have the right properties
   v1.PropsSet(property, 42);
   v2.PropsSet(property, 1);
   v4.PropsSet(property, 42);
@@ -139,8 +150,14 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
   symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1");
   auto produce = MakeProduce(node_filter, output);
 
-  ResultStreamFaker result = CollectProduce(produce, symbol_table, *dba);
-  EXPECT_EQ(result.GetResults().size(), 1);
+  EXPECT_EQ(1, PullAll(produce, *dba, symbol_table));
+
+  //  test that filtering works with old records
+  v4.Reconstruct();
+  v4.add_label(label);
+  EXPECT_EQ(1, PullAll(produce, *dba, symbol_table));
+  dba->advance_command();
+  EXPECT_EQ(2, PullAll(produce, *dba, symbol_table));
 }
 
 TEST(QueryPlan, NodeFilterMultipleLabels) {
@@ -210,11 +227,10 @@ TEST(QueryPlan, Expand) {
   AstTreeStorage storage;
   SymbolTable symbol_table;
 
-  auto test_expand = [&](EdgeAtom::Direction direction,
-                         int expected_result_count) {
+  auto test_expand = [&](EdgeAtom::Direction direction, GraphView graph_view) {
     auto n = MakeScanAll(storage, symbol_table, "n");
     auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r", direction,
-                          false, "m", false);
+                          false, "m", false, graph_view);
 
     // make a named expression and a produce
     auto output = NEXPR("m", IDENT("m"));
@@ -222,16 +238,32 @@ TEST(QueryPlan, Expand) {
     symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1");
     auto produce = MakeProduce(r_m.op_, output);
 
-    ResultStreamFaker result = CollectProduce(produce, symbol_table, *dba);
-    EXPECT_EQ(result.GetResults().size(), expected_result_count);
+    return PullAll(produce, *dba, symbol_table);
   };
 
-  test_expand(EdgeAtom::Direction::RIGHT, 2);
-  test_expand(EdgeAtom::Direction::LEFT, 2);
-  test_expand(EdgeAtom::Direction::BOTH, 4);
+  EXPECT_EQ(2, test_expand(EdgeAtom::Direction::RIGHT, GraphView::AS_IS));
+  EXPECT_EQ(2, test_expand(EdgeAtom::Direction::LEFT, GraphView::AS_IS));
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, GraphView::AS_IS));
+  //
+  // test that expand works well for both old and new graph state
+  v1.Reconstruct();
+  v2.Reconstruct();
+  v3.Reconstruct();
+  dba->insert_edge(v1, v2, edge_type);
+  dba->insert_edge(v1, v3, edge_type);
+  EXPECT_EQ(2, test_expand(EdgeAtom::Direction::RIGHT, GraphView::OLD));
+  EXPECT_EQ(2, test_expand(EdgeAtom::Direction::LEFT, GraphView::OLD));
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::RIGHT, GraphView::NEW));
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::LEFT, GraphView::NEW));
+  EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::NEW));
+  dba->advance_command();
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::RIGHT, GraphView::OLD));
+  EXPECT_EQ(4, test_expand(EdgeAtom::Direction::LEFT, GraphView::OLD));
+  EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
 }
 
-TEST(QueryPlan, ExpandNodeCycle) {
+TEST(QueryPlan, ExpandExistingNode) {
   Dbms dbms;
   auto dba = dbms.active();
 
@@ -247,11 +279,12 @@ TEST(QueryPlan, ExpandNodeCycle) {
   AstTreeStorage storage;
   SymbolTable symbol_table;
 
-  auto test_cycle = [&](bool with_cycle, int expected_result_count) {
+  auto test_existing = [&](bool with_existing, int expected_result_count) {
     auto n = MakeScanAll(storage, symbol_table, "n");
-    auto r_n = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
-                          EdgeAtom::Direction::RIGHT, false, "n", with_cycle);
-    if (with_cycle)
+    auto r_n =
+        MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
+                   EdgeAtom::Direction::RIGHT, false, "n", with_existing);
+    if (with_existing)
       symbol_table[*r_n.node_->identifier_] =
           symbol_table[*n.node_->identifier_];
 
@@ -265,11 +298,11 @@ TEST(QueryPlan, ExpandNodeCycle) {
     EXPECT_EQ(result.GetResults().size(), expected_result_count);
   };
 
-  test_cycle(true, 1);
-  test_cycle(false, 2);
+  test_existing(true, 1);
+  test_existing(false, 2);
 }
 
-TEST(QueryPlan, ExpandEdgeCycle) {
+TEST(QueryPlan, ExpandExistingEdge) {
   Dbms dbms;
   auto dba = dbms.active();
 
@@ -288,13 +321,13 @@ TEST(QueryPlan, ExpandEdgeCycle) {
   AstTreeStorage storage;
   SymbolTable symbol_table;
 
-  auto test_cycle = [&](bool with_cycle, int expected_result_count) {
+  auto test_existing = [&](bool with_existing, int expected_result_count) {
     auto i = MakeScanAll(storage, symbol_table, "i");
     auto r_j = MakeExpand(storage, symbol_table, i.op_, i.sym_, "r",
                           EdgeAtom::Direction::BOTH, false, "j", false);
     auto r_k = MakeExpand(storage, symbol_table, r_j.op_, r_j.node_sym_, "r",
-                          EdgeAtom::Direction::BOTH, with_cycle, "k", false);
-    if (with_cycle)
+                          EdgeAtom::Direction::BOTH, with_existing, "k", false);
+    if (with_existing)
       symbol_table[*r_k.edge_->identifier_] =
           symbol_table[*r_j.edge_->identifier_];
 
@@ -309,8 +342,8 @@ TEST(QueryPlan, ExpandEdgeCycle) {
 
   };
 
-  test_cycle(true, 4);
-  test_cycle(false, 6);
+  test_existing(true, 4);
+  test_existing(false, 6);
 }
 
 TEST(QueryPlan, ExpandBothCycleEdgeCase) {
@@ -362,29 +395,39 @@ TEST(QueryPlan, EdgeFilter) {
     }
   }
   dba->advance_command();
+  for (auto &vertex : vertices) vertex.Reconstruct();
+  for (auto &edge : edges) edge.Reconstruct();
 
   AstTreeStorage storage;
   SymbolTable symbol_table;
 
-  // define an operator tree for query
-  // MATCH (n)-[r]->(m) RETURN m
+  auto test_filter = [&]() {
+    // define an operator tree for query
+    // MATCH (n)-[r]->(m) RETURN m
 
-  auto n = MakeScanAll(storage, symbol_table, "n");
-  auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
-                        EdgeAtom::Direction::RIGHT, false, "m", false);
-  r_m.edge_->edge_types_.push_back(edge_types[0]);
-  r_m.edge_->properties_[prop] = LITERAL(42);
-  auto edge_filter =
-      std::make_shared<EdgeFilter>(r_m.op_, r_m.edge_sym_, r_m.edge_);
+    auto n = MakeScanAll(storage, symbol_table, "n");
+    auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
+                          EdgeAtom::Direction::RIGHT, false, "m", false);
+    r_m.edge_->edge_types_.push_back(edge_types[0]);
+    r_m.edge_->properties_[prop] = LITERAL(42);
+    auto edge_filter =
+        std::make_shared<EdgeFilter>(r_m.op_, r_m.edge_sym_, r_m.edge_);
 
-  // make a named expression and a produce
-  auto output = NEXPR("m", IDENT("m"));
-  symbol_table[*output->expression_] = r_m.node_sym_;
-  symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1");
-  auto produce = MakeProduce(edge_filter, output);
+    // make a named expression and a produce
+    auto output = NEXPR("m", IDENT("m"));
+    symbol_table[*output->expression_] = r_m.node_sym_;
+    symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1");
+    auto produce = MakeProduce(edge_filter, output);
 
-  ResultStreamFaker result = CollectProduce(produce, symbol_table, *dba);
-  EXPECT_EQ(result.GetResults().size(), 1);
+    return PullAll(produce, *dba, symbol_table);
+  };
+
+  EXPECT_EQ(1, test_filter());
+  // test that edge filtering always filters on old state
+  for (auto &edge : edges) edge.PropsSet(prop, 42);
+  EXPECT_EQ(1, test_filter());
+  dba->advance_command();
+  EXPECT_EQ(3, test_filter());
 }
 
 TEST(QueryPlan, EdgeFilterEmpty) {
diff --git a/tests/unit/record_edge_vertex_accessor.cpp b/tests/unit/record_edge_vertex_accessor.cpp
index 4bcfb4798..f470a3d1e 100644
--- a/tests/unit/record_edge_vertex_accessor.cpp
+++ b/tests/unit/record_edge_vertex_accessor.cpp
@@ -80,11 +80,11 @@ TEST(RecordAccessor, RecordLessThan) {
 TEST(RecordAccessor, SwitchOldAndSwitchNewMemberFunctionTest) {
   Dbms dbms;
 
-  // test SwitchOld failure on new record, SwitchNew OK
+  // test both Switches work on new record
   {
     auto dba = dbms.active();
     auto v1 = dba->insert_vertex();
-    EXPECT_DEATH(v1.SwitchOld(), "");
+    v1.SwitchOld();
     v1.SwitchNew();
     dba->commit();
   }