From 3604046f686fa112f8bc8320078468f5fca51cf3 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Mon, 12 Dec 2022 10:53:07 +0100
Subject: [PATCH 01/90] Implement cypher query based simulation testing

Make the Interpreter be able to handle SimulatorTransport as well. This
includes introducing changes that make it possible to use the different
transport types in a semi-polymorphic way with the introduction of
factory methods in the RequestRouter. The reason for this solution is
that the classes that represent the different transport types have
member function templates, that we can not make virtual. This solution
seemed to be the least convoluted. In the testing itself now it is
possible to pass a set of cypher queried to the interpreter which would
run these queries against the interpreter and the individual shards that
are managed and started up by the MachineManager with the different
entities communicating over the simulated network.
---
 src/io/local_transport/local_system.hpp       |   2 +
 src/io/simulator/simulator.hpp                |   4 +-
 src/io/simulator/simulator_transport.hpp      |   2 +-
 src/memgraph.cpp                              |  14 ++-
 src/query/v2/interpreter.cpp                  |  32 ++---
 src/query/v2/interpreter.hpp                  |  19 +--
 src/query/v2/request_router.hpp               | 109 ++++++++++++++++++
 tests/simulation/CMakeLists.txt               |   3 +-
 tests/simulation/cluster_property_test_v2.cpp |  65 +++++++++++
 tests/simulation/simulation_interpreter.hpp   |  95 +++++++++++++++
 tests/simulation/test_cluster.hpp             |  62 ++++++++++
 11 files changed, 368 insertions(+), 39 deletions(-)
 create mode 100644 tests/simulation/cluster_property_test_v2.cpp
 create mode 100644 tests/simulation/simulation_interpreter.hpp

diff --git a/src/io/local_transport/local_system.hpp b/src/io/local_transport/local_system.hpp
index 2e54f8d75..ec148b50b 100644
--- a/src/io/local_transport/local_system.hpp
+++ b/src/io/local_transport/local_system.hpp
@@ -29,6 +29,8 @@ class LocalSystem {
     return Io{local_transport, address};
   }
 
+  std::shared_ptr<LocalTransportHandle> &GetTransportHandle() { return local_transport_handle_; }
+
   void ShutDown() { local_transport_handle_->ShutDown(); }
 };
 
diff --git a/src/io/simulator/simulator.hpp b/src/io/simulator/simulator.hpp
index 622c264b4..5095ab58b 100644
--- a/src/io/simulator/simulator.hpp
+++ b/src/io/simulator/simulator.hpp
@@ -41,7 +41,7 @@ class Simulator {
   Io<SimulatorTransport> Register(Address address) {
     std::uniform_int_distribution<uint64_t> seed_distrib;
     uint64_t seed = seed_distrib(rng_);
-    return Io{SimulatorTransport{simulator_handle_, address, seed}, address};
+    return Io{SimulatorTransport(simulator_handle_, address, seed), address};
   }
 
   void IncrementServerCountAndWaitForQuiescentState(Address address) {
@@ -49,5 +49,7 @@ class Simulator {
   }
 
   SimulatorStats Stats() { return simulator_handle_->Stats(); }
+
+  std::shared_ptr<SimulatorHandle> GetSimulatorHandle() { return simulator_handle_; }
 };
 };  // namespace memgraph::io::simulator
diff --git a/src/io/simulator/simulator_transport.hpp b/src/io/simulator/simulator_transport.hpp
index 5e5a24aa9..492b59d3a 100644
--- a/src/io/simulator/simulator_transport.hpp
+++ b/src/io/simulator/simulator_transport.hpp
@@ -25,7 +25,7 @@ using memgraph::io::Time;
 
 class SimulatorTransport {
   std::shared_ptr<SimulatorHandle> simulator_handle_;
-  const Address address_;
+  Address address_;
   std::mt19937 rng_;
 
  public:
diff --git a/src/memgraph.cpp b/src/memgraph.cpp
index d825cc0e7..ca03024fe 100644
--- a/src/memgraph.cpp
+++ b/src/memgraph.cpp
@@ -607,15 +607,15 @@ int main(int argc, char **argv) {
   // to minimize the impact of their failure on the main storage.
 
   memgraph::io::local_transport::LocalSystem ls;
-  auto unique_local_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
-  auto io = ls.Register(unique_local_addr_query);
+  auto unique_local_coord_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
+  auto io = ls.Register(unique_local_coord_addr_query);
 
   memgraph::machine_manager::MachineConfig config{
-      .coordinator_addresses = std::vector<memgraph::io::Address>{unique_local_addr_query},
+      .coordinator_addresses = std::vector<memgraph::io::Address>{unique_local_coord_addr_query},
       .is_storage = true,
       .is_coordinator = true,
-      .listen_ip = unique_local_addr_query.last_known_ip,
-      .listen_port = unique_local_addr_query.last_known_port,
+      .listen_ip = unique_local_coord_addr_query.last_known_ip,
+      .listen_port = unique_local_coord_addr_query.last_known_port,
   };
 
   memgraph::coordinator::ShardMap sm;
@@ -640,6 +640,8 @@ int main(int argc, char **argv) {
   memgraph::machine_manager::MachineManager<memgraph::io::local_transport::LocalTransport> mm{io, config, coordinator};
   std::jthread mm_thread([&mm] { mm.Run(); });
 
+  auto rr_factory = std::make_unique<memgraph::query::v2::LocalRequestRouterFactory>(ls.GetTransportHandle());
+
   memgraph::query::v2::InterpreterContext interpreter_context{
       (memgraph::storage::v3::Shard *)(nullptr),
       {.query = {.allow_load_csv = FLAGS_allow_load_csv},
@@ -650,7 +652,7 @@ int main(int argc, char **argv) {
        .stream_transaction_conflict_retries = FLAGS_stream_transaction_conflict_retries,
        .stream_transaction_retry_interval = std::chrono::milliseconds(FLAGS_stream_transaction_retry_interval)},
       FLAGS_data_directory,
-      std::move(io),
+      std::move(rr_factory),
       mm.CoordinatorAddress()};
 
   SessionData session_data{&interpreter_context};
diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 1c2d6dadf..c428d1a39 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -793,34 +793,24 @@ using RWType = plan::ReadWriteTypeChecker::RWType;
 
 InterpreterContext::InterpreterContext(storage::v3::Shard *db, const InterpreterConfig config,
                                        const std::filesystem::path & /*data_directory*/,
-                                       io::Io<io::local_transport::LocalTransport> io,
+                                       std::unique_ptr<RequestRouterFactory> &&request_router_factory,
                                        coordinator::Address coordinator_addr)
-    : db(db), config(config), io{std::move(io)}, coordinator_address{coordinator_addr} {}
+    : db(db),
+      config(config),
+      coordinator_address{coordinator_addr},
+      request_router_factory_{std::move(request_router_factory)} {}
 
 Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_context_(interpreter_context) {
   MG_ASSERT(interpreter_context_, "Interpreter context must not be NULL");
 
-  // TODO(tyler) make this deterministic so that it can be tested.
-  auto random_uuid = boost::uuids::uuid{boost::uuids::random_generator()()};
-  auto query_io = interpreter_context_->io.ForkLocal(random_uuid);
+  request_router_ =
+      interpreter_context_->request_router_factory_->CreateRequestRouter(interpreter_context_->coordinator_address);
 
-  request_router_ = std::make_unique<RequestRouter<io::local_transport::LocalTransport>>(
-      coordinator::CoordinatorClient<io::local_transport::LocalTransport>(
-          query_io, interpreter_context_->coordinator_address, std::vector{interpreter_context_->coordinator_address}),
-      std::move(query_io));
   // Get edge ids
-  coordinator::CoordinatorWriteRequests requests{coordinator::AllocateEdgeIdBatchRequest{.batch_size = 1000000}};
-  io::rsm::WriteRequest<coordinator::CoordinatorWriteRequests> ww;
-  ww.operation = requests;
-  auto resp = interpreter_context_->io
-                  .Request<io::rsm::WriteRequest<coordinator::CoordinatorWriteRequests>,
-                           io::rsm::WriteResponse<coordinator::CoordinatorWriteResponses>>(
-                      interpreter_context_->coordinator_address, ww)
-                  .Wait();
-  if (resp.HasValue()) {
-    const auto alloc_edge_id_reps =
-        std::get<coordinator::AllocateEdgeIdBatchResponse>(resp.GetValue().message.write_return);
-    interpreter_context_->edge_ids_alloc = {alloc_edge_id_reps.low, alloc_edge_id_reps.high};
+  const auto edge_ids_alloc_min_max_pair =
+      request_router_->AllocateInitialEdgeIds(interpreter_context_->coordinator_address);
+  if (edge_ids_alloc_min_max_pair) {
+    interpreter_context_->edge_ids_alloc = {edge_ids_alloc_min_max_pair->first, edge_ids_alloc_min_max_pair->second};
   }
 }
 
diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 985c9a90c..a83a26f11 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -17,6 +17,7 @@
 #include "coordinator/coordinator.hpp"
 #include "coordinator/coordinator_client.hpp"
 #include "io/local_transport/local_transport.hpp"
+#include "io/simulator/simulator_transport.hpp"
 #include "io/transport.hpp"
 #include "query/v2/auth_checker.hpp"
 #include "query/v2/bindings/cypher_main_visitor.hpp"
@@ -172,7 +173,8 @@ struct PreparedQuery {
 struct InterpreterContext {
   explicit InterpreterContext(storage::v3::Shard *db, InterpreterConfig config,
                               const std::filesystem::path &data_directory,
-                              io::Io<io::local_transport::LocalTransport> io, coordinator::Address coordinator_addr);
+                              std::unique_ptr<RequestRouterFactory> &&request_router_factory,
+                              coordinator::Address coordinator_addr);
 
   storage::v3::Shard *db;
 
@@ -188,26 +190,25 @@ struct InterpreterContext {
   const InterpreterConfig config;
   IdAllocator edge_ids_alloc;
 
-  // TODO (antaljanosbenjamin) Figure out an abstraction for io::Io to make it possible to construct an interpreter
-  // context with a simulator transport without templatizing it.
-  io::Io<io::local_transport::LocalTransport> io;
   coordinator::Address coordinator_address;
 
   storage::v3::LabelId NameToLabelId(std::string_view label_name) {
-    return storage::v3::LabelId::FromUint(query_id_mapper.NameToId(label_name));
+    return storage::v3::LabelId::FromUint(query_id_mapper_.NameToId(label_name));
   }
 
   storage::v3::PropertyId NameToPropertyId(std::string_view property_name) {
-    return storage::v3::PropertyId::FromUint(query_id_mapper.NameToId(property_name));
+    return storage::v3::PropertyId::FromUint(query_id_mapper_.NameToId(property_name));
   }
 
   storage::v3::EdgeTypeId NameToEdgeTypeId(std::string_view edge_type_name) {
-    return storage::v3::EdgeTypeId::FromUint(query_id_mapper.NameToId(edge_type_name));
+    return storage::v3::EdgeTypeId::FromUint(query_id_mapper_.NameToId(edge_type_name));
   }
 
+  std::unique_ptr<RequestRouterFactory> request_router_factory_;
+
  private:
   // TODO Replace with local map of labels, properties and edge type ids
-  storage::v3::NameIdMapper query_id_mapper;
+  storage::v3::NameIdMapper query_id_mapper_;
 };
 
 /// Function that is used to tell all active interpreters that they should stop
@@ -296,7 +297,7 @@ class Interpreter final {
    */
   void Abort();
 
-  const RequestRouterInterface *GetRequestRouter() const { return request_router_.get(); }
+  RequestRouterInterface *GetRequestRouter() { return request_router_.get(); }
 
  private:
   struct QueryExecution {
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 996272fdc..de63cd76e 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -11,6 +11,7 @@
 
 #pragma once
 
+#include <boost/uuid/uuid.hpp>
 #include <chrono>
 #include <deque>
 #include <iostream>
@@ -23,6 +24,7 @@
 #include <stdexcept>
 #include <thread>
 #include <unordered_map>
+#include <variant>
 #include <vector>
 
 #include "coordinator/coordinator.hpp"
@@ -31,6 +33,7 @@
 #include "coordinator/shard_map.hpp"
 #include "io/address.hpp"
 #include "io/errors.hpp"
+#include "io/local_transport/local_transport.hpp"
 #include "io/rsm/raft.hpp"
 #include "io/rsm/rsm_client.hpp"
 #include "io/rsm/shard_rsm.hpp"
@@ -124,6 +127,10 @@ class RequestRouterInterface {
   virtual std::optional<storage::v3::LabelId> MaybeNameToLabel(const std::string &name) const = 0;
   virtual bool IsPrimaryLabel(storage::v3::LabelId label) const = 0;
   virtual bool IsPrimaryKey(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const = 0;
+
+  virtual std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) {
+    return {};
+  }
 };
 
 // TODO(kostasrim)rename this class template
@@ -595,6 +602,23 @@ class RequestRouter : public RequestRouterInterface {
     edge_types_.StoreMapping(std::move(id_to_name));
   }
 
+  std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) override {
+    coordinator::CoordinatorWriteRequests requests{coordinator::AllocateEdgeIdBatchRequest{.batch_size = 1000000}};
+
+    io::rsm::WriteRequest<coordinator::CoordinatorWriteRequests> ww;
+    ww.operation = requests;
+    auto resp =
+        io_.template Request<io::rsm::WriteRequest<coordinator::CoordinatorWriteRequests>,
+                             io::rsm::WriteResponse<coordinator::CoordinatorWriteResponses>>(coordinator_address, ww)
+            .Wait();
+    if (resp.HasValue()) {
+      const auto alloc_edge_id_reps =
+          std::get<coordinator::AllocateEdgeIdBatchResponse>(resp.GetValue().message.write_return);
+      return std::make_pair(alloc_edge_id_reps.low, alloc_edge_id_reps.high);
+    }
+    return {};
+  }
+
   ShardMap shards_map_;
   storage::v3::NameIdMapper properties_;
   storage::v3::NameIdMapper edge_types_;
@@ -605,4 +629,89 @@ class RequestRouter : public RequestRouterInterface {
   coordinator::Hlc transaction_id_;
   // TODO(kostasrim) Add batch prefetching
 };
+
+class RequestRouterFactory {
+ protected:
+  using LocalTransport = io::Io<io::local_transport::LocalTransport>;
+  using SimulatorTransport = io::Io<io::simulator::SimulatorTransport>;
+
+  using LocalTransportHandlePtr = std::shared_ptr<io::local_transport::LocalTransportHandle>;
+  using SimulatorTransportHandlePtr = std::shared_ptr<io::simulator::SimulatorHandle>;
+
+  using TransportHandleVariant = std::variant<LocalTransportHandlePtr, SimulatorTransportHandlePtr>;
+
+  TransportHandleVariant transport_handle_;
+
+ public:
+  explicit RequestRouterFactory(const TransportHandleVariant &transport_handle) : transport_handle_(transport_handle) {}
+
+  RequestRouterFactory(const RequestRouterFactory &) = delete;
+  RequestRouterFactory &operator=(const RequestRouterFactory &) = delete;
+  RequestRouterFactory(RequestRouterFactory &&) = delete;
+  RequestRouterFactory &operator=(RequestRouterFactory &&) = delete;
+
+  virtual ~RequestRouterFactory() = default;
+
+  virtual TransportHandleVariant GetTransportHandle() { return transport_handle_; }
+
+  virtual std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
+      const coordinator::Address &coordinator_address) const noexcept = 0;
+};
+
+class LocalRequestRouterFactory : public RequestRouterFactory {
+ public:
+  explicit LocalRequestRouterFactory(const TransportHandleVariant &transport_handle)
+      : RequestRouterFactory(transport_handle) {}
+
+  std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
+      const coordinator::Address &coordinator_address) const noexcept override {
+    using TransportType = io::local_transport::LocalTransport;
+    auto actual_transport_handle = std::get<LocalTransportHandlePtr>(transport_handle_);
+
+    boost::uuids::uuid random_uuid;
+    io::Address unique_local_addr_query;
+
+    random_uuid = boost::uuids::uuid{boost::uuids::random_generator()()};
+    unique_local_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
+
+    TransportType local_transport(actual_transport_handle);
+    auto local_transport_io = io::Io<TransportType>(local_transport, unique_local_addr_query);
+
+    auto query_io = local_transport_io.ForkLocal(random_uuid);
+
+    return std::make_unique<RequestRouter<TransportType>>(
+        coordinator::CoordinatorClient<TransportType>(query_io, coordinator_address, {coordinator_address}),
+        std::move(local_transport_io));
+  }
+};
+
+class SimulatedRequestRouterFactory : public RequestRouterFactory {
+  mutable io::simulator::Simulator *simulator_;
+  coordinator::Address address_;
+
+ public:
+  explicit SimulatedRequestRouterFactory(io::simulator::Simulator &simulator, coordinator::Address address)
+      : RequestRouterFactory(simulator.GetSimulatorHandle()), simulator_(&simulator), address_(address) {}
+
+  std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
+      const coordinator::Address &coordinator_address) const noexcept override {
+    using TransportType = io::simulator::SimulatorTransport;
+    auto actual_transport_handle = std::get<SimulatorTransportHandlePtr>(transport_handle_);
+
+    boost::uuids::uuid random_uuid;
+    io::Address unique_local_addr_query;
+
+    // The simulated RR should not introduce stochastic behavior.
+    random_uuid = boost::uuids::uuid{3, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0};
+    unique_local_addr_query = {.unique_id = boost::uuids::uuid{4, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}};
+
+    auto io = simulator_->Register(unique_local_addr_query);
+    auto query_io = io.ForkLocal(random_uuid);
+
+    return std::make_unique<RequestRouter<TransportType>>(
+        coordinator::CoordinatorClient<TransportType>(query_io, coordinator_address, {coordinator_address}),
+        std::move(io));
+  }
+};
+
 }  // namespace memgraph::query::v2
diff --git a/tests/simulation/CMakeLists.txt b/tests/simulation/CMakeLists.txt
index 9e1a4c71e..f81f89798 100644
--- a/tests/simulation/CMakeLists.txt
+++ b/tests/simulation/CMakeLists.txt
@@ -17,7 +17,7 @@ function(add_simulation_test test_cpp)
   # requires unique logical target names
   set_target_properties(${target_name} PROPERTIES OUTPUT_NAME ${exec_name})
 
-  target_link_libraries(${target_name} mg-storage-v3 mg-communication mg-utils mg-io mg-io-simulator mg-coordinator mg-query-v2)
+  target_link_libraries(${target_name} mg-communication mg-utils mg-io mg-io-simulator mg-coordinator mg-query-v2 mg-storage-v3)
   target_link_libraries(${target_name} Boost::headers)
   target_link_libraries(${target_name} gtest gtest_main gmock rapidcheck rapidcheck_gtest)
 
@@ -32,3 +32,4 @@ add_simulation_test(trial_query_storage/query_storage_test.cpp)
 add_simulation_test(sharded_map.cpp)
 add_simulation_test(shard_rsm.cpp)
 add_simulation_test(cluster_property_test.cpp)
+add_simulation_test(cluster_property_test_v2.cpp)
diff --git a/tests/simulation/cluster_property_test_v2.cpp b/tests/simulation/cluster_property_test_v2.cpp
new file mode 100644
index 000000000..2f121647b
--- /dev/null
+++ b/tests/simulation/cluster_property_test_v2.cpp
@@ -0,0 +1,65 @@
+// Copyright 2022 Memgraph Ltd.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
+// License, and you may not use this file except in compliance with the Business Source License.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+// This test serves as an example of a property-based model test.
+// It generates a cluster configuration and a set of operations to
+// apply against both the real system and a greatly simplified model.
+
+#include <chrono>
+
+#include <gtest/gtest.h>
+#include <rapidcheck.h>
+#include <rapidcheck/gtest.h>
+#include <spdlog/cfg/env.h>
+
+#include "generated_operations.hpp"
+#include "io/simulator/simulator_config.hpp"
+#include "io/time.hpp"
+#include "storage/v3/shard_manager.hpp"
+#include "test_cluster.hpp"
+
+namespace memgraph::tests::simulation {
+
+using io::Duration;
+using io::Time;
+using io::simulator::SimulatorConfig;
+using storage::v3::kMaximumCronInterval;
+
+RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, NonEmptyOpVec ops, uint64_t rng_seed)) {
+  spdlog::cfg::load_env_levels();
+
+  SimulatorConfig sim_config{
+      .drop_percent = 0,
+      .perform_timeouts = false,
+      .scramble_messages = true,
+      .rng_seed = rng_seed,
+      .start_time = Time::min(),
+      // TODO(tyler) set abort_time to something more restrictive than Time::max()
+      .abort_time = Time::max(),
+  };
+
+  std::vector<std::string> queries = {"CREATE (n:test_label{property_1: 0, property_2: 0});", "MATCH (n) RETURN n;"};
+
+  auto [sim_stats_1, latency_stats_1] = RunClusterSimulationWithQueries(sim_config, cluster_config, queries);
+  auto [sim_stats_2, latency_stats_2] = RunClusterSimulationWithQueries(sim_config, cluster_config, queries);
+
+  if (latency_stats_1 != latency_stats_2) {
+    spdlog::error("simulator stats diverged across runs");
+    spdlog::error("run 1 simulator stats: {}", sim_stats_1);
+    spdlog::error("run 2 simulator stats: {}", sim_stats_2);
+    spdlog::error("run 1 latency:\n{}", latency_stats_1.SummaryTable());
+    spdlog::error("run 2 latency:\n{}", latency_stats_2.SummaryTable());
+    RC_ASSERT(latency_stats_1 == latency_stats_2);
+    RC_ASSERT(sim_stats_1 == sim_stats_2);
+  }
+}
+
+}  // namespace memgraph::tests::simulation
diff --git a/tests/simulation/simulation_interpreter.hpp b/tests/simulation/simulation_interpreter.hpp
new file mode 100644
index 000000000..b59f335be
--- /dev/null
+++ b/tests/simulation/simulation_interpreter.hpp
@@ -0,0 +1,95 @@
+// Copyright 2022 Memgraph Ltd.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
+// License, and you may not use this file except in compliance with the Business Source License.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+#include <machine_manager/machine_config.hpp>
+#include <machine_manager/machine_manager.hpp>
+#include "io/simulator/simulator_handle.hpp"
+#include "query/v2/config.hpp"
+#include "query/v2/discard_value_stream.hpp"
+#include "query/v2/frontend/ast/ast.hpp"
+#include "query/v2/interpreter.hpp"
+#include "query/v2/request_router.hpp"
+
+#include <memory>
+
+// TODO(gvolfing)
+// -How to set up the entire raft cluster with the QE. Also provide abrstraction for that.
+// -Pass an argument to the setup to determine, how many times the retry of a query should happen.
+
+namespace memgraph::io::simulator {
+
+class SimulatedInterpreter {
+  using ResultStream = query::v2::DiscardValueResultStream;
+
+ public:
+  explicit SimulatedInterpreter(std::unique_ptr<query::v2::InterpreterContext> &&interpreter_context)
+      : interpreter_context_(std::move(interpreter_context)) {
+    interpreter_ = std::make_unique<memgraph::query::v2::Interpreter>(&(*interpreter_context_));
+  }
+
+  SimulatedInterpreter(const SimulatedInterpreter &) = delete;
+  SimulatedInterpreter &operator=(const SimulatedInterpreter &) = delete;
+  SimulatedInterpreter(SimulatedInterpreter &&) = delete;
+  SimulatedInterpreter &operator=(SimulatedInterpreter &&) = delete;
+  ~SimulatedInterpreter() = default;
+
+  std::vector<ResultStream> RunQueries(const std::vector<std::string> &queries) {
+    std::vector<ResultStream> results;
+    results.reserve(queries.size());
+
+    for (const auto &query : queries) {
+      results.emplace_back(RunQuery(query));
+    }
+    return results;
+  }
+
+ private:
+  ResultStream RunQuery(const std::string &query) {
+    ResultStream stream;
+
+    std::map<std::string, memgraph::storage::v3::PropertyValue> params;
+    const std::string *username = nullptr;
+
+    interpreter_->BeginTransaction();
+
+    auto *rr = interpreter_->GetRequestRouter();
+    rr->StartTransaction();
+
+    interpreter_->Prepare(query, params, username);
+    interpreter_->PullAll(&stream);
+    interpreter_->CommitTransaction();
+
+    return stream;
+  }
+
+  std::unique_ptr<query::v2::InterpreterContext> interpreter_context_;
+  std::unique_ptr<query::v2::Interpreter> interpreter_;
+};
+
+SimulatedInterpreter SetUpInterpreter(Address coordinator_address, Simulator &simulator) {
+  auto rr_factory =
+      std::make_unique<memgraph::query::v2::SimulatedRequestRouterFactory>(simulator, coordinator_address);
+
+  auto interpreter_context = std::make_unique<memgraph::query::v2::InterpreterContext>(
+      (memgraph::storage::v3::Shard *)(nullptr),
+      memgraph::query::v2::InterpreterConfig{.query = {.allow_load_csv = true},
+                                             .execution_timeout_sec = 600,
+                                             .replication_replica_check_frequency = std::chrono::seconds(1),
+                                             .default_kafka_bootstrap_servers = "",
+                                             .default_pulsar_service_url = "",
+                                             .stream_transaction_conflict_retries = 30,
+                                             .stream_transaction_retry_interval = std::chrono::milliseconds(500)},
+      std::filesystem::path("mg_data"), std::move(rr_factory), coordinator_address);
+
+  return SimulatedInterpreter(std::move(interpreter_context));
+}
+
+}  // namespace memgraph::io::simulator
diff --git a/tests/simulation/test_cluster.hpp b/tests/simulation/test_cluster.hpp
index 1392a0632..5aa792c16 100644
--- a/tests/simulation/test_cluster.hpp
+++ b/tests/simulation/test_cluster.hpp
@@ -36,6 +36,8 @@
 #include "utils/print_helpers.hpp"
 #include "utils/variant_helpers.hpp"
 
+#include "simulation_interpreter.hpp"
+
 namespace memgraph::tests::simulation {
 
 using coordinator::Coordinator;
@@ -277,4 +279,64 @@ std::pair<SimulatorStats, LatencyHistogramSummaries> RunClusterSimulation(const
   return std::make_pair(stats, histo);
 }
 
+std::pair<SimulatorStats, LatencyHistogramSummaries> RunClusterSimulationWithQueries(
+    const SimulatorConfig &sim_config, const ClusterConfig &cluster_config, const std::vector<std::string> &queries) {
+  spdlog::info("========================== NEW SIMULATION ==========================");
+
+  auto simulator = Simulator(sim_config);
+
+  auto machine_1_addr = Address::TestAddress(1);
+  auto cli_addr = Address::TestAddress(2);
+  auto cli_addr_2 = Address::TestAddress(3);
+
+  Io<SimulatorTransport> cli_io = simulator.Register(cli_addr);
+  Io<SimulatorTransport> cli_io_2 = simulator.Register(cli_addr_2);
+
+  auto coordinator_addresses = std::vector{
+      machine_1_addr,
+  };
+
+  ShardMap initialization_sm = TestShardMap(cluster_config.shards - 1, cluster_config.replication_factor);
+
+  auto mm_1 = MkMm(simulator, coordinator_addresses, machine_1_addr, initialization_sm);
+  Address coordinator_address = mm_1.CoordinatorAddress();
+
+  auto mm_thread_1 = std::jthread(RunMachine, std::move(mm_1));
+  simulator.IncrementServerCountAndWaitForQuiescentState(machine_1_addr);
+
+  auto detach_on_error = DetachIfDropped{.handle = mm_thread_1};
+
+  // TODO(tyler) clarify addresses of coordinator etc... as it's a mess
+
+  CoordinatorClient<SimulatorTransport> coordinator_client(cli_io, coordinator_address, {coordinator_address});
+  WaitForShardsToInitialize(coordinator_client);
+
+  auto simulated_interpreter = io::simulator::SetUpInterpreter(coordinator_address, simulator);
+
+  auto query_results = simulated_interpreter.RunQueries(queries);
+
+  // We have now completed our workload without failing any assertions, so we can
+  // disable detaching the worker thread, which will cause the mm_thread_1 jthread
+  // to be joined when this function returns.
+  detach_on_error.detach = false;
+
+  simulator.ShutDown();
+
+  mm_thread_1.join();
+
+  SimulatorStats stats = simulator.Stats();
+
+  spdlog::info("total messages:     {}", stats.total_messages);
+  spdlog::info("dropped messages:   {}", stats.dropped_messages);
+  spdlog::info("timed out requests: {}", stats.timed_out_requests);
+  spdlog::info("total requests:     {}", stats.total_requests);
+  spdlog::info("total responses:    {}", stats.total_responses);
+  spdlog::info("simulator ticks:    {}", stats.simulator_ticks);
+
+  auto histo = cli_io_2.ResponseLatencies();
+
+  spdlog::info("========================== SUCCESS :) ==========================");
+  return std::make_pair(stats, histo);
+}
+
 }  // namespace memgraph::tests::simulation

From af812d1311f75d936e1dd18c4dbec047ffc9c986 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Tue, 13 Dec 2022 09:05:39 +0100
Subject: [PATCH 02/90] Implement scanAll MultiFrame version

---
 src/query/v2/multiframe.hpp    |  5 +-
 src/query/v2/plan/operator.cpp | 85 ++++++++++++++++++++++++++++++----
 2 files changed, 77 insertions(+), 13 deletions(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index e13eb07ac..3e063bdde 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -201,10 +201,9 @@ class ValidFramesConsumer {
 
   ~ValidFramesConsumer() noexcept;
   ValidFramesConsumer(const ValidFramesConsumer &other) = delete;
-  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete;
+  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = default;
   ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete;
-  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete;
-
+  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = default;
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;
     using difference_type = std::ptrdiff_t;
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 09c0837c0..804312800 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -37,6 +37,7 @@
 #include "query/v2/db_accessor.hpp"
 #include "query/v2/exceptions.hpp"
 #include "query/v2/frontend/ast/ast.hpp"
+#include "query/v2/multiframe.hpp"
 #include "query/v2/path.hpp"
 #include "query/v2/plan/scoped_profile.hpp"
 #include "query/v2/request_router.hpp"
@@ -473,11 +474,11 @@ class DistributedScanAllAndFilterCursor : public Cursor {
       if (label_.has_value()) {
         request_label = request_router.LabelToName(*label_);
       }
-      current_batch = request_router.ScanVertices(request_label);
+      current_batch_ = request_router.ScanVertices(request_label);
     }
-    current_vertex_it = current_batch.begin();
+    current_vertex_it_ = current_batch_.begin();
     request_state_ = State::COMPLETED;
-    return !current_batch.empty();
+    return !current_batch_.empty();
   }
 
   bool Pull(Frame &frame, ExecutionContext &context) override {
@@ -495,23 +496,85 @@ class DistributedScanAllAndFilterCursor : public Cursor {
         }
       }
 
-      if (current_vertex_it == current_batch.end() &&
+      if (current_vertex_it_ == current_batch_.end() &&
           (request_state_ == State::COMPLETED || !MakeRequest(request_router, context))) {
         ResetExecutionState();
         continue;
       }
 
-      frame[output_symbol_] = TypedValue(std::move(*current_vertex_it));
-      ++current_vertex_it;
+      frame[output_symbol_] = TypedValue(std::move(*current_vertex_it_));
+      ++current_vertex_it_;
       return true;
     }
   }
 
+  void Generate(ExecutionContext &context) {
+    auto &request_router = *context.request_router;
+
+    input_cursor_->PullMultiple(*own_multi_frames_, context);
+
+    if (!MakeRequest(request_router, context)) {
+      return;
+    }
+
+    auto valid_frames_consumer = own_multi_frames_->GetValidFramesConsumer();
+    auto valid_frames_it = valid_frames_consumer.begin();
+
+    for (auto valid_frames_it = valid_frames_consumer.begin(); valid_frames_it != valid_frames_consumer.end();
+         ++valid_frames_it) {
+      for (auto vertex_it = current_batch_.begin(); vertex_it != current_batch_.end(); ++vertex_it) {
+        auto frame = *valid_frames_it;
+        frame[output_symbol_] = TypedValue(*vertex_it);
+        frames_buffer_.push(std::move(frame));
+      }
+      valid_frames_it->MakeInvalid();
+    }
+  }
+
+  void PullMultiple(MultiFrame &input_multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP(op_name_);
+
+    if (!own_multi_frames_.has_value()) {
+      own_multi_frames_.emplace(MultiFrame(input_multi_frame.GetFirstFrame().elems().size(),
+                                           kNumberOfFramesInMultiframe, input_multi_frame.GetMemoryResource()));
+    }
+
+    auto &request_router = *context.request_router;
+    auto should_make_request = false;
+    auto should_pull = false;
+    auto should_generate = false;
+
+    while (true) {
+      if (MustAbort(context)) {
+        throw HintedAbortError();
+      }
+
+      const auto should_generate = frames_buffer_.empty();
+      if (should_generate) {
+        Generate(context);
+      }
+
+      auto invalid_frames_populator = input_multi_frame.GetInvalidFramesPopulator();
+      auto invalid_frame_it = invalid_frames_populator.begin();
+      auto has_modified_at_least_one_frame = false;
+      while (invalid_frames_populator.end() != invalid_frame_it && !frames_buffer_.empty()) {
+        has_modified_at_least_one_frame = true;
+        *invalid_frame_it = frames_buffer_.front();
+        ++invalid_frame_it;
+        frames_buffer_.pop();
+      }
+
+      if (!has_modified_at_least_one_frame) {
+        return;
+      }
+    }
+  };
+
   void Shutdown() override { input_cursor_->Shutdown(); }
 
   void ResetExecutionState() {
-    current_batch.clear();
-    current_vertex_it = current_batch.end();
+    current_batch_.clear();
+    current_vertex_it_ = current_batch_.end();
     request_state_ = State::INITIALIZING;
   }
 
@@ -524,12 +587,14 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   const Symbol output_symbol_;
   const UniqueCursorPtr input_cursor_;
   const char *op_name_;
-  std::vector<VertexAccessor> current_batch;
-  std::vector<VertexAccessor>::iterator current_vertex_it;
+  std::vector<VertexAccessor> current_batch_;
+  std::vector<VertexAccessor>::iterator current_vertex_it_;
   State request_state_ = State::INITIALIZING;
   std::optional<storage::v3::LabelId> label_;
   std::optional<std::pair<storage::v3::PropertyId, Expression *>> property_expression_pair_;
   std::optional<std::vector<Expression *>> filter_expressions_;
+  std::optional<MultiFrame> own_multi_frames_;
+  std::queue<FrameWithValidity> frames_buffer_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From ac16348fff3d5f567d90faae944540b02b2821f4 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Tue, 13 Dec 2022 09:50:42 +0100
Subject: [PATCH 03/90] Remove unused variable

---
 src/query/v2/plan/operator.cpp | 6 ------
 1 file changed, 6 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 804312800..1569840c4 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -518,7 +518,6 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     }
 
     auto valid_frames_consumer = own_multi_frames_->GetValidFramesConsumer();
-    auto valid_frames_it = valid_frames_consumer.begin();
 
     for (auto valid_frames_it = valid_frames_consumer.begin(); valid_frames_it != valid_frames_consumer.end();
          ++valid_frames_it) {
@@ -539,11 +538,6 @@ class DistributedScanAllAndFilterCursor : public Cursor {
                                            kNumberOfFramesInMultiframe, input_multi_frame.GetMemoryResource()));
     }
 
-    auto &request_router = *context.request_router;
-    auto should_make_request = false;
-    auto should_pull = false;
-    auto should_generate = false;
-
     while (true) {
       if (MustAbort(context)) {
         throw HintedAbortError();

From 83306d21defd278ca8f0bba7edc71b685338e832 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Tue, 13 Dec 2022 09:50:50 +0100
Subject: [PATCH 04/90] Revert changes

---
 src/query/v2/multiframe.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index 3e063bdde..d61755857 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -201,9 +201,9 @@ class ValidFramesConsumer {
 
   ~ValidFramesConsumer() noexcept;
   ValidFramesConsumer(const ValidFramesConsumer &other) = delete;
-  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = default;
+  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete;
   ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete;
-  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = default;
+  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete;
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;
     using difference_type = std::ptrdiff_t;

From 54ce79baa02b281c8ca96d41c2c69dc0ef54c0d2 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Tue, 13 Dec 2022 09:51:42 +0100
Subject: [PATCH 05/90] Add empty line

---
 src/query/v2/multiframe.hpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index d61755857..e13eb07ac 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -204,6 +204,7 @@ class ValidFramesConsumer {
   ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete;
   ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete;
   ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete;
+
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;
     using difference_type = std::ptrdiff_t;

From 311994a36ddda5ed7cddaff9fd364e3b96ba4f44 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Thu, 15 Dec 2022 14:31:43 +0100
Subject: [PATCH 06/90] Impl of version more memory friendly

---
 src/query/v2/multiframe.hpp    |  6 ++--
 src/query/v2/plan/operator.cpp | 54 +++++++++++++++++++++-------------
 2 files changed, 36 insertions(+), 24 deletions(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index e13eb07ac..aeacda7d3 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -200,9 +200,9 @@ class ValidFramesConsumer {
   explicit ValidFramesConsumer(MultiFrame &multiframe);
 
   ~ValidFramesConsumer() noexcept;
-  ValidFramesConsumer(const ValidFramesConsumer &other) = delete;
-  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = delete;
-  ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete;
+  ValidFramesConsumer(const ValidFramesConsumer &other) = default;
+  ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = default;
+  ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = default;
   ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete;
 
   struct Iterator {
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 1569840c4..3e8d533c5 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -508,26 +508,39 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     }
   }
 
-  void Generate(ExecutionContext &context) {
+  void PrepareNextFrames(ExecutionContext &context) {
     auto &request_router = *context.request_router;
 
     input_cursor_->PullMultiple(*own_multi_frames_, context);
+    valid_frames_consumer_ = own_multi_frames_->GetValidFramesConsumer();
+    valid_frames_it_ = valid_frames_consumer_->begin();
 
-    if (!MakeRequest(request_router, context)) {
-      return;
-    }
+    MakeRequest(request_router, context);
 
-    auto valid_frames_consumer = own_multi_frames_->GetValidFramesConsumer();
+    has_next_frame_ = current_vertex_it_ != current_batch_.end() && valid_frames_it_ != valid_frames_consumer_->end();
+  }
 
-    for (auto valid_frames_it = valid_frames_consumer.begin(); valid_frames_it != valid_frames_consumer.end();
-         ++valid_frames_it) {
-      for (auto vertex_it = current_batch_.begin(); vertex_it != current_batch_.end(); ++vertex_it) {
-        auto frame = *valid_frames_it;
-        frame[output_symbol_] = TypedValue(*vertex_it);
-        frames_buffer_.push(std::move(frame));
+  inline bool HasNextFrame() { return has_next_frame_; }
+
+  FrameWithValidity GetNextFrame(ExecutionContext &context) {
+    MG_ASSERT(HasNextFrame());
+
+    auto frame = *valid_frames_it_;
+    frame[output_symbol_] = TypedValue(*current_vertex_it_);
+
+    ++current_vertex_it_;
+    if (current_vertex_it_ == current_batch_.end()) {
+      valid_frames_it_->MakeInvalid();
+      ++valid_frames_it_;
+
+      if (valid_frames_it_ == valid_frames_consumer_->end()) {
+        PrepareNextFrames(context);
+      } else {
+        current_vertex_it_ = current_batch_.begin();
       }
-      valid_frames_it->MakeInvalid();
-    }
+    };
+
+    return frame;
   }
 
   void PullMultiple(MultiFrame &input_multi_frame, ExecutionContext &context) override {
@@ -536,6 +549,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     if (!own_multi_frames_.has_value()) {
       own_multi_frames_.emplace(MultiFrame(input_multi_frame.GetFirstFrame().elems().size(),
                                            kNumberOfFramesInMultiframe, input_multi_frame.GetMemoryResource()));
+      PrepareNextFrames(context);
     }
 
     while (true) {
@@ -543,19 +557,14 @@ class DistributedScanAllAndFilterCursor : public Cursor {
         throw HintedAbortError();
       }
 
-      const auto should_generate = frames_buffer_.empty();
-      if (should_generate) {
-        Generate(context);
-      }
-
       auto invalid_frames_populator = input_multi_frame.GetInvalidFramesPopulator();
       auto invalid_frame_it = invalid_frames_populator.begin();
       auto has_modified_at_least_one_frame = false;
-      while (invalid_frames_populator.end() != invalid_frame_it && !frames_buffer_.empty()) {
+
+      while (invalid_frames_populator.end() != invalid_frame_it && HasNextFrame()) {
         has_modified_at_least_one_frame = true;
-        *invalid_frame_it = frames_buffer_.front();
+        *invalid_frame_it = GetNextFrame(context);
         ++invalid_frame_it;
-        frames_buffer_.pop();
       }
 
       if (!has_modified_at_least_one_frame) {
@@ -588,7 +597,10 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   std::optional<std::pair<storage::v3::PropertyId, Expression *>> property_expression_pair_;
   std::optional<std::vector<Expression *>> filter_expressions_;
   std::optional<MultiFrame> own_multi_frames_;
+  std::optional<ValidFramesConsumer> valid_frames_consumer_;
+  ValidFramesConsumer::Iterator valid_frames_it_;
   std::queue<FrameWithValidity> frames_buffer_;
+  bool has_next_frame_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From ae57fa31999a998676f750296ee7ec6c254eb334 Mon Sep 17 00:00:00 2001
From: gvolfing <107616712+gvolfing@users.noreply.github.com>
Date: Thu, 15 Dec 2022 15:25:46 +0100
Subject: [PATCH 07/90] Apply suggestions from code review

Co-authored-by: Kostas Kyrimis  <kostaskyrim@gmail.com>
---
 src/io/local_transport/local_system.hpp       |  2 +-
 src/io/simulator/simulator.hpp                |  2 +-
 src/query/v2/interpreter.cpp                  |  6 +++---
 src/query/v2/interpreter.hpp                  |  2 +-
 src/query/v2/request_router.hpp               |  8 ++++----
 tests/simulation/cluster_property_test_v2.cpp |  1 -
 tests/simulation/simulation_interpreter.hpp   | 13 +++++++------
 7 files changed, 17 insertions(+), 17 deletions(-)

diff --git a/src/io/local_transport/local_system.hpp b/src/io/local_transport/local_system.hpp
index ec148b50b..feea44244 100644
--- a/src/io/local_transport/local_system.hpp
+++ b/src/io/local_transport/local_system.hpp
@@ -29,7 +29,7 @@ class LocalSystem {
     return Io{local_transport, address};
   }
 
-  std::shared_ptr<LocalTransportHandle> &GetTransportHandle() { return local_transport_handle_; }
+  std::shared_ptr<LocalTransportHandle> GetTransportHandle() const { return local_transport_handle_; }
 
   void ShutDown() { local_transport_handle_->ShutDown(); }
 };
diff --git a/src/io/simulator/simulator.hpp b/src/io/simulator/simulator.hpp
index 5095ab58b..87ced7180 100644
--- a/src/io/simulator/simulator.hpp
+++ b/src/io/simulator/simulator.hpp
@@ -50,6 +50,6 @@ class Simulator {
 
   SimulatorStats Stats() { return simulator_handle_->Stats(); }
 
-  std::shared_ptr<SimulatorHandle> GetSimulatorHandle() { return simulator_handle_; }
+  std::shared_ptr<SimulatorHandle> GetSimulatorHandle() const { return simulator_handle_; }
 };
 };  // namespace memgraph::io::simulator
diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index c428d1a39..8042a763c 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -793,7 +793,7 @@ using RWType = plan::ReadWriteTypeChecker::RWType;
 
 InterpreterContext::InterpreterContext(storage::v3::Shard *db, const InterpreterConfig config,
                                        const std::filesystem::path & /*data_directory*/,
-                                       std::unique_ptr<RequestRouterFactory> &&request_router_factory,
+                                       std::unique_ptr<RequestRouterFactory> request_router_factory,
                                        coordinator::Address coordinator_addr)
     : db(db),
       config(config),
@@ -807,10 +807,10 @@ Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_
       interpreter_context_->request_router_factory_->CreateRequestRouter(interpreter_context_->coordinator_address);
 
   // Get edge ids
-  const auto edge_ids_alloc_min_max_pair =
+  const auto [min, max] =
       request_router_->AllocateInitialEdgeIds(interpreter_context_->coordinator_address);
   if (edge_ids_alloc_min_max_pair) {
-    interpreter_context_->edge_ids_alloc = {edge_ids_alloc_min_max_pair->first, edge_ids_alloc_min_max_pair->second};
+    interpreter_context_->edge_ids_alloc = {min, max}
   }
 }
 
diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index a83a26f11..05b05dd8e 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -173,7 +173,7 @@ struct PreparedQuery {
 struct InterpreterContext {
   explicit InterpreterContext(storage::v3::Shard *db, InterpreterConfig config,
                               const std::filesystem::path &data_directory,
-                              std::unique_ptr<RequestRouterFactory> &&request_router_factory,
+                              std::unique_ptr<RequestRouterFactory> request_router_factory,
                               coordinator::Address coordinator_addr);
 
   storage::v3::Shard *db;
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index de63cd76e..7ac5d1816 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -655,7 +655,7 @@ class RequestRouterFactory {
   virtual TransportHandleVariant GetTransportHandle() { return transport_handle_; }
 
   virtual std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
-      const coordinator::Address &coordinator_address) const noexcept = 0;
+      const coordinator::Address &coordinator_address) const = 0;
 };
 
 class LocalRequestRouterFactory : public RequestRouterFactory {
@@ -664,7 +664,7 @@ class LocalRequestRouterFactory : public RequestRouterFactory {
       : RequestRouterFactory(transport_handle) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
-      const coordinator::Address &coordinator_address) const noexcept override {
+      const coordinator::Address &coordinator_address) const override {
     using TransportType = io::local_transport::LocalTransport;
     auto actual_transport_handle = std::get<LocalTransportHandlePtr>(transport_handle_);
 
@@ -686,7 +686,7 @@ class LocalRequestRouterFactory : public RequestRouterFactory {
 };
 
 class SimulatedRequestRouterFactory : public RequestRouterFactory {
-  mutable io::simulator::Simulator *simulator_;
+  io::simulator::Simulator *simulator_;
   coordinator::Address address_;
 
  public:
@@ -694,7 +694,7 @@ class SimulatedRequestRouterFactory : public RequestRouterFactory {
       : RequestRouterFactory(simulator.GetSimulatorHandle()), simulator_(&simulator), address_(address) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
-      const coordinator::Address &coordinator_address) const noexcept override {
+      const coordinator::Address &coordinator_address) const override {
     using TransportType = io::simulator::SimulatorTransport;
     auto actual_transport_handle = std::get<SimulatorTransportHandlePtr>(transport_handle_);
 
diff --git a/tests/simulation/cluster_property_test_v2.cpp b/tests/simulation/cluster_property_test_v2.cpp
index 2f121647b..4996b4c2f 100644
--- a/tests/simulation/cluster_property_test_v2.cpp
+++ b/tests/simulation/cluster_property_test_v2.cpp
@@ -42,7 +42,6 @@ RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, Non
       .scramble_messages = true,
       .rng_seed = rng_seed,
       .start_time = Time::min(),
-      // TODO(tyler) set abort_time to something more restrictive than Time::max()
       .abort_time = Time::max(),
   };
 
diff --git a/tests/simulation/simulation_interpreter.hpp b/tests/simulation/simulation_interpreter.hpp
index b59f335be..e25dfaa99 100644
--- a/tests/simulation/simulation_interpreter.hpp
+++ b/tests/simulation/simulation_interpreter.hpp
@@ -9,8 +9,8 @@
 // by the Apache License, Version 2.0, included in the file
 // licenses/APL.txt.
 
-#include <machine_manager/machine_config.hpp>
-#include <machine_manager/machine_manager.hpp>
+#include "machine_manager/machine_config.hpp"
+#include "machine_manager/machine_manager.hpp"
 #include "io/simulator/simulator_handle.hpp"
 #include "query/v2/config.hpp"
 #include "query/v2/discard_value_stream.hpp"
@@ -18,7 +18,8 @@
 #include "query/v2/interpreter.hpp"
 #include "query/v2/request_router.hpp"
 
-#include <memory>
+#include <vector>
+#include <string>
 
 // TODO(gvolfing)
 // -How to set up the entire raft cluster with the QE. Also provide abrstraction for that.
@@ -30,9 +31,9 @@ class SimulatedInterpreter {
   using ResultStream = query::v2::DiscardValueResultStream;
 
  public:
-  explicit SimulatedInterpreter(std::unique_ptr<query::v2::InterpreterContext> &&interpreter_context)
+  explicit SimulatedInterpreter(std::unique_ptr<query::v2::InterpreterContext> interpreter_context)
       : interpreter_context_(std::move(interpreter_context)) {
-    interpreter_ = std::make_unique<memgraph::query::v2::Interpreter>(&(*interpreter_context_));
+    interpreter_ = std::make_unique<memgraph::query::v2::Interpreter>(interpreter_context_);
   }
 
   SimulatedInterpreter(const SimulatedInterpreter &) = delete;
@@ -79,7 +80,7 @@ SimulatedInterpreter SetUpInterpreter(Address coordinator_address, Simulator &si
       std::make_unique<memgraph::query::v2::SimulatedRequestRouterFactory>(simulator, coordinator_address);
 
   auto interpreter_context = std::make_unique<memgraph::query::v2::InterpreterContext>(
-      (memgraph::storage::v3::Shard *)(nullptr),
+      nullptr
       memgraph::query::v2::InterpreterConfig{.query = {.allow_load_csv = true},
                                              .execution_timeout_sec = 600,
                                              .replication_replica_check_frequency = std::chrono::seconds(1),

From 1aa40e5e3fad1e9b4a482157988ec8bd355a3ed0 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Thu, 15 Dec 2022 16:24:45 +0100
Subject: [PATCH 08/90] Add const to method

---
 src/query/v2/plan/operator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 3e8d533c5..b6bbf9ce1 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -520,7 +520,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     has_next_frame_ = current_vertex_it_ != current_batch_.end() && valid_frames_it_ != valid_frames_consumer_->end();
   }
 
-  inline bool HasNextFrame() { return has_next_frame_; }
+  inline bool HasNextFrame() const { return has_next_frame_; }
 
   FrameWithValidity GetNextFrame(ExecutionContext &context) {
     MG_ASSERT(HasNextFrame());

From fa39c6740b96d96bda49960f46656cc80519329d Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Thu, 15 Dec 2022 17:02:01 +0100
Subject: [PATCH 09/90] Apply review comments

---
 src/memgraph.cpp                                      | 10 +++++-----
 src/query/v2/interpreter.cpp                          |  4 ++--
 src/query/v2/request_router.hpp                       |  5 ++---
 tests/simulation/CMakeLists.txt                       |  2 +-
 ...2.cpp => cluster_property_test_cypher_queries.cpp} |  0
 tests/simulation/simulation_interpreter.hpp           | 11 +++++------
 6 files changed, 15 insertions(+), 17 deletions(-)
 rename tests/simulation/{cluster_property_test_v2.cpp => cluster_property_test_cypher_queries.cpp} (100%)

diff --git a/src/memgraph.cpp b/src/memgraph.cpp
index ca03024fe..cdb1e63df 100644
--- a/src/memgraph.cpp
+++ b/src/memgraph.cpp
@@ -607,15 +607,15 @@ int main(int argc, char **argv) {
   // to minimize the impact of their failure on the main storage.
 
   memgraph::io::local_transport::LocalSystem ls;
-  auto unique_local_coord_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
-  auto io = ls.Register(unique_local_coord_addr_query);
+  auto unique_local_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
+  auto io = ls.Register(unique_local_addr_query);
 
   memgraph::machine_manager::MachineConfig config{
-      .coordinator_addresses = std::vector<memgraph::io::Address>{unique_local_coord_addr_query},
+      .coordinator_addresses = std::vector<memgraph::io::Address>{unique_local_addr_query},
       .is_storage = true,
       .is_coordinator = true,
-      .listen_ip = unique_local_coord_addr_query.last_known_ip,
-      .listen_port = unique_local_coord_addr_query.last_known_port,
+      .listen_ip = unique_local_addr_query.last_known_ip,
+      .listen_port = unique_local_addr_query.last_known_port,
   };
 
   memgraph::coordinator::ShardMap sm;
diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 8042a763c..46a479f36 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -807,10 +807,10 @@ Interpreter::Interpreter(InterpreterContext *interpreter_context) : interpreter_
       interpreter_context_->request_router_factory_->CreateRequestRouter(interpreter_context_->coordinator_address);
 
   // Get edge ids
-  const auto [min, max] =
+  const auto edge_ids_alloc_min_max_pair =
       request_router_->AllocateInitialEdgeIds(interpreter_context_->coordinator_address);
   if (edge_ids_alloc_min_max_pair) {
-    interpreter_context_->edge_ids_alloc = {min, max}
+    interpreter_context_->edge_ids_alloc = {edge_ids_alloc_min_max_pair->first, edge_ids_alloc_min_max_pair->second};
   }
 }
 
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 7ac5d1816..74117f52e 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -687,11 +687,10 @@ class LocalRequestRouterFactory : public RequestRouterFactory {
 
 class SimulatedRequestRouterFactory : public RequestRouterFactory {
   io::simulator::Simulator *simulator_;
-  coordinator::Address address_;
 
  public:
-  explicit SimulatedRequestRouterFactory(io::simulator::Simulator &simulator, coordinator::Address address)
-      : RequestRouterFactory(simulator.GetSimulatorHandle()), simulator_(&simulator), address_(address) {}
+  explicit SimulatedRequestRouterFactory(io::simulator::Simulator &simulator)
+      : RequestRouterFactory(simulator.GetSimulatorHandle()), simulator_(&simulator) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
       const coordinator::Address &coordinator_address) const override {
diff --git a/tests/simulation/CMakeLists.txt b/tests/simulation/CMakeLists.txt
index f81f89798..28ff60b02 100644
--- a/tests/simulation/CMakeLists.txt
+++ b/tests/simulation/CMakeLists.txt
@@ -32,4 +32,4 @@ add_simulation_test(trial_query_storage/query_storage_test.cpp)
 add_simulation_test(sharded_map.cpp)
 add_simulation_test(shard_rsm.cpp)
 add_simulation_test(cluster_property_test.cpp)
-add_simulation_test(cluster_property_test_v2.cpp)
+add_simulation_test(cluster_property_test_cypher_queries.cpp)
diff --git a/tests/simulation/cluster_property_test_v2.cpp b/tests/simulation/cluster_property_test_cypher_queries.cpp
similarity index 100%
rename from tests/simulation/cluster_property_test_v2.cpp
rename to tests/simulation/cluster_property_test_cypher_queries.cpp
diff --git a/tests/simulation/simulation_interpreter.hpp b/tests/simulation/simulation_interpreter.hpp
index e25dfaa99..8e37f4f70 100644
--- a/tests/simulation/simulation_interpreter.hpp
+++ b/tests/simulation/simulation_interpreter.hpp
@@ -9,17 +9,17 @@
 // by the Apache License, Version 2.0, included in the file
 // licenses/APL.txt.
 
+#include "io/simulator/simulator_handle.hpp"
 #include "machine_manager/machine_config.hpp"
 #include "machine_manager/machine_manager.hpp"
-#include "io/simulator/simulator_handle.hpp"
 #include "query/v2/config.hpp"
 #include "query/v2/discard_value_stream.hpp"
 #include "query/v2/frontend/ast/ast.hpp"
 #include "query/v2/interpreter.hpp"
 #include "query/v2/request_router.hpp"
 
-#include <vector>
 #include <string>
+#include <vector>
 
 // TODO(gvolfing)
 // -How to set up the entire raft cluster with the QE. Also provide abrstraction for that.
@@ -33,7 +33,7 @@ class SimulatedInterpreter {
  public:
   explicit SimulatedInterpreter(std::unique_ptr<query::v2::InterpreterContext> interpreter_context)
       : interpreter_context_(std::move(interpreter_context)) {
-    interpreter_ = std::make_unique<memgraph::query::v2::Interpreter>(interpreter_context_);
+    interpreter_ = std::make_unique<memgraph::query::v2::Interpreter>(interpreter_context_.get());
   }
 
   SimulatedInterpreter(const SimulatedInterpreter &) = delete;
@@ -76,11 +76,10 @@ class SimulatedInterpreter {
 };
 
 SimulatedInterpreter SetUpInterpreter(Address coordinator_address, Simulator &simulator) {
-  auto rr_factory =
-      std::make_unique<memgraph::query::v2::SimulatedRequestRouterFactory>(simulator, coordinator_address);
+  auto rr_factory = std::make_unique<memgraph::query::v2::SimulatedRequestRouterFactory>(simulator);
 
   auto interpreter_context = std::make_unique<memgraph::query::v2::InterpreterContext>(
-      nullptr
+      nullptr,
       memgraph::query::v2::InterpreterConfig{.query = {.allow_load_csv = true},
                                              .execution_timeout_sec = 600,
                                              .replication_replica_check_frequency = std::chrono::seconds(1),

From 32231fe49a7707bb61db4cf745f83fe0cc4b1862 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Thu, 15 Dec 2022 17:10:27 +0100
Subject: [PATCH 10/90] Move the implementation of AllocateInitialEdgeIds into
 the child class

---
 src/query/v2/request_router.hpp              | 4 +---
 tests/unit/query_v2_expression_evaluator.cpp | 4 ++++
 2 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 74117f52e..aeaf5740a 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -128,9 +128,7 @@ class RequestRouterInterface {
   virtual bool IsPrimaryLabel(storage::v3::LabelId label) const = 0;
   virtual bool IsPrimaryKey(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const = 0;
 
-  virtual std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) {
-    return {};
-  }
+  virtual std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) = 0;
 };
 
 // TODO(kostasrim)rename this class template
diff --git a/tests/unit/query_v2_expression_evaluator.cpp b/tests/unit/query_v2_expression_evaluator.cpp
index 5f77ed4e7..393952423 100644
--- a/tests/unit/query_v2_expression_evaluator.cpp
+++ b/tests/unit/query_v2_expression_evaluator.cpp
@@ -120,6 +120,10 @@ class MockedRequestRouter : public RequestRouterInterface {
 
   bool IsPrimaryKey(LabelId primary_label, PropertyId property) const override { return true; }
 
+  std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) override {
+    return {};
+  }
+
  private:
   void SetUpNameIdMappers() {
     std::unordered_map<uint64_t, std::string> id_to_name;

From 751c27f792d91d6a58d7601334eb17ae33295061 Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Tue, 20 Dec 2022 10:12:50 +0100
Subject: [PATCH 11/90] Get ride of attribute has_valid_frames_

---
 src/query/v2/plan/operator.cpp | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index b6bbf9ce1..f162ec9ea 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -516,11 +516,11 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     valid_frames_it_ = valid_frames_consumer_->begin();
 
     MakeRequest(request_router, context);
-
-    has_next_frame_ = current_vertex_it_ != current_batch_.end() && valid_frames_it_ != valid_frames_consumer_->end();
   }
 
-  inline bool HasNextFrame() const { return has_next_frame_; }
+  inline bool HasNextFrame() {
+    return current_vertex_it_ != current_batch_.end() && valid_frames_it_ != valid_frames_consumer_->end();
+  }
 
   FrameWithValidity GetNextFrame(ExecutionContext &context) {
     MG_ASSERT(HasNextFrame());
@@ -600,7 +600,6 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   std::optional<ValidFramesConsumer> valid_frames_consumer_;
   ValidFramesConsumer::Iterator valid_frames_it_;
   std::queue<FrameWithValidity> frames_buffer_;
-  bool has_next_frame_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From 9589dd97b676896a062e3b2dc9f566de5524db8e Mon Sep 17 00:00:00 2001
From: jeremy <jeremy.bailleux@memgraph.io>
Date: Fri, 30 Dec 2022 16:21:41 +0100
Subject: [PATCH 12/90] Impl and correct aggregate

---
 src/query/v2/multiframe.hpp    |   2 +-
 src/query/v2/plan/operator.cpp | 116 +++++++++++++++++++++++++++++++++
 2 files changed, 117 insertions(+), 1 deletion(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index aeacda7d3..0365b449f 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -168,7 +168,7 @@ class ValidFramesModifier {
     Iterator &operator++() {
       do {
         ptr_++;
-      } while (*this != iterator_wrapper_->end() && ptr_->IsValid());
+      } while (*this != iterator_wrapper_->end() && !ptr_->IsValid());
 
       return *this;
     }
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index b6bbf9ce1..43376aaed 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1234,6 +1234,55 @@ class AggregateCursor : public Cursor {
     return true;
   }
 
+  void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP("AggregateMF");
+
+    if (!pulled_all_input_) {
+      while (!pulled_all_input_) {
+        ProcessAll(multi_frame, &context);
+      }
+      multi_frame.MakeAllFramesInvalid();
+      aggregation_it_ = aggregation_.begin();
+
+      // in case there is no input and no group_bys we need to return true
+      // just this once
+      if (aggregation_.empty() && self_.group_by_.empty()) {
+        auto frame = multi_frame.GetFirstFrame();
+        frame.MakeValid();
+        auto *pull_memory = context.evaluation_context.memory;
+        // place default aggregation values on the frame
+        for (const auto &elem : self_.aggregations_) {
+          frame[elem.output_sym] = DefaultAggregationOpValue(elem, pull_memory);
+        }
+        // place null as remember values on the frame
+        for (const Symbol &remember_sym : self_.remember_) {
+          frame[remember_sym] = TypedValue(pull_memory);
+        }
+        return;
+      }
+    }
+
+    if (aggregation_it_ == aggregation_.end()) {
+      return;
+    }
+
+    // place aggregation values on the frame
+    auto &frame = multi_frame.GetFirstFrame();
+    frame.MakeValid();
+    auto aggregation_values_it = aggregation_it_->second.values_.begin();
+    for (const auto &aggregation_elem : self_.aggregations_) {
+      frame[aggregation_elem.output_sym] = *aggregation_values_it++;
+    }
+
+    // place remember values on the frame
+    auto remember_values_it = aggregation_it_->second.remember_.begin();
+    for (const Symbol &remember_sym : self_.remember_) {
+      frame[remember_sym] = *remember_values_it++;
+    }
+
+    aggregation_it_++;
+  }
+
   void Shutdown() override { input_cursor_->Shutdown(); }
 
   void Reset() override {
@@ -1312,6 +1361,36 @@ class AggregateCursor : public Cursor {
     }
   }
 
+  void ProcessAll(MultiFrame &multi_frame, ExecutionContext *context) {
+    input_cursor_->PullMultiple(multi_frame, *context);
+    auto valid_frames_modifier =
+        multi_frame.GetValidFramesConsumer();  // consumer is needed i.o. reader because of the evaluator
+    if (valid_frames_modifier.begin() == valid_frames_modifier.end()) {
+      // There are no valid frames, we stop
+      pulled_all_input_ = true;
+      return;
+    }
+
+    for (auto &frame : valid_frames_modifier) {
+      ExpressionEvaluator evaluator(&frame, context->symbol_table, context->evaluation_context, context->request_router,
+                                    storage::v3::View::NEW);
+      ProcessOne(frame, &evaluator);
+    }
+
+    // calculate AVG aggregations (so far they have only been summed)
+    for (size_t pos = 0; pos < self_.aggregations_.size(); ++pos) {
+      if (self_.aggregations_[pos].op != Aggregation::Op::AVG) continue;
+      for (auto &kv : aggregation_) {
+        AggregationValue &agg_value = kv.second;
+        auto count = agg_value.counts_[pos];
+        auto *pull_memory = context->evaluation_context.memory;
+        if (count > 0) {
+          agg_value.values_[pos] = agg_value.values_[pos] / TypedValue(static_cast<double>(count), pull_memory);
+        }
+      }
+    }
+  }
+
   /**
    * Performs a single accumulation.
    */
@@ -1327,6 +1406,21 @@ class AggregateCursor : public Cursor {
     Update(evaluator, &agg_value);
   }
 
+  /**
+   * Performs a single accumulation.
+   */
+  void ProcessOne(FrameWithValidity &frame, ExpressionEvaluator *evaluator) {
+    auto *mem = aggregation_.get_allocator().GetMemoryResource();
+    utils::pmr::vector<TypedValue> group_by(mem);
+    group_by.reserve(self_.group_by_.size());
+    for (Expression *expression : self_.group_by_) {
+      group_by.emplace_back(expression->Accept(*evaluator));
+    }
+    auto &agg_value = aggregation_.try_emplace(std::move(group_by), mem).first->second;
+    EnsureInitialized(frame, &agg_value);
+    Update(evaluator, &agg_value);
+  }
+
   /** Ensures the new AggregationValue has been initialized. This means
    * that the value vectors are filled with an appropriate number of Nulls,
    * counts are set to 0 and remember values are remembered.
@@ -1343,6 +1437,28 @@ class AggregateCursor : public Cursor {
     for (const Symbol &remember_sym : self_.remember_) agg_value->remember_.push_back(frame[remember_sym]);
   }
 
+  /** Ensures the new AggregationValue has been initialized. This means
+   * that the value vectors are filled with an appropriate number of Nulls,
+   * counts are set to 0 and remember values are remembered.
+   */
+  void EnsureInitialized(FrameWithValidity &frame, AggregateCursor::AggregationValue *agg_value) const {
+    if (!agg_value->values_.empty()) {
+      frame.MakeInvalid();
+      return;
+    }
+
+    for (const auto &agg_elem : self_.aggregations_) {
+      auto *mem = agg_value->values_.get_allocator().GetMemoryResource();
+      agg_value->values_.emplace_back(DefaultAggregationOpValue(agg_elem, mem));
+    }
+    agg_value->counts_.resize(self_.aggregations_.size(), 0);
+
+    for (const Symbol &remember_sym : self_.remember_) {
+      agg_value->remember_.push_back(frame[remember_sym]);
+    }
+    frame.MakeInvalid();
+  }
+
   /** Updates the given AggregationValue with new data. Assumes that
    * the AggregationValue has been initialized */
   void Update(ExpressionEvaluator *evaluator, AggregateCursor::AggregationValue *agg_value) {

From b91b16de963b3c7b2cd2202dc2b5602aa0d4e915 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 07:06:25 +0100
Subject: [PATCH 13/90] Fix `Interpreter::PullMultiple` for queries that return
 some values

---
 src/query/v2/interpreter.cpp | 20 +++++++++++---------
 src/query/v2/multiframe.cpp  |  4 ----
 src/query/v2/multiframe.hpp  |  1 -
 3 files changed, 11 insertions(+), 14 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 594942aec..23386bd1c 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -704,7 +704,6 @@ PullPlan::PullPlan(const std::shared_ptr<CachedPlan> plan, const Parameters &par
   ctx_.request_router = request_router;
   ctx_.edge_ids_alloc = &interpreter_context->edge_ids_alloc;
 }
-
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStream *stream, std::optional<int> n,
                                                                         const std::vector<Symbol> &output_symbols,
                                                                         std::map<std::string, TypedValue> *summary) {
@@ -734,7 +733,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
   // Returns true if a result was pulled.
   const auto pull_result = [&]() -> bool {
     cursor_->PullMultiple(multi_frame_, ctx_);
-    return !multi_frame_.HasInvalidFrame();
+    return multi_frame_.HasValidFrame();
   };
 
   const auto stream_values = [&output_symbols, &stream](const Frame &frame) {
@@ -755,13 +754,14 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
   int i = 0;
   if (has_unsent_results_ && !output_symbols.empty()) {
     // stream unsent results from previous pull
-
-    auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader();
-    for (const auto &frame : iterator_for_valid_frame_only) {
+    for (auto &frame : multi_frame_.GetValidFramesConsumer()) {
       stream_values(frame);
+      frame.MakeInvalid();
       ++i;
+      if (i == n) {
+        break;
+      }
     }
-    multi_frame_.MakeAllFramesInvalid();
   }
 
   for (; !n || i < n;) {
@@ -770,13 +770,15 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
     }
 
     if (!output_symbols.empty()) {
-      auto iterator_for_valid_frame_only = multi_frame_.GetValidFramesReader();
-      for (const auto &frame : iterator_for_valid_frame_only) {
+      for (auto &frame : multi_frame_.GetValidFramesConsumer()) {
         stream_values(frame);
+        frame.MakeInvalid();
         ++i;
+        if (i == n) {
+          break;
+        }
       }
     }
-    multi_frame_.MakeAllFramesInvalid();
   }
 
   // If we finished because we streamed the requested n results,
diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index 0ddfd3aa7..835cdbc0f 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -48,10 +48,6 @@ bool MultiFrame::HasValidFrame() const noexcept {
   return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); });
 }
 
-bool MultiFrame::HasInvalidFrame() const noexcept {
-  return std::any_of(frames_.rbegin(), frames_.rend(), [](auto &frame) { return !frame.IsValid(); });
-}
-
 // NOLINTNEXTLINE (bugprone-exception-escape)
 void MultiFrame::DefragmentValidFrames() noexcept {
   /*
diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index f84fe6421..5f821bb6b 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -81,7 +81,6 @@ class MultiFrame {
   void MakeAllFramesInvalid() noexcept;
 
   bool HasValidFrame() const noexcept;
-  bool HasInvalidFrame() const noexcept;
 
   inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); }
 

From 36891c119b28d472e4d831af2f318cc36c3fd670 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 07:17:53 +0100
Subject: [PATCH 14/90] Remove unnecessary state from
 `DistributedScanAllAndFilterCursor`

---
 src/query/v2/plan/operator.cpp | 20 ++++----------------
 1 file changed, 4 insertions(+), 16 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index a57799367..1e84eaae5 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -467,31 +467,24 @@ class DistributedScanAllAndFilterCursor : public Cursor {
       current_batch_ = request_router.ScanVertices(request_label);
     }
     current_vertex_it_ = current_batch_.begin();
-    request_state_ = State::COMPLETED;
     return !current_batch_.empty();
   }
 
   bool Pull(Frame &frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
-    auto &request_router = *context.request_router;
     while (true) {
       if (MustAbort(context)) {
         throw HintedAbortError();
       }
 
-      if (request_state_ == State::INITIALIZING) {
-        if (!input_cursor_->Pull(frame, context)) {
+      if (current_vertex_it_ == current_batch_.end()) {
+        ResetExecutionState();
+        if (!input_cursor_->Pull(frame, context) || !MakeRequest(*context.request_router, context)) {
           return false;
         }
       }
 
-      if (current_vertex_it_ == current_batch_.end() &&
-          (request_state_ == State::COMPLETED || !MakeRequest(request_router, context))) {
-        ResetExecutionState();
-        continue;
-      }
-
       frame[output_symbol_] = TypedValue(std::move(*current_vertex_it_));
       ++current_vertex_it_;
       return true;
@@ -568,7 +561,6 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   void ResetExecutionState() {
     current_batch_.clear();
     current_vertex_it_ = current_batch_.end();
-    request_state_ = State::INITIALIZING;
   }
 
   void Reset() override {
@@ -581,15 +573,13 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   const UniqueCursorPtr input_cursor_;
   const char *op_name_;
   std::vector<VertexAccessor> current_batch_;
-  std::vector<VertexAccessor>::iterator current_vertex_it_;
-  State request_state_ = State::INITIALIZING;
+  std::vector<VertexAccessor>::iterator current_vertex_it_{current_batch_.begin()};
   std::optional<storage::v3::LabelId> label_;
   std::optional<std::pair<storage::v3::PropertyId, Expression *>> property_expression_pair_;
   std::optional<std::vector<Expression *>> filter_expressions_;
   std::optional<MultiFrame> own_multi_frames_;
   std::optional<ValidFramesConsumer> valid_frames_consumer_;
   ValidFramesConsumer::Iterator valid_frames_it_;
-  std::queue<FrameWithValidity> frames_buffer_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)
@@ -597,8 +587,6 @@ ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_sy
 
 ACCEPT_WITH_INPUT(ScanAll)
 
-class DistributedScanAllCursor;
-
 UniqueCursorPtr ScanAll::MakeCursor(utils::MemoryResource *mem) const {
   EventCounter::IncrementCounter(EventCounter::ScanAllOperator);
 

From d11d5c3fa93c02383566f712775c669861e32ae6 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 08:33:40 +0100
Subject: [PATCH 15/90] Make special member functions of `MultiFrame` iterators
 consistent

---
 src/query/v2/multiframe.hpp | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index 5f821bb6b..28009eb5d 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -96,9 +96,9 @@ class ValidFramesReader {
 
   ~ValidFramesReader() = default;
   ValidFramesReader(const ValidFramesReader &other) = delete;
-  ValidFramesReader(ValidFramesReader &&other) noexcept = delete;
+  ValidFramesReader(ValidFramesReader &&other) noexcept = default;
   ValidFramesReader &operator=(const ValidFramesReader &other) = delete;
-  ValidFramesReader &operator=(ValidFramesReader &&other) noexcept = delete;
+  ValidFramesReader &operator=(ValidFramesReader &&other) noexcept = default;
 
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;
@@ -146,9 +146,9 @@ class ValidFramesModifier {
 
   ~ValidFramesModifier() = default;
   ValidFramesModifier(const ValidFramesModifier &other) = delete;
-  ValidFramesModifier(ValidFramesModifier &&other) noexcept = delete;
+  ValidFramesModifier(ValidFramesModifier &&other) noexcept = default;
   ValidFramesModifier &operator=(const ValidFramesModifier &other) = delete;
-  ValidFramesModifier &operator=(ValidFramesModifier &&other) noexcept = delete;
+  ValidFramesModifier &operator=(ValidFramesModifier &&other) noexcept = default;
 
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;
@@ -200,10 +200,10 @@ class ValidFramesConsumer {
   explicit ValidFramesConsumer(MultiFrame &multiframe);
 
   ~ValidFramesConsumer() noexcept;
-  ValidFramesConsumer(const ValidFramesConsumer &other) = default;
+  ValidFramesConsumer(const ValidFramesConsumer &other) = delete;
   ValidFramesConsumer(ValidFramesConsumer &&other) noexcept = default;
-  ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = default;
-  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = delete;
+  ValidFramesConsumer &operator=(const ValidFramesConsumer &other) = delete;
+  ValidFramesConsumer &operator=(ValidFramesConsumer &&other) noexcept = default;
 
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;

From 57690c5390b0ba6e59ffe9410a0b95c07681239c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 08:34:08 +0100
Subject: [PATCH 16/90] Refactor `DistributedScanAllAndFilterCursor`

---
 src/query/v2/plan/operator.cpp | 81 +++++++++++++++-------------------
 1 file changed, 36 insertions(+), 45 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 1e84eaae5..9841791e6 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -457,14 +457,14 @@ class DistributedScanAllAndFilterCursor : public Cursor {
 
   using VertexAccessor = accessors::VertexAccessor;
 
-  bool MakeRequest(RequestRouterInterface &request_router, ExecutionContext &context) {
+  bool MakeRequest(ExecutionContext &context) {
     {
       SCOPED_REQUEST_WAIT_PROFILE;
       std::optional<std::string> request_label = std::nullopt;
       if (label_.has_value()) {
-        request_label = request_router.LabelToName(*label_);
+        request_label = context.request_router->LabelToName(*label_);
       }
-      current_batch_ = request_router.ScanVertices(request_label);
+      current_batch_ = context.request_router->ScanVertices(request_label);
     }
     current_vertex_it_ = current_batch_.begin();
     return !current_batch_.empty();
@@ -480,7 +480,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
 
       if (current_vertex_it_ == current_batch_.end()) {
         ResetExecutionState();
-        if (!input_cursor_->Pull(frame, context) || !MakeRequest(*context.request_router, context)) {
+        if (!input_cursor_->Pull(frame, context) || !MakeRequest(context)) {
           return false;
         }
       }
@@ -491,66 +491,57 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     }
   }
 
-  void PrepareNextFrames(ExecutionContext &context) {
-    auto &request_router = *context.request_router;
-
-    input_cursor_->PullMultiple(*own_multi_frames_, context);
-    valid_frames_consumer_ = own_multi_frames_->GetValidFramesConsumer();
-    valid_frames_it_ = valid_frames_consumer_->begin();
-
-    MakeRequest(request_router, context);
+  bool PullNextFrames(ExecutionContext &context) {
+    input_cursor_->PullMultiple(*own_multi_frame_, context);
+    own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
+    own_frames_it_ = own_frames_consumer_->begin();
+    return own_multi_frame_->HasValidFrame();
   }
 
-  inline bool HasNextFrame() {
-    return current_vertex_it_ != current_batch_.end() && valid_frames_it_ != valid_frames_consumer_->end();
+  inline bool HasMoreResult() {
+    return current_vertex_it_ != current_batch_.end() && own_frames_it_ != own_frames_consumer_->end();
   }
 
-  FrameWithValidity GetNextFrame(ExecutionContext &context) {
-    MG_ASSERT(HasNextFrame());
+  bool PopulateFrame(ExecutionContext &context, FrameWithValidity &frame) {
+    MG_ASSERT(HasMoreResult());
 
-    auto frame = *valid_frames_it_;
+    frame = *own_frames_it_;
     frame[output_symbol_] = TypedValue(*current_vertex_it_);
 
     ++current_vertex_it_;
     if (current_vertex_it_ == current_batch_.end()) {
-      valid_frames_it_->MakeInvalid();
-      ++valid_frames_it_;
+      own_frames_it_->MakeInvalid();
+      ++own_frames_it_;
 
-      if (valid_frames_it_ == valid_frames_consumer_->end()) {
-        PrepareNextFrames(context);
-      } else {
-        current_vertex_it_ = current_batch_.begin();
+      current_vertex_it_ = current_batch_.begin();
+
+      if (own_frames_it_ == own_frames_consumer_->end()) {
+        return PullNextFrames(context);
       }
     };
-
-    return frame;
+    return true;
   }
 
   void PullMultiple(MultiFrame &input_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
-    if (!own_multi_frames_.has_value()) {
-      own_multi_frames_.emplace(MultiFrame(input_multi_frame.GetFirstFrame().elems().size(),
-                                           kNumberOfFramesInMultiframe, input_multi_frame.GetMemoryResource()));
-      PrepareNextFrames(context);
+    if (!own_multi_frame_.has_value()) {
+      own_multi_frame_.emplace(MultiFrame(input_multi_frame.GetFirstFrame().elems().size(), kNumberOfFramesInMultiframe,
+                                          input_multi_frame.GetMemoryResource()));
+
+      MakeRequest(context);
+      PullNextFrames(context);
     }
 
-    while (true) {
+    if (!HasMoreResult()) {
+      return;
+    }
+
+    for (auto &frame : input_multi_frame.GetInvalidFramesPopulator()) {
       if (MustAbort(context)) {
         throw HintedAbortError();
       }
-
-      auto invalid_frames_populator = input_multi_frame.GetInvalidFramesPopulator();
-      auto invalid_frame_it = invalid_frames_populator.begin();
-      auto has_modified_at_least_one_frame = false;
-
-      while (invalid_frames_populator.end() != invalid_frame_it && HasNextFrame()) {
-        has_modified_at_least_one_frame = true;
-        *invalid_frame_it = GetNextFrame(context);
-        ++invalid_frame_it;
-      }
-
-      if (!has_modified_at_least_one_frame) {
+      if (!PopulateFrame(context, frame)) {
         return;
       }
     }
@@ -577,9 +568,9 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   std::optional<storage::v3::LabelId> label_;
   std::optional<std::pair<storage::v3::PropertyId, Expression *>> property_expression_pair_;
   std::optional<std::vector<Expression *>> filter_expressions_;
-  std::optional<MultiFrame> own_multi_frames_;
-  std::optional<ValidFramesConsumer> valid_frames_consumer_;
-  ValidFramesConsumer::Iterator valid_frames_it_;
+  std::optional<MultiFrame> own_multi_frame_;
+  std::optional<ValidFramesConsumer> own_frames_consumer_;
+  ValidFramesConsumer::Iterator own_frames_it_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From 901da4c9b3ad3636aebb3bf4f0210f739ce35d60 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 21:01:22 +0100
Subject: [PATCH 17/90] Update `InvalidFramesPopulator` to follow the
 conventions

---
 src/query/v2/multiframe.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index 28009eb5d..e286e00a3 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -255,9 +255,9 @@ class InvalidFramesPopulator {
   ~InvalidFramesPopulator() = default;
 
   InvalidFramesPopulator(const InvalidFramesPopulator &other) = delete;
-  InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = delete;
+  InvalidFramesPopulator(InvalidFramesPopulator &&other) noexcept = default;
   InvalidFramesPopulator &operator=(const InvalidFramesPopulator &other) = delete;
-  InvalidFramesPopulator &operator=(InvalidFramesPopulator &&other) noexcept = delete;
+  InvalidFramesPopulator &operator=(InvalidFramesPopulator &&other) noexcept = default;
 
   struct Iterator {
     using iterator_category = std::forward_iterator_tag;

From 575361827e9b3a1e9772deb76b02530ccf2dea23 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 17 Jan 2023 21:01:54 +0100
Subject: [PATCH 18/90] Add comment about invalid usage of `MutliFrame`

---
 src/query/v2/multiframe.hpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index e286e00a3..f464343b4 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -33,6 +33,7 @@ class MultiFrame {
   MultiFrame(size_t size_of_frame, size_t number_of_frames, utils::MemoryResource *execution_memory);
   ~MultiFrame() = default;
 
+  // Assigning and moving the MultiFrame is not allowed if any accessor from the above ones are alive.
   MultiFrame(const MultiFrame &other);
   MultiFrame(MultiFrame &&other) noexcept;
   MultiFrame &operator=(const MultiFrame &other) = delete;

From f39a937323ccdec367eb37373cc15b6063b7539b Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 18 Jan 2023 13:31:35 +0100
Subject: [PATCH 19/90] Add first, but buggy implementation

---
 src/expr/interpret/frame.hpp   |   3 +-
 src/query/v2/plan/operator.cpp | 173 ++++++++++++++++++++++++++++++---
 src/query/v2/requests.hpp      |  46 +++++++++
 3 files changed, 206 insertions(+), 16 deletions(-)

diff --git a/src/expr/interpret/frame.hpp b/src/expr/interpret/frame.hpp
index 1cd6a99ce..9f4068226 100644
--- a/src/expr/interpret/frame.hpp
+++ b/src/expr/interpret/frame.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -34,6 +34,7 @@ class Frame {
   const TypedValue &at(const Symbol &symbol) const { return elems_.at(symbol.position()); }
 
   auto &elems() { return elems_; }
+  const auto &elems() const { return elems_; }
 
   utils::MemoryResource *GetMemoryResource() const { return elems_.get_allocator().GetMemoryResource(); }
 
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 9841791e6..900ec15a0 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -522,12 +522,12 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     return true;
   }
 
-  void PullMultiple(MultiFrame &input_multi_frame, ExecutionContext &context) override {
+  void PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
     if (!own_multi_frame_.has_value()) {
-      own_multi_frame_.emplace(MultiFrame(input_multi_frame.GetFirstFrame().elems().size(), kNumberOfFramesInMultiframe,
-                                          input_multi_frame.GetMemoryResource()));
+      own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
+                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
 
       MakeRequest(context);
       PullNextFrames(context);
@@ -537,7 +537,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
       return;
     }
 
-    for (auto &frame : input_multi_frame.GetInvalidFramesPopulator()) {
+    for (auto &frame : output_multi_frame.GetInvalidFramesPopulator()) {
       if (MustAbort(context)) {
         throw HintedAbortError();
       }
@@ -2629,7 +2629,7 @@ class DistributedCreateExpandCursor : public Cursor {
 
 class DistributedExpandCursor : public Cursor {
  public:
-  explicit DistributedExpandCursor(const Expand &self, utils::MemoryResource *mem)
+  DistributedExpandCursor(const Expand &self, utils::MemoryResource *mem)
       : self_(self),
         input_cursor_(self.input_->MakeCursor(mem)),
         current_in_edge_it_(current_in_edges_.begin()),
@@ -2666,16 +2666,10 @@ class DistributedExpandCursor : public Cursor {
           throw std::runtime_error("EdgeDirection Both not implemented");
       }
     };
-    msgs::ExpandOneRequest request;
-    // to not fetch any properties of the edges
-    request.edge_properties.emplace();
-    request.src_vertices.push_back(get_dst_vertex(edge, direction));
-    request.direction = (direction == EdgeAtom::Direction::IN) ? msgs::EdgeDirection::OUT : msgs::EdgeDirection::IN;
-    auto result_rows = context.request_router->ExpandOne(std::move(request));
-    MG_ASSERT(result_rows.size() == 1);
-    auto &result_row = result_rows.front();
-    frame[self_.common_.node_symbol] = accessors::VertexAccessor(
-        msgs::Vertex{result_row.src_vertex}, result_row.src_vertex_properties, context.request_router);
+
+    frame[self_.common_.node_symbol] =
+        accessors::VertexAccessor(msgs::Vertex{get_dst_vertex(edge, direction)},
+                                  std::vector<std::pair<msgs::PropertyId, msgs::Value>>{}, context.request_router);
   }
 
   bool InitEdges(Frame &frame, ExecutionContext &context) {
@@ -2784,6 +2778,149 @@ class DistributedExpandCursor : public Cursor {
     }
   }
 
+  void InitEdgesMultiple(ExecutionContext &context) {
+    TypedValue &vertex_value = (*own_frames_it_)[self_.input_symbol_];
+
+    if (vertex_value.IsNull()) {
+      return;
+    }
+
+    ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
+    auto &vertex = vertex_value.ValueVertex();
+
+    const auto convert_edges = [&vertex, &context](
+                                   std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties> &&edge_messages,
+                                   const EdgeAtom::Direction direction) {
+      std::vector<EdgeAccessor> edge_accessors;
+      edge_accessors.reserve(edge_messages.size());
+
+      switch (direction) {
+        case EdgeAtom::Direction::IN: {
+          for (auto &edge : edge_messages) {
+            edge_accessors.emplace_back(msgs::Edge{std::move(edge.other_end), vertex.Id(), {}, {edge.gid}, edge.type},
+                                        context.request_router);
+          }
+          break;
+        }
+        case EdgeAtom::Direction::OUT: {
+          for (auto &edge : edge_messages) {
+            edge_accessors.emplace_back(msgs::Edge{vertex.Id(), std::move(edge.other_end), {}, {edge.gid}, edge.type},
+                                        context.request_router);
+          }
+          break;
+        }
+        case EdgeAtom::Direction::BOTH: {
+          LOG_FATAL("Must indicate exact expansion direction here");
+        }
+      }
+      return edge_accessors;
+    };
+
+    auto *result_row = vertex_id_to_result_row[vertex.Id()];
+    current_in_edges_.clear();
+    current_in_edges_ =
+        convert_edges(std::move(result_row->in_edges_with_specific_properties), EdgeAtom::Direction::IN);
+    current_in_edge_it_ = current_in_edges_.begin();
+    current_out_edges_ =
+        convert_edges(std::move(result_row->out_edges_with_specific_properties), EdgeAtom::Direction::OUT);
+    current_out_edge_it_ = current_out_edges_.begin();
+    vertex_id_to_result_row.erase(vertex.Id());
+  }
+
+  void PullEdgesFromStorage(ExecutionContext &context) {
+    // Input Vertex could be null if it is created by a failed optional match. In
+    // those cases we skip that input pull and continue with the next.
+
+    msgs::ExpandOneRequest request;
+    request.direction = DirectionToMsgsDirection(self_.common_.direction);
+    // to not fetch any properties of the edges
+    request.edge_properties.emplace();
+    for (const auto &frame : own_multi_frame_->GetValidFramesReader()) {
+      const auto &vertex_value = frame[self_.input_symbol_];
+
+      // Null check due to possible failed optional match.
+      MG_ASSERT(!vertex_value.IsNull());
+
+      ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
+      auto &vertex = vertex_value.ValueVertex();
+      request.src_vertices.push_back(vertex.Id());
+    }
+
+    result_rows_ = std::invoke([&context, &request]() mutable {
+      SCOPED_REQUEST_WAIT_PROFILE;
+      return context.request_router->ExpandOne(std::move(request));
+    });
+    vertex_id_to_result_row.clear();
+    for (auto &row : result_rows_) {
+      vertex_id_to_result_row[row.src_vertex.id] = &row;
+    }
+  }
+
+  void PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP("DistributedExpandMF");
+    MG_ASSERT(!self_.common_.existing_node);
+    EnsureOwnMultiFrameIsGood(output_multi_frame);
+    // A helper function for expanding a node from an edge.
+
+    while (true) {
+      if (MustAbort(context)) throw HintedAbortError();
+      if (own_frames_it_ == own_frames_consumer_->end()) {
+        input_cursor_->PullMultiple(*own_multi_frame_, context);
+        own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
+        own_frames_it_ = own_frames_consumer_->begin();
+        if (!own_multi_frame_->HasValidFrame()) {
+          break;
+        }
+
+        PullEdgesFromStorage(context);
+        InitEdgesMultiple(context);
+      }
+
+      while (own_frames_it_ != own_frames_consumer_->end()) {
+        if (current_in_edge_it_ == current_in_edges_.end() && current_out_edge_it_ == current_out_edges_.end()) {
+          own_frames_it_->MakeInvalid();
+          ++own_frames_it_;
+
+          InitEdgesMultiple(context);
+        }
+
+        auto &input_frame = *own_frames_it_;
+
+        auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+
+        auto populate_edges = [this, &context, &output_frames_populator, &input_frame](
+                                  std::vector<EdgeAccessor>::iterator &current,
+                                  const std::vector<EdgeAccessor>::iterator &end) {
+          for (auto output_frame_it = output_frames_populator.begin();
+               output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
+            auto &edge = *current;
+            ++current;
+            auto &output_frame = *output_frame_it++;
+            output_frame = input_frame;
+            output_frame[self_.common_.edge_symbol] = edge;
+            PullDstVertex(output_frame, context, EdgeAtom::Direction::IN);
+          }
+        };
+        populate_edges(current_in_edge_it_, current_in_edges_.end());
+        populate_edges(current_out_edge_it_, current_out_edges_.end());
+
+        if (output_frames_populator.begin() == output_frames_populator.end()) {
+          return;
+        }
+      }
+    }
+  }
+
+  void EnsureOwnMultiFrameIsGood(MultiFrame &output_multi_frame) {
+    if (!own_multi_frame_.has_value()) {
+      own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
+                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+      own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+      own_frames_it_ = own_frames_consumer_->begin();
+    }
+    MG_ASSERT(output_multi_frame.GetFirstFrame().elems().size() == own_multi_frame_->GetFirstFrame().elems().size());
+  }
+
   void Shutdown() override { input_cursor_->Shutdown(); }
 
   void Reset() override {
@@ -2801,6 +2938,12 @@ class DistributedExpandCursor : public Cursor {
   std::vector<EdgeAccessor> current_out_edges_;
   std::vector<EdgeAccessor>::iterator current_in_edge_it_;
   std::vector<EdgeAccessor>::iterator current_out_edge_it_;
+  std::optional<MultiFrame> own_multi_frame_;
+  std::optional<ValidFramesConsumer> own_frames_consumer_;
+  ValidFramesConsumer::Iterator own_frames_it_;
+  std::vector<msgs::ExpandOneResultRow> result_rows_;
+  // This won't work if any vertex id is duplicated in the input
+  std::unordered_map<msgs::VertexId, msgs::ExpandOneResultRow *> vertex_id_to_result_row;
 };
 
 }  // namespace memgraph::query::v2::plan
diff --git a/src/query/v2/requests.hpp b/src/query/v2/requests.hpp
index 2335fea7d..b2d7f9123 100644
--- a/src/query/v2/requests.hpp
+++ b/src/query/v2/requests.hpp
@@ -25,6 +25,7 @@
 #include "storage/v3/id_types.hpp"
 #include "storage/v3/property_value.hpp"
 #include "storage/v3/result.hpp"
+#include "utils/fnv.hpp"
 
 namespace memgraph::msgs {
 
@@ -579,3 +580,48 @@ using WriteResponses = std::variant<CreateVerticesResponse, DeleteVerticesRespon
                                     CreateExpandResponse, DeleteEdgesResponse, UpdateEdgesResponse, CommitResponse>;
 
 }  // namespace memgraph::msgs
+
+namespace std {
+
+template <>
+struct hash<memgraph::msgs::Value>;
+
+template <>
+struct hash<memgraph::msgs::VertexId> {
+  size_t operator()(const memgraph::msgs::VertexId &id) const {
+    using LabelId = memgraph::storage::v3::LabelId;
+    using Value = memgraph::msgs::Value;
+    return memgraph::utils::HashCombine<LabelId, std::vector<Value>, std::hash<LabelId>,
+                                        memgraph::utils::FnvCollection<std::vector<Value>, Value>>{}(id.first.id,
+                                                                                                     id.second);
+  }
+};
+
+template <>
+struct hash<memgraph::msgs::Value> {
+  size_t operator()(const memgraph::msgs::Value &value) const {
+    using Type = memgraph::msgs::Value::Type;
+    switch (value.type) {
+      case Type::Null:
+        return std::hash<size_t>{}(0U);
+      case Type::Bool:
+        return std::hash<bool>{}(value.bool_v);
+      case Type::Int64:
+        return std::hash<int64_t>{}(value.int_v);
+      case Type::Double:
+        return std::hash<double>{}(value.double_v);
+      case Type::String:
+        return std::hash<std::string>{}(value.string_v);
+      case Type::List:
+        LOG_FATAL("Add hash for lists");
+      case Type::Map:
+        LOG_FATAL("Add hash for maps");
+      case Type::Vertex:
+        LOG_FATAL("Add hash for vertices");
+      case Type::Edge:
+        LOG_FATAL("Add hash for edges");
+    }
+  }
+};
+
+}  // namespace std

From 94a536a2b93af239dff985ca2a85e2d29a5e153e Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 18 Jan 2023 16:03:34 +0100
Subject: [PATCH 20/90] Fix hanging conditionvariable

---
 src/query/v2/interpreter.hpp                 | 5 +++--
 src/query/v2/request_router.hpp              | 5 +++--
 tests/simulation/simulation_interpreter.hpp  | 8 +++++++-
 tests/simulation/test_cluster.hpp            | 3 ++-
 tests/unit/query_v2_expression_evaluator.cpp | 4 +++-
 5 files changed, 18 insertions(+), 7 deletions(-)

diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 05b05dd8e..8a535bf3c 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -301,9 +301,9 @@ class Interpreter final {
 
  private:
   struct QueryExecution {
-    std::optional<PreparedQuery> prepared_query;
     utils::MonotonicBufferResource execution_memory{kExecutionMemoryBlockSize};
     utils::ResourceWithOutOfMemoryException execution_memory_with_exception{&execution_memory};
+    std::optional<PreparedQuery> prepared_query;
 
     std::map<std::string, TypedValue> summary;
     std::vector<Notification> notifications;
@@ -388,6 +388,7 @@ std::map<std::string, TypedValue> Interpreter::Pull(TStream *result_stream, std:
     // Wrap the (statically polymorphic) stream type into a common type which
     // the handler knows.
     AnyStream stream{result_stream, &query_execution->execution_memory};
+    auto asd = query_execution->prepared_query->query_handler;
     const auto maybe_res = query_execution->prepared_query->query_handler(&stream, n);
     // Stream is using execution memory of the query_execution which
     // can be deleted after its execution so the stream should be cleared
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 2b7b7c89a..d6c484e86 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -119,6 +119,7 @@ class RequestRouterInterface {
   virtual bool IsPrimaryKey(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const = 0;
 
   virtual std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) = 0;
+  virtual void InstallSimulatorTicker(std::function<bool()> tick_simulator) = 0;
 };
 
 // TODO(kostasrim)rename this class template
@@ -143,7 +144,7 @@ class RequestRouter : public RequestRouterInterface {
 
   ~RequestRouter() override {}
 
-  void InstallSimulatorTicker(std::function<bool()> tick_simulator) {
+  void InstallSimulatorTicker(std::function<bool()> tick_simulator) override {
     notifier_.InstallSimulatorTicker(tick_simulator);
   }
 
diff --git a/tests/simulation/simulation_interpreter.hpp b/tests/simulation/simulation_interpreter.hpp
index 8e37f4f70..497ee7103 100644
--- a/tests/simulation/simulation_interpreter.hpp
+++ b/tests/simulation/simulation_interpreter.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -42,6 +42,12 @@ class SimulatedInterpreter {
   SimulatedInterpreter &operator=(SimulatedInterpreter &&) = delete;
   ~SimulatedInterpreter() = default;
 
+  void InstallSimulatorTicker(Simulator &simulator) {
+    std::function<bool()> tick_simulator = simulator.GetSimulatorTickClosure();
+    auto *request_router = interpreter_->GetRequestRouter();
+    request_router->InstallSimulatorTicker(tick_simulator);
+  }
+
   std::vector<ResultStream> RunQueries(const std::vector<std::string> &queries) {
     std::vector<ResultStream> results;
     results.reserve(queries.size());
diff --git a/tests/simulation/test_cluster.hpp b/tests/simulation/test_cluster.hpp
index 9fa331818..f10e88e61 100644
--- a/tests/simulation/test_cluster.hpp
+++ b/tests/simulation/test_cluster.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -314,6 +314,7 @@ std::pair<SimulatorStats, LatencyHistogramSummaries> RunClusterSimulationWithQue
   WaitForShardsToInitialize(coordinator_client);
 
   auto simulated_interpreter = io::simulator::SetUpInterpreter(coordinator_address, simulator);
+  simulated_interpreter.InstallSimulatorTicker(simulator);
 
   auto query_results = simulated_interpreter.RunQueries(queries);
 
diff --git a/tests/unit/query_v2_expression_evaluator.cpp b/tests/unit/query_v2_expression_evaluator.cpp
index b799bdafb..1127958d0 100644
--- a/tests/unit/query_v2_expression_evaluator.cpp
+++ b/tests/unit/query_v2_expression_evaluator.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -129,6 +129,8 @@ class MockedRequestRouter : public RequestRouterInterface {
     return {};
   }
 
+  void InstallSimulatorTicker(std::function<bool()> tick_simulator) override {}
+
  private:
   void SetUpNameIdMappers() {
     std::unordered_map<uint64_t, std::string> id_to_name;

From e888464de212a50cf90a28b52b777d8ca5a59a7c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 18 Jan 2023 17:32:22 +0100
Subject: [PATCH 21/90] Implement automaton for `ExpandOneCursor`

---
 src/query/v2/multiframe.cpp    |   4 ++
 src/query/v2/multiframe.hpp    |   1 +
 src/query/v2/plan/operator.cpp | 109 ++++++++++++++++++++-------------
 3 files changed, 71 insertions(+), 43 deletions(-)

diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index 835cdbc0f..14841d2c4 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -48,6 +48,10 @@ bool MultiFrame::HasValidFrame() const noexcept {
   return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return frame.IsValid(); });
 }
 
+bool MultiFrame::HasInvalidFrame() const noexcept {
+  return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); });
+}
+
 // NOLINTNEXTLINE (bugprone-exception-escape)
 void MultiFrame::DefragmentValidFrames() noexcept {
   /*
diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index f464343b4..6958ffbe8 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -82,6 +82,7 @@ class MultiFrame {
   void MakeAllFramesInvalid() noexcept;
 
   bool HasValidFrame() const noexcept;
+  bool HasInvalidFrame() const noexcept;
 
   inline utils::MemoryResource *GetMemoryResource() { return frames_[0].GetMemoryResource(); }
 
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 900ec15a0..eba45b0b7 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2827,9 +2827,15 @@ class DistributedExpandCursor : public Cursor {
     vertex_id_to_result_row.erase(vertex.Id());
   }
 
-  void PullEdgesFromStorage(ExecutionContext &context) {
-    // Input Vertex could be null if it is created by a failed optional match. In
-    // those cases we skip that input pull and continue with the next.
+  bool PullInputFrames(ExecutionContext &context) {
+    input_cursor_->PullMultiple(*own_multi_frame_, context);
+    // These needs to be updated regardless of the result of the pull, otherwise the consumer and iterator might
+    // get corrupted because of the operations done on our MultiFrame.
+    own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
+    own_frames_it_ = own_frames_consumer_->begin();
+    if (!own_multi_frame_->HasValidFrame()) {
+      return false;
+    }
 
     msgs::ExpandOneRequest request;
     request.direction = DirectionToMsgsDirection(self_.common_.direction);
@@ -2842,7 +2848,7 @@ class DistributedExpandCursor : public Cursor {
       MG_ASSERT(!vertex_value.IsNull());
 
       ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
-      auto &vertex = vertex_value.ValueVertex();
+      const auto &vertex = vertex_value.ValueVertex();
       request.src_vertices.push_back(vertex.Id());
     }
 
@@ -2854,6 +2860,8 @@ class DistributedExpandCursor : public Cursor {
     for (auto &row : result_rows_) {
       vertex_id_to_result_row[row.src_vertex.id] = &row;
     }
+
+    return true;
   }
 
   void PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
@@ -2862,49 +2870,55 @@ class DistributedExpandCursor : public Cursor {
     EnsureOwnMultiFrameIsGood(output_multi_frame);
     // A helper function for expanding a node from an edge.
 
+    auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+
     while (true) {
-      if (MustAbort(context)) throw HintedAbortError();
-      if (own_frames_it_ == own_frames_consumer_->end()) {
-        input_cursor_->PullMultiple(*own_multi_frame_, context);
-        own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
-        own_frames_it_ = own_frames_consumer_->begin();
-        if (!own_multi_frame_->HasValidFrame()) {
+      switch (state_) {
+        case State::PullInputAndEdges: {
+          if (!PullInputFrames(context)) {
+            state_ = State::Exhausted;
+            return;
+          }
+          state_ = State::InitInOutEdgesIt;
           break;
         }
-
-        PullEdgesFromStorage(context);
-        InitEdgesMultiple(context);
-      }
-
-      while (own_frames_it_ != own_frames_consumer_->end()) {
-        if (current_in_edge_it_ == current_in_edges_.end() && current_out_edge_it_ == current_out_edges_.end()) {
-          own_frames_it_->MakeInvalid();
-          ++own_frames_it_;
-
-          InitEdgesMultiple(context);
-        }
-
-        auto &input_frame = *own_frames_it_;
-
-        auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
-
-        auto populate_edges = [this, &context, &output_frames_populator, &input_frame](
-                                  std::vector<EdgeAccessor>::iterator &current,
-                                  const std::vector<EdgeAccessor>::iterator &end) {
-          for (auto output_frame_it = output_frames_populator.begin();
-               output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
-            auto &edge = *current;
-            ++current;
-            auto &output_frame = *output_frame_it++;
-            output_frame = input_frame;
-            output_frame[self_.common_.edge_symbol] = edge;
-            PullDstVertex(output_frame, context, EdgeAtom::Direction::IN);
+        case State::InitInOutEdgesIt: {
+          if (own_frames_it_ == own_frames_consumer_->end()) {
+            state_ = State::PullInputAndEdges;
+          } else {
+            InitEdges(*own_frames_it_, context);
+            state_ = State::PopulateOutput;
           }
-        };
-        populate_edges(current_in_edge_it_, current_in_edges_.end());
-        populate_edges(current_out_edge_it_, current_out_edges_.end());
-
-        if (output_frames_populator.begin() == output_frames_populator.end()) {
+          break;
+        }
+        case State::PopulateOutput: {
+          if (!output_multi_frame.HasInvalidFrame()) {
+            return;
+          }
+          if (current_in_edge_it_ == current_in_edges_.end() && current_out_edge_it_ == current_out_edges_.end()) {
+            own_frames_it_->MakeInvalid();
+            ++own_frames_it_;
+            state_ = State::InitInOutEdgesIt;
+            continue;
+          }
+          auto populate_edges = [this, &context, &output_frames_populator](
+                                    const EdgeAtom::Direction direction, std::vector<EdgeAccessor>::iterator &current,
+                                    const std::vector<EdgeAccessor>::iterator &end) {
+            for (auto output_frame_it = output_frames_populator.begin();
+                 output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
+              auto &edge = *current;
+              ++current;
+              auto &output_frame = *output_frame_it++;
+              output_frame = *own_frames_it_;
+              output_frame[self_.common_.edge_symbol] = edge;
+              PullDstVertex(output_frame, context, direction);
+            }
+          };
+          populate_edges(EdgeAtom::Direction::IN, current_in_edge_it_, current_in_edges_.end());
+          populate_edges(EdgeAtom::Direction::OUT, current_out_edge_it_, current_out_edges_.end());
+          break;
+        }
+        case State::Exhausted: {
           return;
         }
       }
@@ -2925,6 +2939,12 @@ class DistributedExpandCursor : public Cursor {
 
   void Reset() override {
     input_cursor_->Reset();
+    vertex_id_to_result_row.clear();
+    result_rows_.clear();
+    own_frames_it_ = ValidFramesConsumer::Iterator{};
+    own_frames_consumer_.reset();
+    own_multi_frame_->MakeAllFramesInvalid();
+    state_ = State::PullInputAndEdges;
     current_in_edges_.clear();
     current_out_edges_.clear();
     current_in_edge_it_ = current_in_edges_.end();
@@ -2932,12 +2952,15 @@ class DistributedExpandCursor : public Cursor {
   }
 
  private:
+  enum class State { PullInputAndEdges, InitInOutEdgesIt, PopulateOutput, Exhausted };
+
   const Expand &self_;
   const UniqueCursorPtr input_cursor_;
   std::vector<EdgeAccessor> current_in_edges_;
   std::vector<EdgeAccessor> current_out_edges_;
   std::vector<EdgeAccessor>::iterator current_in_edge_it_;
   std::vector<EdgeAccessor>::iterator current_out_edge_it_;
+  State state_{State::PullInputAndEdges};
   std::optional<MultiFrame> own_multi_frame_;
   std::optional<ValidFramesConsumer> own_frames_consumer_;
   ValidFramesConsumer::Iterator own_frames_it_;

From a0274bbdd91759ef303d091ab38d8de179e56e98 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 10:42:09 +0100
Subject: [PATCH 22/90] Prevent reexecution of cursor when no output symbols
 are present

---
 src/query/v2/interpreter.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 23386bd1c..b447a0a2b 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -778,6 +778,8 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
           break;
         }
       }
+    } else {
+      multi_frame_.MakeAllFramesInvalid();
     }
   }
 

From 52baaf80303561746a4cd8f6553710ceca36ff0a Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 10:43:07 +0100
Subject: [PATCH 23/90] Detect when no work should be done because of lack of
 input data in cursors

---
 src/query/v2/plan/operator.cpp | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index d343afbfc..82a1df4ee 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -193,6 +193,9 @@ class DistributedCreateNodeCursor : public Cursor {
     SCOPED_PROFILE_OP("CreateNodeMF");
     input_cursor_->PullMultiple(multi_frame, context);
     auto *request_router = context.request_router;
+    if (!multi_frame.HasValidFrame()) {
+      return;
+    }
     {
       SCOPED_REQUEST_WAIT_PROFILE;
       request_router->CreateVertices(NodeCreationInfoToRequests(context, multi_frame));
@@ -2484,6 +2487,9 @@ class DistributedCreateExpandCursor : public Cursor {
   void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("CreateExpandMF");
     input_cursor_->PullMultiple(multi_frame, context);
+    if (!multi_frame.HasValidFrame()) {
+      return;
+    }
     auto request_vertices = ExpandCreationInfoToRequests(multi_frame, context);
     {
       SCOPED_REQUEST_WAIT_PROFILE;

From d1548c9253d82b24b4519b0217dec2dc945d9643 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 10:43:54 +0100
Subject: [PATCH 24/90] Eliminate fully

---
 src/query/v2/plan/operator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 82a1df4ee..4381da0dd 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2922,7 +2922,7 @@ class DistributedExpandCursor : public Cursor {
                  output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
               auto &edge = *current;
               ++current;
-              auto &output_frame = *output_frame_it++;
+              auto &output_frame = *output_frame_it;
               output_frame = *own_frames_it_;
               output_frame[self_.common_.edge_symbol] = edge;
               PullDstVertex(output_frame, context, direction);

From ca62fa51233f865517f05d072b556b3c86a95fae Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 11:42:24 +0100
Subject: [PATCH 25/90] Fetch properties of destination vertex

---
 src/query/v2/plan/operator.cpp |  9 +++++++--
 src/storage/v3/shard_rsm.cpp   | 11 +++++++----
 2 files changed, 14 insertions(+), 6 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 4381da0dd..15f34fe34 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2681,9 +2681,14 @@ class DistributedExpandCursor : public Cursor {
       }
     };
 
+    msgs::GetPropertiesRequest request;
+    // to not fetch any properties of the edges
+    request.vertex_ids.push_back(get_dst_vertex(edge, direction));
+    auto result_rows = context.request_router->GetProperties(std::move(request));
+    MG_ASSERT(result_rows.size() == 1);
+    auto &result_row = result_rows.front();
     frame[self_.common_.node_symbol] =
-        accessors::VertexAccessor(msgs::Vertex{get_dst_vertex(edge, direction)},
-                                  std::vector<std::pair<msgs::PropertyId, msgs::Value>>{}, context.request_router);
+        accessors::VertexAccessor(msgs::Vertex{result_row.vertex}, result_row.props, context.request_router);
   }
 
   bool InitEdges(Frame &frame, ExecutionContext &context) {
diff --git a/src/storage/v3/shard_rsm.cpp b/src/storage/v3/shard_rsm.cpp
index b919d217c..67a58e5cc 100644
--- a/src/storage/v3/shard_rsm.cpp
+++ b/src/storage/v3/shard_rsm.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -535,13 +535,16 @@ msgs::ReadResponses ShardRsm::HandleRead(msgs::GetPropertiesRequest &&req) {
     return result;
   };
 
-  auto collect_props = [&req](const VertexAccessor &v_acc,
-                              const std::optional<EdgeAccessor> &e_acc) -> ShardResult<std::map<PropertyId, Value>> {
+  auto collect_props = [this, &req](
+                           const VertexAccessor &v_acc,
+                           const std::optional<EdgeAccessor> &e_acc) -> ShardResult<std::map<PropertyId, Value>> {
     if (!req.property_ids) {
       if (e_acc) {
         return CollectAllPropertiesFromAccessor(*e_acc, view);
       }
-      return CollectAllPropertiesFromAccessor(v_acc, view);
+      const auto *schema = shard_->GetSchema(shard_->PrimaryLabel());
+      MG_ASSERT(schema);
+      return CollectAllPropertiesFromAccessor(v_acc, view, *schema);
     }
 
     if (e_acc) {

From a0ada914abb1e8c0fa40e1d0d449f74c876d027f Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Thu, 19 Jan 2023 13:10:53 +0100
Subject: [PATCH 26/90] Fix segfault

---
 src/query/v2/interpreter.hpp | 1 -
 1 file changed, 1 deletion(-)

diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 8a535bf3c..8f754b84c 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -388,7 +388,6 @@ std::map<std::string, TypedValue> Interpreter::Pull(TStream *result_stream, std:
     // Wrap the (statically polymorphic) stream type into a common type which
     // the handler knows.
     AnyStream stream{result_stream, &query_execution->execution_memory};
-    auto asd = query_execution->prepared_query->query_handler;
     const auto maybe_res = query_execution->prepared_query->query_handler(&stream, n);
     // Stream is using execution memory of the query_execution which
     // can be deleted after its execution so the stream should be cleared

From ede6281e0005da20a6d9372b0a09e102bf37e923 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 16:58:11 +0100
Subject: [PATCH 27/90] Fix unit tests

---
 tests/simulation/shard_rsm.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/tests/simulation/shard_rsm.cpp b/tests/simulation/shard_rsm.cpp
index 768217945..8a79cb23b 100644
--- a/tests/simulation/shard_rsm.cpp
+++ b/tests/simulation/shard_rsm.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -1316,7 +1316,7 @@ void TestGetProperties(ShardClient &client) {
     MG_ASSERT(!result.result_row.empty());
     MG_ASSERT(result.result_row.size() == 3);
     for (const auto &elem : result.result_row) {
-      MG_ASSERT(elem.props.size() == 3);
+      MG_ASSERT(elem.props.size() == 4);
     }
   }
   {

From 6fe244b20926b3be396423dc966572f7d7fd37e7 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Thu, 19 Jan 2023 17:16:29 +0100
Subject: [PATCH 28/90] Add missing MOCK_METHODS to MockedRequestRouter

---
 tests/unit/mock_helpers.hpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/tests/unit/mock_helpers.hpp b/tests/unit/mock_helpers.hpp
index c522b8602..89ae9bb7d 100644
--- a/tests/unit/mock_helpers.hpp
+++ b/tests/unit/mock_helpers.hpp
@@ -42,6 +42,8 @@ class MockedRequestRouter : public RequestRouterInterface {
   MOCK_METHOD(std::optional<storage::v3::LabelId>, MaybeNameToLabel, (const std::string &), (const));
   MOCK_METHOD(bool, IsPrimaryLabel, (storage::v3::LabelId), (const));
   MOCK_METHOD(bool, IsPrimaryKey, (storage::v3::LabelId, storage::v3::PropertyId), (const));
+  MOCK_METHOD(std::optional<std::pair<uint64_t, uint64_t>>, AllocateInitialEdgeIds, (io::Address));
+  MOCK_METHOD(void, InstallSimulatorTicker, (std::function<bool()>));
 };
 
 class MockedLogicalOperator : public plan::LogicalOperator {

From e7f10ec8f482d7c99d94b203ecdff54ad0f2a21c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 19 Jan 2023 17:44:03 +0100
Subject: [PATCH 29/90] Remove duplicated definition

---
 src/query/v2/multiframe.cpp | 4 ----
 1 file changed, 4 deletions(-)

diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index a6d34ee11..38cc7549a 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -52,10 +52,6 @@ bool MultiFrame::HasInvalidFrame() const noexcept {
   return std::any_of(frames_.rbegin(), frames_.rend(), [](const auto &frame) { return !frame.IsValid(); });
 }
 
-bool MultiFrame::HasInvalidFrame() const noexcept {
-  return std::any_of(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); });
-}
-
 // NOLINTNEXTLINE (bugprone-exception-escape)
 void MultiFrame::DefragmentValidFrames() noexcept {
   /*

From cc643aac69eee6b28d5f48d3cb42a75efa22c1e1 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Thu, 19 Jan 2023 17:45:15 +0100
Subject: [PATCH 30/90] Deal with unprotected comma in MOCKED_METHOD

---
 tests/unit/mock_helpers.hpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/unit/mock_helpers.hpp b/tests/unit/mock_helpers.hpp
index 89ae9bb7d..278649e47 100644
--- a/tests/unit/mock_helpers.hpp
+++ b/tests/unit/mock_helpers.hpp
@@ -42,7 +42,7 @@ class MockedRequestRouter : public RequestRouterInterface {
   MOCK_METHOD(std::optional<storage::v3::LabelId>, MaybeNameToLabel, (const std::string &), (const));
   MOCK_METHOD(bool, IsPrimaryLabel, (storage::v3::LabelId), (const));
   MOCK_METHOD(bool, IsPrimaryKey, (storage::v3::LabelId, storage::v3::PropertyId), (const));
-  MOCK_METHOD(std::optional<std::pair<uint64_t, uint64_t>>, AllocateInitialEdgeIds, (io::Address));
+  MOCK_METHOD((std::optional<std::pair<uint64_t, uint64_t>>), AllocateInitialEdgeIds, (io::Address));
   MOCK_METHOD(void, InstallSimulatorTicker, (std::function<bool()>));
 };
 

From be39fac72ebc80d91f70845322e686df81061ed3 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 15:36:24 +0100
Subject: [PATCH 31/90] Add return value to `PullMultiple`

Because the `FilterCursor` might push down the same multiframe multiple
times, it is easier if each cursor maintains whether it put any new data
on the `MultiFrame` or not. This way each cursor can decide easily
whether it has to do more work or not.
---
 src/query/v2/interpreter.cpp   |  5 +--
 src/query/v2/plan/operator.cpp | 64 +++++++++++++++++++++-------------
 src/query/v2/plan/operator.lcp |  6 ++--
 3 files changed, 43 insertions(+), 32 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index b447a0a2b..c7943e6df 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -731,10 +731,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
   }
 
   // Returns true if a result was pulled.
-  const auto pull_result = [&]() -> bool {
-    cursor_->PullMultiple(multi_frame_, ctx_);
-    return multi_frame_.HasValidFrame();
-  };
+  const auto pull_result = [&]() -> bool { return cursor_->PullMultiple(multi_frame_, ctx_); };
 
   const auto stream_values = [&output_symbols, &stream](const Frame &frame) {
     // TODO: The streamed values should also probably use the above memory.
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 15f34fe34..1ac5d98b8 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -189,18 +189,19 @@ class DistributedCreateNodeCursor : public Cursor {
     return false;
   }
 
-  void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
+  bool PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("CreateNodeMF");
-    input_cursor_->PullMultiple(multi_frame, context);
+
     auto *request_router = context.request_router;
-    if (!multi_frame.HasValidFrame()) {
-      return;
+    if (!input_cursor_->PullMultiple(multi_frame, context)) {
+      return false;
     }
     {
       SCOPED_REQUEST_WAIT_PROFILE;
       request_router->CreateVertices(NodeCreationInfoToRequests(context, multi_frame));
     }
     PlaceNodesOnTheMultiFrame(multi_frame, context);
+    return false;
   }
 
   void Shutdown() override { input_cursor_->Shutdown(); }
@@ -324,14 +325,16 @@ bool Once::OnceCursor::Pull(Frame &, ExecutionContext &context) {
   return false;
 }
 
-void Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) {
+bool Once::OnceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) {
   SCOPED_PROFILE_OP("OnceMF");
 
   if (!did_pull_) {
     auto &first_frame = multi_frame.GetFirstFrame();
     first_frame.MakeValid();
     did_pull_ = true;
+    return true;
   }
+  return false;
 }
 
 UniqueCursorPtr Once::MakeCursor(utils::MemoryResource *mem) const {
@@ -491,10 +494,10 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   }
 
   bool PullNextFrames(ExecutionContext &context) {
-    input_cursor_->PullMultiple(*own_multi_frame_, context);
+    const auto pulled_any = input_cursor_->PullMultiple(*own_multi_frame_, context);
     own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
     own_frames_it_ = own_frames_consumer_->begin();
-    return own_multi_frame_->HasValidFrame();
+    return pulled_any;
   }
 
   inline bool HasMoreResult() {
@@ -521,7 +524,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     return true;
   }
 
-  void PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
     if (!own_multi_frame_.has_value()) {
@@ -533,17 +536,20 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     }
 
     if (!HasMoreResult()) {
-      return;
+      return false;
     }
 
+    bool populated_any = false;
     for (auto &frame : output_multi_frame.GetInvalidFramesPopulator()) {
       if (MustAbort(context)) {
         throw HintedAbortError();
       }
       if (!PopulateFrame(context, frame)) {
-        return;
+        break;
       }
+      populated_any = true;
     }
+    return populated_any;
   };
 
   void Shutdown() override { input_cursor_->Shutdown(); }
@@ -859,19 +865,22 @@ bool Produce::ProduceCursor::Pull(Frame &frame, ExecutionContext &context) {
     // Produce should always yield the latest results.
     ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
                                   storage::v3::View::NEW);
-    for (auto named_expr : self_.named_expressions_) named_expr->Accept(evaluator);
+    for (auto *named_expr : self_.named_expressions_) named_expr->Accept(evaluator);
 
     return true;
   }
   return false;
 }
 
-void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) {
+bool Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) {
   SCOPED_PROFILE_OP("ProduceMF");
 
-  input_cursor_->PullMultiple(multi_frame, context);
+  if (!input_cursor_->PullMultiple(multi_frame, context)) {
+    return false;
+  }
 
   auto iterator_for_valid_frame_only = multi_frame.GetValidFramesModifier();
+
   for (auto &frame : iterator_for_valid_frame_only) {
     // Produce should always yield the latest results.
     ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
@@ -881,7 +890,9 @@ void Produce::ProduceCursor::PullMultiple(MultiFrame &multi_frame, ExecutionCont
       named_expr->Accept(evaluator);
     }
   }
-};
+
+  return true;
+}
 
 void Produce::ProduceCursor::Shutdown() { input_cursor_->Shutdown(); }
 
@@ -2484,11 +2495,10 @@ class DistributedCreateExpandCursor : public Cursor {
     return true;
   }
 
-  void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
+  bool PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("CreateExpandMF");
-    input_cursor_->PullMultiple(multi_frame, context);
-    if (!multi_frame.HasValidFrame()) {
-      return;
+    if (!input_cursor_->PullMultiple(multi_frame, context)) {
+      return false;
     }
     auto request_vertices = ExpandCreationInfoToRequests(multi_frame, context);
     {
@@ -2501,6 +2511,7 @@ class DistributedCreateExpandCursor : public Cursor {
         }
       }
     }
+    return true;
   }
 
   void Shutdown() override { input_cursor_->Shutdown(); }
@@ -2847,12 +2858,12 @@ class DistributedExpandCursor : public Cursor {
   }
 
   bool PullInputFrames(ExecutionContext &context) {
-    input_cursor_->PullMultiple(*own_multi_frame_, context);
+    const auto pulled_any = !input_cursor_->PullMultiple(*own_multi_frame_, context);
     // These needs to be updated regardless of the result of the pull, otherwise the consumer and iterator might
     // get corrupted because of the operations done on our MultiFrame.
     own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
     own_frames_it_ = own_frames_consumer_->begin();
-    if (!own_multi_frame_->HasValidFrame()) {
+    if (!pulled_any) {
       return false;
     }
 
@@ -2883,20 +2894,21 @@ class DistributedExpandCursor : public Cursor {
     return true;
   }
 
-  void PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("DistributedExpandMF");
     MG_ASSERT(!self_.common_.existing_node);
     EnsureOwnMultiFrameIsGood(output_multi_frame);
     // A helper function for expanding a node from an edge.
 
     auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+    auto populated_any = false;
 
     while (true) {
       switch (state_) {
         case State::PullInputAndEdges: {
           if (!PullInputFrames(context)) {
             state_ = State::Exhausted;
-            return;
+            return populated_any;
           }
           state_ = State::InitInOutEdgesIt;
           break;
@@ -2912,7 +2924,7 @@ class DistributedExpandCursor : public Cursor {
         }
         case State::PopulateOutput: {
           if (!output_multi_frame.HasInvalidFrame()) {
-            return;
+            return populated_any;
           }
           if (current_in_edge_it_ == current_in_edges_.end() && current_out_edge_it_ == current_out_edges_.end()) {
             own_frames_it_->MakeInvalid();
@@ -2920,7 +2932,7 @@ class DistributedExpandCursor : public Cursor {
             state_ = State::InitInOutEdgesIt;
             continue;
           }
-          auto populate_edges = [this, &context, &output_frames_populator](
+          auto populate_edges = [this, &context, &output_frames_populator, &populated_any](
                                     const EdgeAtom::Direction direction, std::vector<EdgeAccessor>::iterator &current,
                                     const std::vector<EdgeAccessor>::iterator &end) {
             for (auto output_frame_it = output_frames_populator.begin();
@@ -2931,6 +2943,7 @@ class DistributedExpandCursor : public Cursor {
               output_frame = *own_frames_it_;
               output_frame[self_.common_.edge_symbol] = edge;
               PullDstVertex(output_frame, context, direction);
+              populated_any = true;
             }
           };
           populate_edges(EdgeAtom::Direction::IN, current_in_edge_it_, current_in_edges_.end());
@@ -2938,10 +2951,11 @@ class DistributedExpandCursor : public Cursor {
           break;
         }
         case State::Exhausted: {
-          return;
+          return populated_any;
         }
       }
     }
+    return populated_any;
   }
 
   void EnsureOwnMultiFrameIsGood(MultiFrame &output_multi_frame) {
diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index efa0d5df0..c31ae9762 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -72,7 +72,7 @@ class Cursor {
   /// @throws QueryRuntimeException if something went wrong with execution
   virtual bool Pull(Frame &, ExecutionContext &) = 0;
 
-  virtual void PullMultiple(MultiFrame &, ExecutionContext &) { LOG_FATAL("PullMultipleIsNotImplemented"); }
+  virtual bool PullMultiple(MultiFrame &, ExecutionContext &) { LOG_FATAL("PullMultipleIsNotImplemented"); }
 
   /// Resets the Cursor to its initial state.
   virtual void Reset() = 0;
@@ -335,7 +335,7 @@ and false on every following Pull.")
    class OnceCursor : public Cursor {
     public:
      OnceCursor() {}
-     void PullMultiple(MultiFrame &, ExecutionContext &) override;
+     bool PullMultiple(MultiFrame &, ExecutionContext &) override;
      bool Pull(Frame &, ExecutionContext &) override;
      void Shutdown() override;
      void Reset() override;
@@ -1211,7 +1211,7 @@ RETURN clause) the Produce's pull succeeds exactly once.")
     public:
      ProduceCursor(const Produce &, utils::MemoryResource *);
      bool Pull(Frame &, ExecutionContext &) override;
-     void PullMultiple(MultiFrame &, ExecutionContext &) override;
+     bool PullMultiple(MultiFrame &, ExecutionContext &) override;
      void Shutdown() override;
      void Reset() override;
 

From de99025c39a1cca5e17e3033d40a009cc0af3d52 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 15:36:38 +0100
Subject: [PATCH 32/90] Implement `PullMultiple` for `FilterCursor`

---
 src/query/v2/plan/operator.cpp | 21 +++++++++++++++++++++
 src/query/v2/plan/operator.lcp |  1 +
 2 files changed, 22 insertions(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 1ac5d98b8..676576098 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -830,6 +830,27 @@ bool Filter::FilterCursor::Pull(Frame &frame, ExecutionContext &context) {
   return false;
 }
 
+bool Filter::FilterCursor::PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) {
+  SCOPED_PROFILE_OP("Filter");
+  auto populated_any = false;
+
+  while (multi_frame.HasInvalidFrame()) {
+    if (!input_cursor_->PullMultiple(multi_frame, context)) {
+      return populated_any;
+    }
+    for (auto &frame : multi_frame.GetValidFramesConsumer()) {
+      ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
+                                    storage::v3::View::OLD);
+      if (!EvaluateFilter(evaluator, self_.expression_)) {
+        frame.MakeInvalid();
+      } else {
+        populated_any = true;
+      }
+    }
+  }
+  return populated_any;
+}
+
 void Filter::FilterCursor::Shutdown() { input_cursor_->Shutdown(); }
 
 void Filter::FilterCursor::Reset() { input_cursor_->Reset(); }
diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index c31ae9762..65a562be3 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -1160,6 +1160,7 @@ a boolean value.")
     public:
      FilterCursor(const Filter &, utils::MemoryResource *);
      bool Pull(Frame &, ExecutionContext &) override;
+     bool PullMultiple(MultiFrame &, ExecutionContext &) override;
      void Shutdown() override;
      void Reset() override;
 

From c9299a6c72c7895be45115f7e53d1a04c69e7ef3 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 21:37:57 +0100
Subject: [PATCH 33/90] Turn the scan all cursor into an automaton

---
 src/query/v2/plan/operator.cpp | 105 ++++++++++++++++++---------------
 1 file changed, 58 insertions(+), 47 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 676576098..b8449c4bb 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -455,8 +455,6 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     ResetExecutionState();
   }
 
-  enum class State : int8_t { INITIALIZING, COMPLETED };
-
   using VertexAccessor = accessors::VertexAccessor;
 
   bool MakeRequest(ExecutionContext &context) {
@@ -493,61 +491,71 @@ class DistributedScanAllAndFilterCursor : public Cursor {
     }
   }
 
-  bool PullNextFrames(ExecutionContext &context) {
-    const auto pulled_any = input_cursor_->PullMultiple(*own_multi_frame_, context);
-    own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
-    own_frames_it_ = own_frames_consumer_->begin();
-    return pulled_any;
-  }
-
-  inline bool HasMoreResult() {
-    return current_vertex_it_ != current_batch_.end() && own_frames_it_ != own_frames_consumer_->end();
-  }
-
-  bool PopulateFrame(ExecutionContext &context, FrameWithValidity &frame) {
-    MG_ASSERT(HasMoreResult());
-
-    frame = *own_frames_it_;
-    frame[output_symbol_] = TypedValue(*current_vertex_it_);
-
-    ++current_vertex_it_;
-    if (current_vertex_it_ == current_batch_.end()) {
-      own_frames_it_->MakeInvalid();
-      ++own_frames_it_;
-
-      current_vertex_it_ = current_batch_.begin();
-
-      if (own_frames_it_ == own_frames_consumer_->end()) {
-        return PullNextFrames(context);
-      }
-    };
-    return true;
-  }
-
   bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
                                           kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
-
-      MakeRequest(context);
-      PullNextFrames(context);
+      own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+      own_frames_it_ = own_frames_consumer_->begin();
     }
 
-    if (!HasMoreResult()) {
-      return false;
-    }
+    auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+    auto populated_any = false;
 
-    bool populated_any = false;
-    for (auto &frame : output_multi_frame.GetInvalidFramesPopulator()) {
-      if (MustAbort(context)) {
-        throw HintedAbortError();
+    while (true) {
+      switch (state_) {
+        case State::PullInput: {
+          if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
+            state_ = State::Exhausted;
+            return populated_any;
+          }
+          own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+          own_frames_it_ = own_frames_consumer_->begin();
+          state_ = State::FetchVertices;
+          break;
+        }
+        case State::FetchVertices: {
+          if (own_frames_it_ == own_frames_consumer_->end()) {
+            state_ = State::PullInput;
+            continue;
+          }
+          if (!filter_expressions_->empty() || property_expression_pair_.has_value() || current_batch_.empty()) {
+            MakeRequest(context);
+          } else {
+            // We can reuse the vertices as they don't depend on any value from the frames
+            current_vertex_it_ = current_batch_.begin();
+          }
+          state_ = State::PopulateOutput;
+          break;
+        }
+        case State::PopulateOutput: {
+          if (!output_multi_frame.HasInvalidFrame()) {
+            return populated_any;
+          }
+          if (current_vertex_it_ == current_batch_.end()) {
+            own_frames_it_->MakeInvalid();
+            ++own_frames_it_;
+            state_ = State::FetchVertices;
+            continue;
+          }
+
+          for (auto output_frame_it = output_frames_populator.begin();
+               output_frame_it != output_frames_populator.end() && current_vertex_it_ != current_batch_.end();
+               ++output_frame_it) {
+            auto &output_frame = *output_frame_it;
+            output_frame = *own_frames_it_;
+            output_frame[output_symbol_] = TypedValue(*current_vertex_it_);
+            current_vertex_it_++;
+            populated_any = true;
+          }
+          break;
+        }
+        case State::Exhausted: {
+          return populated_any;
+        }
       }
-      if (!PopulateFrame(context, frame)) {
-        break;
-      }
-      populated_any = true;
     }
     return populated_any;
   };
@@ -565,6 +573,9 @@ class DistributedScanAllAndFilterCursor : public Cursor {
   }
 
  private:
+  enum class State { PullInput, FetchVertices, PopulateOutput, Exhausted };
+
+  State state_{State::PullInput};
   const Symbol output_symbol_;
   const UniqueCursorPtr input_cursor_;
   const char *op_name_;

From 0eee3ad7b7c43176b563e1e7eb7217f96be1e22f Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 21:38:35 +0100
Subject: [PATCH 34/90] Fix `DistributedExpandCursor`

---
 src/query/v2/plan/operator.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index b8449c4bb..4461ab580 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2890,7 +2890,7 @@ class DistributedExpandCursor : public Cursor {
   }
 
   bool PullInputFrames(ExecutionContext &context) {
-    const auto pulled_any = !input_cursor_->PullMultiple(*own_multi_frame_, context);
+    const auto pulled_any = input_cursor_->PullMultiple(*own_multi_frame_, context);
     // These needs to be updated regardless of the result of the pull, otherwise the consumer and iterator might
     // get corrupted because of the operations done on our MultiFrame.
     own_frames_consumer_ = own_multi_frame_->GetValidFramesConsumer();
@@ -2949,7 +2949,7 @@ class DistributedExpandCursor : public Cursor {
           if (own_frames_it_ == own_frames_consumer_->end()) {
             state_ = State::PullInputAndEdges;
           } else {
-            InitEdges(*own_frames_it_, context);
+            InitEdgesMultiple(context);
             state_ = State::PopulateOutput;
           }
           break;

From 55b5d7609244c4414a4e5b3f54cf098925016a6c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 21:38:51 +0100
Subject: [PATCH 35/90] Add docs to `PullMultiple`

---
 src/query/v2/plan/operator.lcp | 16 +++++++++++++++-
 1 file changed, 15 insertions(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index 65a562be3..82e98df03 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -72,7 +72,21 @@ class Cursor {
   /// @throws QueryRuntimeException if something went wrong with execution
   virtual bool Pull(Frame &, ExecutionContext &) = 0;
 
-  virtual bool PullMultiple(MultiFrame &, ExecutionContext &) { LOG_FATAL("PullMultipleIsNotImplemented"); }
+  /// Run an iteration of a @c LogicalOperator with MultiFrame.
+  ///
+  /// Since operators may be chained, the iteration may pull results from
+  /// multiple operators.
+  ///
+  /// @param MultiFrame May be read from or written to while performing the
+  ///     iteration.
+  /// @param ExecutionContext Used to get the position of symbols in frame and
+  ///     other information.
+  /// @return True if the operator was able to populate at least one Frame on the MultiFrame,
+  ///     thus if an operator returns true, that means there is at least one valid Frame in the
+  ///     MultiFrame.
+  ///
+  /// @throws QueryRuntimeException if something went wrong with execution
+  virtual bool PullMultiple(MultiFrame &, ExecutionContext &) {MG_ASSERT(false, "PullMultipleIsNotImplemented"); return false; }
 
   /// Resets the Cursor to its initial state.
   virtual void Reset() = 0;

From 515a52130e554c81a6d9cd3e2f28c0d48b716af0 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 22:12:24 +0100
Subject: [PATCH 36/90] Prevent moving from valid frames during defregmentation
 of `MultiFrame`

---
 src/query/v2/multiframe.cpp | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index 38cc7549a..477ef6c0c 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -54,15 +54,17 @@ bool MultiFrame::HasInvalidFrame() const noexcept {
 
 // NOLINTNEXTLINE (bugprone-exception-escape)
 void MultiFrame::DefragmentValidFrames() noexcept {
-  /*
-  from: https://en.cppreference.com/w/cpp/algorithm/remove
-  "Removing is done by shifting (by means of copy assignment (until C++11)move assignment (since C++11)) the elements
-  in the range in such a way that the elements that are not to be removed appear in the beginning of the range.
-  Relative order of the elements that remain is preserved and the physical size of the container is unchanged."
-  */
-
-  // NOLINTNEXTLINE (bugprone-unused-return-value)
-  std::remove_if(frames_.begin(), frames_.end(), [](auto &frame) { return !frame.IsValid(); });
+  static constexpr auto kIsValid = [](const FrameWithValidity &frame) { return frame.IsValid(); };
+  static constexpr auto kIsInvalid = [](const FrameWithValidity &frame) { return !frame.IsValid(); };
+  auto first_invalid_frame = std::find_if(frames_.begin(), frames_.end(), kIsInvalid);
+  auto following_first_valid = std::find_if(first_invalid_frame, frames_.end(), kIsValid);
+  while (first_invalid_frame != frames_.end() && following_first_valid != frames_.end()) {
+    std::swap(*first_invalid_frame, *following_first_valid);
+    first_invalid_frame++;
+    first_invalid_frame = std::find_if(first_invalid_frame, frames_.end(), kIsInvalid);
+    following_first_valid++;
+    following_first_valid = std::find_if(following_first_valid, frames_.end(), kIsValid);
+  }
 }
 
 ValidFramesReader MultiFrame::GetValidFramesReader() { return ValidFramesReader{*this}; }

From 0285b5691518323d67abb9b168a37e914f12fbdd Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 23:01:23 +0100
Subject: [PATCH 37/90] Fix compilation error

---
 tests/unit/mock_helpers.hpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/unit/mock_helpers.hpp b/tests/unit/mock_helpers.hpp
index c522b8602..302b2ff55 100644
--- a/tests/unit/mock_helpers.hpp
+++ b/tests/unit/mock_helpers.hpp
@@ -58,7 +58,7 @@ class MockedLogicalOperator : public plan::LogicalOperator {
 class MockedCursor : public plan::Cursor {
  public:
   MOCK_METHOD(bool, Pull, (Frame &, expr::ExecutionContext &));
-  MOCK_METHOD(void, PullMultiple, (MultiFrame &, expr::ExecutionContext &));
+  MOCK_METHOD(bool, PullMultiple, (MultiFrame &, expr::ExecutionContext &));
   MOCK_METHOD(void, Reset, ());
   MOCK_METHOD(void, Shutdown, ());
 };

From 544c75c212f8bcac954406c45c1cdfaead8e4b61 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 23:04:33 +0100
Subject: [PATCH 38/90] Add explanation about limitations of current
 implementation

---
 src/query/v2/plan/operator.cpp | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 4461ab580..ad91da728 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2841,14 +2841,19 @@ class DistributedExpandCursor : public Cursor {
   }
 
   void InitEdgesMultiple(ExecutionContext &context) {
-    TypedValue &vertex_value = (*own_frames_it_)[self_.input_symbol_];
+    // This function won't work if any vertex id is duplicated in the input, because:
+    //  1. vertex_id_to_result_row is not a multimap
+    //  2. if self_.common_.existing_node is true, then we erase edges that might be necessary for the input vertex on a
+    //     later frame
+    const auto &frame = (*own_frames_it_);
+    const auto &vertex_value = frame[self_.input_symbol_];
 
     if (vertex_value.IsNull()) {
       return;
     }
 
     ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
-    auto &vertex = vertex_value.ValueVertex();
+    const auto &vertex = vertex_value.ValueVertex();
 
     const auto convert_edges = [&vertex, &context](
                                    std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties> &&edge_messages,

From 900ece8109f3ee635d0bfef54cb42122008aea93 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 20 Jan 2023 23:04:52 +0100
Subject: [PATCH 39/90] Add `PullMultiple` to `DeleteCursor`

---
 src/query/v2/plan/operator.cpp | 2 ++
 src/query/v2/plan/operator.lcp | 1 +
 2 files changed, 3 insertions(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index ad91da728..fdb89168c 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -949,6 +949,8 @@ Delete::DeleteCursor::DeleteCursor(const Delete &self, utils::MemoryResource *me
 
 bool Delete::DeleteCursor::Pull(Frame & /*frame*/, ExecutionContext & /*context*/) { return false; }
 
+bool Delete::DeleteCursor::PullMultiple(MultiFrame & /*multi_frame*/, ExecutionContext & /*context*/) { return false; }
+
 void Delete::DeleteCursor::Shutdown() { input_cursor_->Shutdown(); }
 
 void Delete::DeleteCursor::Reset() { input_cursor_->Reset(); }
diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index 82e98df03..91726752b 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -1274,6 +1274,7 @@ Has a flag for using DETACH DELETE when deleting vertices.")
     public:
      DeleteCursor(const Delete &, utils::MemoryResource *);
      bool Pull(Frame &, ExecutionContext &) override;
+     bool PullMultiple(MultiFrame &, ExecutionContext &) override;
      void Shutdown() override;
      void Reset() override;
 

From b4ae8aea95298ffd2f875f78192fd0fa0fba9c1d Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Tue, 24 Jan 2023 15:59:50 +0100
Subject: [PATCH 40/90] Apply suggestions from code review

---
 src/io/local_transport/local_system.hpp       |  4 +--
 src/memgraph.cpp                              |  4 +--
 src/query/v2/interpreter.hpp                  |  3 +-
 src/query/v2/request_router.hpp               | 30 +++++--------------
 .../cluster_property_test_cypher_queries.cpp  |  2 +-
 5 files changed, 13 insertions(+), 30 deletions(-)

diff --git a/src/io/local_transport/local_system.hpp b/src/io/local_transport/local_system.hpp
index feea44244..7b0cda537 100644
--- a/src/io/local_transport/local_system.hpp
+++ b/src/io/local_transport/local_system.hpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -29,8 +29,6 @@ class LocalSystem {
     return Io{local_transport, address};
   }
 
-  std::shared_ptr<LocalTransportHandle> GetTransportHandle() const { return local_transport_handle_; }
-
   void ShutDown() { local_transport_handle_->ShutDown(); }
 };
 
diff --git a/src/memgraph.cpp b/src/memgraph.cpp
index cdb1e63df..35fd20ad7 100644
--- a/src/memgraph.cpp
+++ b/src/memgraph.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -640,7 +640,7 @@ int main(int argc, char **argv) {
   memgraph::machine_manager::MachineManager<memgraph::io::local_transport::LocalTransport> mm{io, config, coordinator};
   std::jthread mm_thread([&mm] { mm.Run(); });
 
-  auto rr_factory = std::make_unique<memgraph::query::v2::LocalRequestRouterFactory>(ls.GetTransportHandle());
+  auto rr_factory = std::make_unique<memgraph::query::v2::LocalRequestRouterFactory>(io);
 
   memgraph::query::v2::InterpreterContext interpreter_context{
       (memgraph::storage::v3::Shard *)(nullptr),
diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 8f754b84c..9c2bef3d2 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -191,6 +191,7 @@ struct InterpreterContext {
   IdAllocator edge_ids_alloc;
 
   coordinator::Address coordinator_address;
+  std::unique_ptr<RequestRouterFactory> request_router_factory_;
 
   storage::v3::LabelId NameToLabelId(std::string_view label_name) {
     return storage::v3::LabelId::FromUint(query_id_mapper_.NameToId(label_name));
@@ -204,8 +205,6 @@ struct InterpreterContext {
     return storage::v3::EdgeTypeId::FromUint(query_id_mapper_.NameToId(edge_type_name));
   }
 
-  std::unique_ptr<RequestRouterFactory> request_router_factory_;
-
  private:
   // TODO Replace with local map of labels, properties and edge type ids
   storage::v3::NameIdMapper query_id_mapper_;
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 0515633c9..252278bb8 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -753,11 +753,8 @@ class RequestRouterFactory {
 
   using TransportHandleVariant = std::variant<LocalTransportHandlePtr, SimulatorTransportHandlePtr>;
 
-  TransportHandleVariant transport_handle_;
-
  public:
-  explicit RequestRouterFactory(const TransportHandleVariant &transport_handle) : transport_handle_(transport_handle) {}
-
+  RequestRouterFactory() = default;
   RequestRouterFactory(const RequestRouterFactory &) = delete;
   RequestRouterFactory &operator=(const RequestRouterFactory &) = delete;
   RequestRouterFactory(RequestRouterFactory &&) = delete;
@@ -765,32 +762,22 @@ class RequestRouterFactory {
 
   virtual ~RequestRouterFactory() = default;
 
-  virtual TransportHandleVariant GetTransportHandle() { return transport_handle_; }
-
   virtual std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
       const coordinator::Address &coordinator_address) const = 0;
 };
 
 class LocalRequestRouterFactory : public RequestRouterFactory {
+  io::Io<memgraph::io::local_transport::LocalTransport> &io_;
+
  public:
-  explicit LocalRequestRouterFactory(const TransportHandleVariant &transport_handle)
-      : RequestRouterFactory(transport_handle) {}
+  explicit LocalRequestRouterFactory(io::Io<memgraph::io::local_transport::LocalTransport> &io) : io_(io) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
       const coordinator::Address &coordinator_address) const override {
     using TransportType = io::local_transport::LocalTransport;
-    auto actual_transport_handle = std::get<LocalTransportHandlePtr>(transport_handle_);
 
-    boost::uuids::uuid random_uuid;
-    io::Address unique_local_addr_query;
-
-    random_uuid = boost::uuids::uuid{boost::uuids::random_generator()()};
-    unique_local_addr_query = memgraph::coordinator::Address::UniqueLocalAddress();
-
-    TransportType local_transport(actual_transport_handle);
-    auto local_transport_io = io::Io<TransportType>(local_transport, unique_local_addr_query);
-
-    auto query_io = local_transport_io.ForkLocal(random_uuid);
+    auto query_io = io_.ForkLocal(boost::uuids::uuid{boost::uuids::random_generator()()});
+    auto local_transport_io = io_.ForkLocal(boost::uuids::uuid{boost::uuids::random_generator()()});
 
     return std::make_unique<RequestRouter<TransportType>>(
         coordinator::CoordinatorClient<TransportType>(query_io, coordinator_address, {coordinator_address}),
@@ -802,13 +789,12 @@ class SimulatedRequestRouterFactory : public RequestRouterFactory {
   io::simulator::Simulator *simulator_;
 
  public:
-  explicit SimulatedRequestRouterFactory(io::simulator::Simulator &simulator)
-      : RequestRouterFactory(simulator.GetSimulatorHandle()), simulator_(&simulator) {}
+  explicit SimulatedRequestRouterFactory(io::simulator::Simulator &simulator) : simulator_(&simulator) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
       const coordinator::Address &coordinator_address) const override {
     using TransportType = io::simulator::SimulatorTransport;
-    auto actual_transport_handle = std::get<SimulatorTransportHandlePtr>(transport_handle_);
+    auto actual_transport_handle = simulator_->GetSimulatorHandle();
 
     boost::uuids::uuid random_uuid;
     io::Address unique_local_addr_query;
diff --git a/tests/simulation/cluster_property_test_cypher_queries.cpp b/tests/simulation/cluster_property_test_cypher_queries.cpp
index 4996b4c2f..e35edc033 100644
--- a/tests/simulation/cluster_property_test_cypher_queries.cpp
+++ b/tests/simulation/cluster_property_test_cypher_queries.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source

From fa8eee2043c0a465a4cfc2ce077fc6f993a842ca Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 24 Jan 2023 16:19:24 +0100
Subject: [PATCH 41/90] Use destination vertex as other end for out edges

---
 src/storage/v3/request_helper.cpp | 24 +++++++++++++++---------
 1 file changed, 15 insertions(+), 9 deletions(-)

diff --git a/src/storage/v3/request_helper.cpp b/src/storage/v3/request_helper.cpp
index 6b889fe16..f13c5a82e 100644
--- a/src/storage/v3/request_helper.cpp
+++ b/src/storage/v3/request_helper.cpp
@@ -1,4 +1,4 @@
-// Copyright 2022 Memgraph Ltd.
+// Copyright 2023 Memgraph Ltd.
 //
 // Use of this software is governed by the Business Source License
 // included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@@ -321,12 +321,15 @@ EdgeFiller InitializeEdgeFillerFunction(const msgs::ExpandOneRequest &req) {
         value_properties.insert(std::make_pair(prop_key, FromPropertyValueToValue(std::move(prop_val))));
       }
       using EdgeWithAllProperties = msgs::ExpandOneResultRow::EdgeWithAllProperties;
-      EdgeWithAllProperties edges{ToMsgsVertexId(edge.From()), msgs::EdgeType{edge.EdgeType()}, edge.Gid().AsUint(),
-                                  std::move(value_properties)};
+
       if (is_in_edge) {
-        result_row.in_edges_with_all_properties.push_back(std::move(edges));
+        result_row.in_edges_with_all_properties.push_back(
+            EdgeWithAllProperties{ToMsgsVertexId(edge.From()), msgs::EdgeType{edge.EdgeType()}, edge.Gid().AsUint(),
+                                  std::move(value_properties)});
       } else {
-        result_row.out_edges_with_all_properties.push_back(std::move(edges));
+        result_row.out_edges_with_all_properties.push_back(
+            EdgeWithAllProperties{ToMsgsVertexId(edge.To()), msgs::EdgeType{edge.EdgeType()}, edge.Gid().AsUint(),
+                                  std::move(value_properties)});
       }
       return {};
     };
@@ -346,12 +349,15 @@ EdgeFiller InitializeEdgeFillerFunction(const msgs::ExpandOneRequest &req) {
         value_properties.emplace_back(FromPropertyValueToValue(std::move(property_result.GetValue())));
       }
       using EdgeWithSpecificProperties = msgs::ExpandOneResultRow::EdgeWithSpecificProperties;
-      EdgeWithSpecificProperties edges{ToMsgsVertexId(edge.From()), msgs::EdgeType{edge.EdgeType()},
-                                       edge.Gid().AsUint(), std::move(value_properties)};
+
       if (is_in_edge) {
-        result_row.in_edges_with_specific_properties.push_back(std::move(edges));
+        result_row.in_edges_with_specific_properties.push_back(
+            EdgeWithSpecificProperties{ToMsgsVertexId(edge.From()), msgs::EdgeType{edge.EdgeType()},
+                                       edge.Gid().AsUint(), std::move(value_properties)});
       } else {
-        result_row.out_edges_with_specific_properties.push_back(std::move(edges));
+        result_row.out_edges_with_specific_properties.push_back(
+            EdgeWithSpecificProperties{ToMsgsVertexId(edge.To()), msgs::EdgeType{edge.EdgeType()}, edge.Gid().AsUint(),
+                                       std::move(value_properties)});
       }
       return {};
     };

From 4908af5a188549f440506a06ebd62d0b91c3ed02 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 24 Jan 2023 16:33:15 +0100
Subject: [PATCH 42/90] Make `DistributedExpand` operator handle repeated
 vertices

---
 src/query/v2/plan/operator.cpp | 132 ++++++++++++++++++++-------------
 1 file changed, 79 insertions(+), 53 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index fdb89168c..0868f24a5 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2842,7 +2842,7 @@ class DistributedExpandCursor : public Cursor {
     }
   }
 
-  void InitEdgesMultiple(ExecutionContext &context) {
+  void InitEdgesMultiple() {
     // This function won't work if any vertex id is duplicated in the input, because:
     //  1. vertex_id_to_result_row is not a multimap
     //  2. if self_.common_.existing_node is true, then we erase edges that might be necessary for the input vertex on a
@@ -2851,49 +2851,28 @@ class DistributedExpandCursor : public Cursor {
     const auto &vertex_value = frame[self_.input_symbol_];
 
     if (vertex_value.IsNull()) {
+      ResetMultiFrameEdgeIts();
       return;
     }
 
     ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
     const auto &vertex = vertex_value.ValueVertex();
 
-    const auto convert_edges = [&vertex, &context](
-                                   std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties> &&edge_messages,
-                                   const EdgeAtom::Direction direction) {
-      std::vector<EdgeAccessor> edge_accessors;
-      edge_accessors.reserve(edge_messages.size());
+    current_vertex_ = &vertex;
 
-      switch (direction) {
-        case EdgeAtom::Direction::IN: {
-          for (auto &edge : edge_messages) {
-            edge_accessors.emplace_back(msgs::Edge{std::move(edge.other_end), vertex.Id(), {}, {edge.gid}, edge.type},
-                                        context.request_router);
-          }
-          break;
-        }
-        case EdgeAtom::Direction::OUT: {
-          for (auto &edge : edge_messages) {
-            edge_accessors.emplace_back(msgs::Edge{vertex.Id(), std::move(edge.other_end), {}, {edge.gid}, edge.type},
-                                        context.request_router);
-          }
-          break;
-        }
-        case EdgeAtom::Direction::BOTH: {
-          LOG_FATAL("Must indicate exact expansion direction here");
-        }
-      }
-      return edge_accessors;
-    };
+    auto &ref_counted_result_row = vertex_id_to_result_row.at(vertex.Id());
+    auto &result_row = *ref_counted_result_row.result_row;
 
-    auto *result_row = vertex_id_to_result_row[vertex.Id()];
-    current_in_edges_.clear();
-    current_in_edges_ =
-        convert_edges(std::move(result_row->in_edges_with_specific_properties), EdgeAtom::Direction::IN);
-    current_in_edge_it_ = current_in_edges_.begin();
-    current_out_edges_ =
-        convert_edges(std::move(result_row->out_edges_with_specific_properties), EdgeAtom::Direction::OUT);
-    current_out_edge_it_ = current_out_edges_.begin();
-    vertex_id_to_result_row.erase(vertex.Id());
+    current_in_edge_mf_it_ = result_row.in_edges_with_specific_properties.begin();
+    in_edges_end_it_ = result_row.in_edges_with_specific_properties.end();
+    current_out_edge_mf_it_ = result_row.out_edges_with_specific_properties.begin();
+    out_edges_end_it_ = result_row.out_edges_with_specific_properties.end();
+
+    if (ref_counted_result_row.ref_count == 1) {
+      vertex_id_to_result_row.erase(vertex.Id());
+    } else {
+      ref_counted_result_row.ref_count--;
+    }
   }
 
   bool PullInputFrames(ExecutionContext &context) {
@@ -2906,6 +2885,8 @@ class DistributedExpandCursor : public Cursor {
       return false;
     }
 
+    vertex_id_to_result_row.clear();
+
     msgs::ExpandOneRequest request;
     request.direction = DirectionToMsgsDirection(self_.common_.direction);
     // to not fetch any properties of the edges
@@ -2918,16 +2899,21 @@ class DistributedExpandCursor : public Cursor {
 
       ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
       const auto &vertex = vertex_value.ValueVertex();
-      request.src_vertices.push_back(vertex.Id());
+      auto [it, inserted] = vertex_id_to_result_row.try_emplace(vertex.Id(), RefCountedResultRow{1U, nullptr});
+
+      if (inserted) {
+        request.src_vertices.push_back(vertex.Id());
+      } else {
+        it->second.ref_count++;
+      }
     }
 
     result_rows_ = std::invoke([&context, &request]() mutable {
       SCOPED_REQUEST_WAIT_PROFILE;
       return context.request_router->ExpandOne(std::move(request));
     });
-    vertex_id_to_result_row.clear();
     for (auto &row : result_rows_) {
-      vertex_id_to_result_row[row.src_vertex.id] = &row;
+      vertex_id_to_result_row[row.src_vertex.id].result_row = &row;
     }
 
     return true;
@@ -2956,7 +2942,7 @@ class DistributedExpandCursor : public Cursor {
           if (own_frames_it_ == own_frames_consumer_->end()) {
             state_ = State::PullInputAndEdges;
           } else {
-            InitEdgesMultiple(context);
+            InitEdgesMultiple();
             state_ = State::PopulateOutput;
           }
           break;
@@ -2965,28 +2951,45 @@ class DistributedExpandCursor : public Cursor {
           if (!output_multi_frame.HasInvalidFrame()) {
             return populated_any;
           }
-          if (current_in_edge_it_ == current_in_edges_.end() && current_out_edge_it_ == current_out_edges_.end()) {
+          if (current_in_edge_mf_it_ == in_edges_end_it_ && current_out_edge_mf_it_ == out_edges_end_it_) {
             own_frames_it_->MakeInvalid();
             ++own_frames_it_;
             state_ = State::InitInOutEdgesIt;
             continue;
           }
           auto populate_edges = [this, &context, &output_frames_populator, &populated_any](
-                                    const EdgeAtom::Direction direction, std::vector<EdgeAccessor>::iterator &current,
-                                    const std::vector<EdgeAccessor>::iterator &end) {
+                                    const EdgeAtom::Direction direction, EdgesIterator &current,
+                                    const EdgesIterator &end) {
             for (auto output_frame_it = output_frames_populator.begin();
                  output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
               auto &edge = *current;
+              // auto &asd = edge.other_end.second[0];
               ++current;
               auto &output_frame = *output_frame_it;
               output_frame = *own_frames_it_;
-              output_frame[self_.common_.edge_symbol] = edge;
+              switch (direction) {
+                case EdgeAtom::Direction::IN: {
+                  output_frame[self_.common_.edge_symbol] =
+                      EdgeAccessor{msgs::Edge{edge.other_end, current_vertex_->Id(), {}, {edge.gid}, edge.type},
+                                   context.request_router};
+                  break;
+                }
+                case EdgeAtom::Direction::OUT: {
+                  output_frame[self_.common_.edge_symbol] =
+                      EdgeAccessor{msgs::Edge{current_vertex_->Id(), edge.other_end, {}, {edge.gid}, edge.type},
+                                   context.request_router};
+                  break;
+                }
+                case EdgeAtom::Direction::BOTH: {
+                  LOG_FATAL("Must indicate exact expansion direction here");
+                }
+              };
               PullDstVertex(output_frame, context, direction);
               populated_any = true;
             }
           };
-          populate_edges(EdgeAtom::Direction::IN, current_in_edge_it_, current_in_edges_.end());
-          populate_edges(EdgeAtom::Direction::OUT, current_out_edge_it_, current_out_edges_.end());
+          populate_edges(EdgeAtom::Direction::IN, current_in_edge_mf_it_, in_edges_end_it_);
+          populate_edges(EdgeAtom::Direction::OUT, current_out_edge_mf_it_, out_edges_end_it_);
           break;
         }
         case State::Exhausted: {
@@ -3017,28 +3020,51 @@ class DistributedExpandCursor : public Cursor {
     own_frames_consumer_.reset();
     own_multi_frame_->MakeAllFramesInvalid();
     state_ = State::PullInputAndEdges;
+
     current_in_edges_.clear();
     current_out_edges_.clear();
     current_in_edge_it_ = current_in_edges_.end();
     current_out_edge_it_ = current_out_edges_.end();
+
+    ResetMultiFrameEdgeIts();
   }
 
  private:
+  void ResetMultiFrameEdgeIts() {
+    in_edges_end_it_ = EdgesIterator{};
+    current_in_edge_mf_it_ = in_edges_end_it_;
+    out_edges_end_it_ = EdgesIterator{};
+    current_out_edge_mf_it_ = out_edges_end_it_;
+  }
   enum class State { PullInputAndEdges, InitInOutEdgesIt, PopulateOutput, Exhausted };
 
+  struct RefCountedResultRow {
+    size_t ref_count{0U};
+    msgs::ExpandOneResultRow *result_row{nullptr};
+  };
+
   const Expand &self_;
   const UniqueCursorPtr input_cursor_;
+  using EdgesVector = std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties>;
+  using EdgesIterator = EdgesVector::iterator;
+  EdgesIterator current_in_edge_mf_it_;
+  EdgesIterator in_edges_end_it_;
+  EdgesIterator current_out_edge_mf_it_;
+  EdgesIterator out_edges_end_it_;
+  State state_{State::PullInputAndEdges};
+  std::optional<MultiFrame> own_multi_frame_;
+  std::optional<ValidFramesConsumer> own_frames_consumer_;
+  const VertexAccessor *current_vertex_{nullptr};
+  ValidFramesConsumer::Iterator own_frames_it_;
+  std::vector<msgs::ExpandOneResultRow> result_rows_;
+  // This won't work if any vertex id is duplicated in the input
+  std::unordered_map<msgs::VertexId, RefCountedResultRow> vertex_id_to_result_row;
+
+  // TODO(antaljanosbenjamin): Remove when single frame approach is removed
   std::vector<EdgeAccessor> current_in_edges_;
   std::vector<EdgeAccessor> current_out_edges_;
   std::vector<EdgeAccessor>::iterator current_in_edge_it_;
   std::vector<EdgeAccessor>::iterator current_out_edge_it_;
-  State state_{State::PullInputAndEdges};
-  std::optional<MultiFrame> own_multi_frame_;
-  std::optional<ValidFramesConsumer> own_frames_consumer_;
-  ValidFramesConsumer::Iterator own_frames_it_;
-  std::vector<msgs::ExpandOneResultRow> result_rows_;
-  // This won't work if any vertex id is duplicated in the input
-  std::unordered_map<msgs::VertexId, msgs::ExpandOneResultRow *> vertex_id_to_result_row;
 };
 
 }  // namespace memgraph::query::v2::plan

From 7cb07672ffcfcd3b4b73563efdebeea5c2165305 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 24 Jan 2023 17:17:47 +0100
Subject: [PATCH 43/90] Make `DistributedExpandCursor` handle existing nodes
 with `MultiFrame`

---
 src/query/v2/plan/operator.cpp | 42 +++++++++++++++++++++++++---------
 1 file changed, 31 insertions(+), 11 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 0868f24a5..64a8c6f8c 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2865,8 +2865,10 @@ class DistributedExpandCursor : public Cursor {
 
     current_in_edge_mf_it_ = result_row.in_edges_with_specific_properties.begin();
     in_edges_end_it_ = result_row.in_edges_with_specific_properties.end();
+    AdvanceUntilSuitableEdge(current_in_edge_mf_it_, in_edges_end_it_);
     current_out_edge_mf_it_ = result_row.out_edges_with_specific_properties.begin();
     out_edges_end_it_ = result_row.out_edges_with_specific_properties.end();
+    AdvanceUntilSuitableEdge(current_out_edge_mf_it_, out_edges_end_it_);
 
     if (ref_counted_result_row.ref_count == 1) {
       vertex_id_to_result_row.erase(vertex.Id());
@@ -2921,7 +2923,6 @@ class DistributedExpandCursor : public Cursor {
 
   bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("DistributedExpandMF");
-    MG_ASSERT(!self_.common_.existing_node);
     EnsureOwnMultiFrameIsGood(output_multi_frame);
     // A helper function for expanding a node from an edge.
 
@@ -2963,8 +2964,6 @@ class DistributedExpandCursor : public Cursor {
             for (auto output_frame_it = output_frames_populator.begin();
                  output_frame_it != output_frames_populator.end() && current != end; ++output_frame_it) {
               auto &edge = *current;
-              // auto &asd = edge.other_end.second[0];
-              ++current;
               auto &output_frame = *output_frame_it;
               output_frame = *own_frames_it_;
               switch (direction) {
@@ -2985,6 +2984,8 @@ class DistributedExpandCursor : public Cursor {
                 }
               };
               PullDstVertex(output_frame, context, direction);
+              ++current;
+              AdvanceUntilSuitableEdge(current, end);
               populated_any = true;
             }
           };
@@ -3030,12 +3031,6 @@ class DistributedExpandCursor : public Cursor {
   }
 
  private:
-  void ResetMultiFrameEdgeIts() {
-    in_edges_end_it_ = EdgesIterator{};
-    current_in_edge_mf_it_ = in_edges_end_it_;
-    out_edges_end_it_ = EdgesIterator{};
-    current_out_edge_mf_it_ = out_edges_end_it_;
-  }
   enum class State { PullInputAndEdges, InitInOutEdgesIt, PopulateOutput, Exhausted };
 
   struct RefCountedResultRow {
@@ -3043,10 +3038,35 @@ class DistributedExpandCursor : public Cursor {
     msgs::ExpandOneResultRow *result_row{nullptr};
   };
 
+  using EdgeWithSpecificProperties = msgs::ExpandOneResultRow::EdgeWithSpecificProperties;
+  using EdgesVector = std::vector<EdgeWithSpecificProperties>;
+  using EdgesIterator = EdgesVector::iterator;
+
+  void ResetMultiFrameEdgeIts() {
+    in_edges_end_it_ = EdgesIterator{};
+    current_in_edge_mf_it_ = in_edges_end_it_;
+    out_edges_end_it_ = EdgesIterator{};
+    current_out_edge_mf_it_ = out_edges_end_it_;
+  }
+
+  void AdvanceUntilSuitableEdge(EdgesIterator &current, const EdgesIterator &end) {
+    if (!self_.common_.existing_node) {
+      return;
+    }
+
+    const auto &existing_node_value = (*own_frames_it_)[self_.common_.node_symbol];
+    if (existing_node_value.IsNull()) {
+      current = end;
+      return;
+    }
+    const auto &existing_node = existing_node_value.ValueVertex();
+    current = std::find_if(current, end, [&existing_node](const EdgeWithSpecificProperties &edge) {
+      return edge.other_end == existing_node.Id();
+    });
+  }
+
   const Expand &self_;
   const UniqueCursorPtr input_cursor_;
-  using EdgesVector = std::vector<msgs::ExpandOneResultRow::EdgeWithSpecificProperties>;
-  using EdgesIterator = EdgesVector::iterator;
   EdgesIterator current_in_edge_mf_it_;
   EdgesIterator in_edges_end_it_;
   EdgesIterator current_out_edge_mf_it_;

From e2a1029120faf4d44301da46ab67d7ed29ef61d4 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 24 Jan 2023 17:24:10 +0100
Subject: [PATCH 44/90] Fix simulation test

---
 tests/simulation/shard_rsm.cpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/tests/simulation/shard_rsm.cpp b/tests/simulation/shard_rsm.cpp
index 8a79cb23b..5c35b822a 100644
--- a/tests/simulation/shard_rsm.cpp
+++ b/tests/simulation/shard_rsm.cpp
@@ -1305,7 +1305,7 @@ void TestGetProperties(ShardClient &client) {
     MG_ASSERT(!result.error);
     MG_ASSERT(result.result_row.size() == 2);
     for (const auto &elem : result.result_row) {
-      MG_ASSERT(elem.props.size() == 3);
+      MG_ASSERT(elem.props.size() == 4);
     }
   }
   {
@@ -1316,7 +1316,7 @@ void TestGetProperties(ShardClient &client) {
     MG_ASSERT(!result.result_row.empty());
     MG_ASSERT(result.result_row.size() == 3);
     for (const auto &elem : result.result_row) {
-      MG_ASSERT(elem.props.size() == 4);
+      MG_ASSERT(elem.props.size() == 3);
     }
   }
   {

From 3bc9c571a03aefaff159bbdfcc0dcbdaaf91e3f6 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 25 Jan 2023 07:02:03 +0100
Subject: [PATCH 45/90] Make the use RequestRouter more restricted

---
 src/query/v2/interpreter.hpp                |  5 ++++-
 src/query/v2/request_router.hpp             |  2 +-
 tests/simulation/simulation_interpreter.hpp | 10 +---------
 3 files changed, 6 insertions(+), 11 deletions(-)

diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 9c2bef3d2..4bffba2fa 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -296,7 +296,10 @@ class Interpreter final {
    */
   void Abort();
 
-  RequestRouterInterface *GetRequestRouter() { return request_router_.get(); }
+  const RequestRouterInterface *GetRequestRouter() const { return request_router_.get(); }
+  void InstallSimulatorTicker(std::function<bool()> &&tick_simulator) {
+    request_router_->InstallSimulatorTicker(tick_simulator);
+  }
 
  private:
   struct QueryExecution {
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 252278bb8..7ccb0a22b 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -12,7 +12,6 @@
 #pragma once
 
 #include <algorithm>
-#include <boost/uuid/uuid.hpp>
 #include <chrono>
 #include <deque>
 #include <iostream>
@@ -28,6 +27,7 @@
 #include <variant>
 #include <vector>
 
+#include "boost/uuid/uuid.hpp"
 #include "coordinator/coordinator.hpp"
 #include "coordinator/coordinator_client.hpp"
 #include "coordinator/coordinator_rsm.hpp"
diff --git a/tests/simulation/simulation_interpreter.hpp b/tests/simulation/simulation_interpreter.hpp
index 497ee7103..e83980787 100644
--- a/tests/simulation/simulation_interpreter.hpp
+++ b/tests/simulation/simulation_interpreter.hpp
@@ -43,9 +43,7 @@ class SimulatedInterpreter {
   ~SimulatedInterpreter() = default;
 
   void InstallSimulatorTicker(Simulator &simulator) {
-    std::function<bool()> tick_simulator = simulator.GetSimulatorTickClosure();
-    auto *request_router = interpreter_->GetRequestRouter();
-    request_router->InstallSimulatorTicker(tick_simulator);
+    interpreter_->InstallSimulatorTicker(simulator.GetSimulatorTickClosure());
   }
 
   std::vector<ResultStream> RunQueries(const std::vector<std::string> &queries) {
@@ -65,14 +63,8 @@ class SimulatedInterpreter {
     std::map<std::string, memgraph::storage::v3::PropertyValue> params;
     const std::string *username = nullptr;
 
-    interpreter_->BeginTransaction();
-
-    auto *rr = interpreter_->GetRequestRouter();
-    rr->StartTransaction();
-
     interpreter_->Prepare(query, params, username);
     interpreter_->PullAll(&stream);
-    interpreter_->CommitTransaction();
 
     return stream;
   }

From e24a6a86e44d328f813ec22e1429bfc09fb7044a Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 25 Jan 2023 12:42:44 +0100
Subject: [PATCH 46/90] Apply changes from code-review

---
 src/query/v2/interpreter.hpp    |  2 --
 src/query/v2/request_router.hpp | 17 +++++------------
 2 files changed, 5 insertions(+), 14 deletions(-)

diff --git a/src/query/v2/interpreter.hpp b/src/query/v2/interpreter.hpp
index 4bffba2fa..4efc85c22 100644
--- a/src/query/v2/interpreter.hpp
+++ b/src/query/v2/interpreter.hpp
@@ -16,8 +16,6 @@
 
 #include "coordinator/coordinator.hpp"
 #include "coordinator/coordinator_client.hpp"
-#include "io/local_transport/local_transport.hpp"
-#include "io/simulator/simulator_transport.hpp"
 #include "io/transport.hpp"
 #include "query/v2/auth_checker.hpp"
 #include "query/v2/bindings/cypher_main_visitor.hpp"
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index 7ccb0a22b..2604fb13e 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -27,7 +27,8 @@
 #include <variant>
 #include <vector>
 
-#include "boost/uuid/uuid.hpp"
+#include <boost/uuid/uuid.hpp>
+
 #include "coordinator/coordinator.hpp"
 #include "coordinator/coordinator_client.hpp"
 #include "coordinator/coordinator_rsm.hpp"
@@ -744,15 +745,6 @@ class RequestRouter : public RequestRouterInterface {
 };
 
 class RequestRouterFactory {
- protected:
-  using LocalTransport = io::Io<io::local_transport::LocalTransport>;
-  using SimulatorTransport = io::Io<io::simulator::SimulatorTransport>;
-
-  using LocalTransportHandlePtr = std::shared_ptr<io::local_transport::LocalTransportHandle>;
-  using SimulatorTransportHandlePtr = std::shared_ptr<io::simulator::SimulatorHandle>;
-
-  using TransportHandleVariant = std::variant<LocalTransportHandlePtr, SimulatorTransportHandlePtr>;
-
  public:
   RequestRouterFactory() = default;
   RequestRouterFactory(const RequestRouterFactory &) = delete;
@@ -767,10 +759,11 @@ class RequestRouterFactory {
 };
 
 class LocalRequestRouterFactory : public RequestRouterFactory {
-  io::Io<memgraph::io::local_transport::LocalTransport> &io_;
+  using LocalTransportIo = io::Io<io::local_transport::LocalTransport>;
+  LocalTransportIo &io_;
 
  public:
-  explicit LocalRequestRouterFactory(io::Io<memgraph::io::local_transport::LocalTransport> &io) : io_(io) {}
+  explicit LocalRequestRouterFactory(LocalTransportIo &io) : io_(io) {}
 
   std::unique_ptr<RequestRouterInterface> CreateRequestRouter(
       const coordinator::Address &coordinator_address) const override {

From 23297c2afb0c9a9787f03b90493b96d789dffc35 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 26 Jan 2023 19:47:56 +0100
Subject: [PATCH 47/90] Remove unnecessary function

---
 src/query/v2/plan/operator.cpp | 4 ----
 1 file changed, 4 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index ee166e3cc..50e5d4fca 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -655,10 +655,6 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     return false;
   }
 
-  void PullMultiple(MultiFrame & /*input_multi_frame*/, ExecutionContext & /*context*/) override {
-    throw utils::NotYetImplemented("Multiframe version of ScanByPrimaryKey is yet to be implemented.");
-  };
-
   void Reset() override { input_cursor_->Reset(); }
 
   void Shutdown() override { input_cursor_->Shutdown(); }

From 33454c7d8ee81222f0aaf2065a4af5e67998b8be Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 08:43:10 +0100
Subject: [PATCH 48/90] Add implementation

---
 src/query/v2/plan/operator.cpp | 100 +++++++++++++++++++++++++++++----
 1 file changed, 90 insertions(+), 10 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 50e5d4fca..190a993a9 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1972,14 +1972,7 @@ class UnwindCursor : public Cursor {
         if (!input_cursor_->Pull(frame, context)) return false;
 
         // successful pull from input, initialize value and iterator
-        ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
-                                      storage::v3::View::OLD);
-        TypedValue input_value = self_.input_expression_->Accept(evaluator);
-        if (input_value.type() != TypedValue::Type::List)
-          throw QueryRuntimeException("Argument of UNWIND must be a list, but '{}' was provided.", input_value.type());
-        // Copy the evaluted input_value_list to our vector.
-        input_value_ = input_value.ValueList();
-        input_value_it_ = input_value_.begin();
+        SetInputValue(frame, context);
       }
 
       // if we reached the end of our list of values goto back to top
@@ -1990,6 +1983,70 @@ class UnwindCursor : public Cursor {
     }
   }
 
+  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP("UnwindMF");
+
+    if (!own_multi_frame_.has_value()) {
+      own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
+                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+      own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+      own_frames_it_ = own_frames_consumer_->begin();
+    }
+
+    auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+    auto populated_any = false;
+
+    while (true) {
+      switch (state_) {
+        case State::PullInput: {
+          if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
+            state_ = State::Exhausted;
+            return populated_any;
+          }
+          own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+          own_frames_it_ = own_frames_consumer_->begin();
+          state_ = State::InitializeInputValue;
+          break;
+        }
+        case State::InitializeInputValue: {
+          if (own_frames_it_ == own_frames_consumer_->end()) {
+            state_ = State::PullInput;
+            continue;
+          }
+          SetInputValue(*own_frames_it_, context);
+          state_ = State::PopulateOutput;
+          break;
+        }
+        case State::PopulateOutput: {
+          if (!output_multi_frame.HasInvalidFrame()) {
+            return populated_any;
+          }
+          if (input_value_it_ == input_value_.end()) {
+            own_frames_it_->MakeInvalid();
+            ++own_frames_it_;
+            state_ = State::InitializeInputValue;
+            continue;
+          }
+
+          for (auto output_frame_it = output_frames_populator.begin();
+               output_frame_it != output_frames_populator.end() && input_value_it_ != input_value_.end();
+               ++output_frame_it) {
+            auto &output_frame = *output_frame_it;
+            output_frame = *own_frames_it_;
+            output_frame[self_.output_symbol_] = std::move(*input_value_it_);
+            input_value_it_++;
+            populated_any = true;
+          }
+          break;
+        }
+        case State::Exhausted: {
+          return populated_any;
+        }
+      }
+    }
+    return populated_any;
+  }
+
   void Shutdown() override { input_cursor_->Shutdown(); }
 
   void Reset() override {
@@ -1998,13 +2055,36 @@ class UnwindCursor : public Cursor {
     input_value_it_ = input_value_.end();
   }
 
+  void SetInputValue(Frame &frame, ExecutionContext &context) {
+    ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
+                                  storage::v3::View::OLD);
+    TypedValue input_value = self_.input_expression_->Accept(evaluator);
+    if (input_value.type() != TypedValue::Type::List) {
+      throw QueryRuntimeException("Argument of UNWIND must be a list, but '{}' was provided.", input_value.type());
+    }
+    // It would be nice if we could move it, however it can be tricky to make it work because of allocators and
+    // different memory resources, be careful.
+    input_value_ = std::move(input_value.ValueList());
+    input_value_it_ = input_value_.begin();
+  }
+
  private:
+  using InputVector = utils::pmr::vector<TypedValue>;
+  using InputIterator = InputVector::iterator;
+
   const Unwind &self_;
   const UniqueCursorPtr input_cursor_;
   // typed values we are unwinding and yielding
-  utils::pmr::vector<TypedValue> input_value_;
+  InputVector input_value_;
   // current position in input_value_
-  decltype(input_value_)::iterator input_value_it_ = input_value_.end();
+  InputIterator input_value_it_ = input_value_.end();
+
+  enum class State { PullInput, InitializeInputValue, PopulateOutput, Exhausted };
+
+  State state_{State::PullInput};
+  std::optional<MultiFrame> own_multi_frame_;
+  std::optional<ValidFramesConsumer> own_frames_consumer_;
+  ValidFramesConsumer::Iterator own_frames_it_;
 };
 
 UniqueCursorPtr Unwind::MakeCursor(utils::MemoryResource *mem) const {

From 883922dba5f2308fb3e64fc859016989d5c22b41 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 13:11:55 +0100
Subject: [PATCH 49/90] Eliminate warning about deprecated macro

---
 tests/unit/query_v2_plan.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/unit/query_v2_plan.cpp b/tests/unit/query_v2_plan.cpp
index 8fc5e11c7..c782f9127 100644
--- a/tests/unit/query_v2_plan.cpp
+++ b/tests/unit/query_v2_plan.cpp
@@ -86,7 +86,7 @@ class TestPlanner : public ::testing::Test {};
 
 using PlannerTypes = ::testing::Types<Planner>;
 
-TYPED_TEST_CASE(TestPlanner, PlannerTypes);
+TYPED_TEST_SUITE(TestPlanner, PlannerTypes);
 
 TYPED_TEST(TestPlanner, MatchFilterPropIsNotNull) {
   const char *prim_label_name = "prim_label_one";

From c12a5a901909adea92ee8fd007345f64f50b7260 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 16:50:16 +0100
Subject: [PATCH 50/90] Make multi-create queries work

---
 src/query/v2/plan/operator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 95a7f6c40..a11b75d4f 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -202,7 +202,7 @@ class DistributedCreateNodeCursor : public Cursor {
       request_router->CreateVertices(NodeCreationInfoToRequests(context, multi_frame));
     }
     PlaceNodesOnTheMultiFrame(multi_frame, context);
-    return false;
+    return true;
   }
 
   void Shutdown() override { input_cursor_->Shutdown(); }

From 2ecf580ae73890012057e1c152422096e21df047 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 16:50:58 +0100
Subject: [PATCH 51/90] Eliminate warnings

---
 src/query/v2/plan/operator.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index a11b75d4f..0422db14c 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1566,7 +1566,7 @@ class AggregateCursor : public Cursor {
     for (; count_it < agg_value->counts_.end(); count_it++, value_it++, agg_elem_it++) {
       // COUNT(*) is the only case where input expression is optional
       // handle it here
-      auto input_expr_ptr = agg_elem_it->value;
+      auto *input_expr_ptr = agg_elem_it->value;
       if (!input_expr_ptr) {
         *count_it += 1;
         *value_it = *count_it;
@@ -1657,7 +1657,7 @@ class AggregateCursor : public Cursor {
 
   /** Checks if the given TypedValue is legal in MIN and MAX. If not
    * an appropriate exception is thrown. */
-  void EnsureOkForMinMax(const TypedValue &value) const {
+  static void EnsureOkForMinMax(const TypedValue &value) {
     switch (value.type()) {
       case TypedValue::Type::Bool:
       case TypedValue::Type::Int:
@@ -1673,7 +1673,7 @@ class AggregateCursor : public Cursor {
 
   /** Checks if the given TypedValue is legal in AVG and SUM. If not
    * an appropriate exception is thrown. */
-  void EnsureOkForAvgSum(const TypedValue &value) const {
+  static void EnsureOkForAvgSum(const TypedValue &value) {
     switch (value.type()) {
       case TypedValue::Type::Int:
       case TypedValue::Type::Double:

From d36c0cc4243a8a2eb312f02534ac9f60501349ee Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 16:52:46 +0100
Subject: [PATCH 52/90] Refactor `AggregateCursor::ProcessAll`

---
 src/query/v2/plan/operator.cpp | 34 +++++++++++++++-------------------
 1 file changed, 15 insertions(+), 19 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 0422db14c..358a638be 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1324,14 +1324,13 @@ class AggregateCursor : public Cursor {
     return true;
   }
 
-  void PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
+  bool PullMultiple(MultiFrame &multi_frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP("AggregateMF");
 
     if (!pulled_all_input_) {
-      while (!pulled_all_input_) {
-        ProcessAll(multi_frame, &context);
-      }
-      multi_frame.MakeAllFramesInvalid();
+      ProcessAll(multi_frame, &context);
+      pulled_all_input_ = true;
+      MG_ASSERT(!multi_frame.HasValidFrame(), "ProcessAll didn't consumed all input frames!");
       aggregation_it_ = aggregation_.begin();
 
       // in case there is no input and no group_bys we need to return true
@@ -1348,12 +1347,12 @@ class AggregateCursor : public Cursor {
         for (const Symbol &remember_sym : self_.remember_) {
           frame[remember_sym] = TypedValue(pull_memory);
         }
-        return;
+        return true;
       }
     }
 
     if (aggregation_it_ == aggregation_.end()) {
-      return;
+      return false;
     }
 
     // place aggregation values on the frame
@@ -1371,6 +1370,7 @@ class AggregateCursor : public Cursor {
     }
 
     aggregation_it_++;
+    return true;
   }
 
   void Shutdown() override { input_cursor_->Shutdown(); }
@@ -1452,19 +1452,15 @@ class AggregateCursor : public Cursor {
   }
 
   void ProcessAll(MultiFrame &multi_frame, ExecutionContext *context) {
-    input_cursor_->PullMultiple(multi_frame, *context);
-    auto valid_frames_modifier =
-        multi_frame.GetValidFramesConsumer();  // consumer is needed i.o. reader because of the evaluator
-    if (valid_frames_modifier.begin() == valid_frames_modifier.end()) {
-      // There are no valid frames, we stop
-      pulled_all_input_ = true;
-      return;
-    }
+    while (input_cursor_->PullMultiple(multi_frame, *context)) {
+      auto valid_frames_modifier =
+          multi_frame.GetValidFramesConsumer();  // consumer is needed i.o. reader because of the evaluator
 
-    for (auto &frame : valid_frames_modifier) {
-      ExpressionEvaluator evaluator(&frame, context->symbol_table, context->evaluation_context, context->request_router,
-                                    storage::v3::View::NEW);
-      ProcessOne(frame, &evaluator);
+      for (auto &frame : valid_frames_modifier) {
+        ExpressionEvaluator evaluator(&frame, context->symbol_table, context->evaluation_context,
+                                      context->request_router, storage::v3::View::NEW);
+        ProcessOne(frame, &evaluator);
+      }
     }
 
     // calculate AVG aggregations (so far they have only been summed)

From 44fc2d01c769a2fbde972b26f3cdbaae85912327 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 17:04:40 +0100
Subject: [PATCH 53/90] Unify logic between multi and single frame pull

---
 src/query/v2/plan/operator.cpp | 49 +++++++++-------------------------
 1 file changed, 13 insertions(+), 36 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 358a638be..f70b4b6b1 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1437,18 +1437,7 @@ class AggregateCursor : public Cursor {
       ProcessOne(*frame, &evaluator);
     }
 
-    // calculate AVG aggregations (so far they have only been summed)
-    for (size_t pos = 0; pos < self_.aggregations_.size(); ++pos) {
-      if (self_.aggregations_[pos].op != Aggregation::Op::AVG) continue;
-      for (auto &kv : aggregation_) {
-        AggregationValue &agg_value = kv.second;
-        auto count = agg_value.counts_[pos];
-        auto *pull_memory = context->evaluation_context.memory;
-        if (count > 0) {
-          agg_value.values_[pos] = agg_value.values_[pos] / TypedValue(static_cast<double>(count), pull_memory);
-        }
-      }
-    }
+    CalculateAverages(*context);
   }
 
   void ProcessAll(MultiFrame &multi_frame, ExecutionContext *context) {
@@ -1463,18 +1452,7 @@ class AggregateCursor : public Cursor {
       }
     }
 
-    // calculate AVG aggregations (so far they have only been summed)
-    for (size_t pos = 0; pos < self_.aggregations_.size(); ++pos) {
-      if (self_.aggregations_[pos].op != Aggregation::Op::AVG) continue;
-      for (auto &kv : aggregation_) {
-        AggregationValue &agg_value = kv.second;
-        auto count = agg_value.counts_[pos];
-        auto *pull_memory = context->evaluation_context.memory;
-        if (count > 0) {
-          agg_value.values_[pos] = agg_value.values_[pos] / TypedValue(static_cast<double>(count), pull_memory);
-        }
-      }
-    }
+    CalculateAverages(*context);
   }
 
   /**
@@ -1492,19 +1470,18 @@ class AggregateCursor : public Cursor {
     Update(evaluator, &agg_value);
   }
 
-  /**
-   * Performs a single accumulation.
-   */
-  void ProcessOne(FrameWithValidity &frame, ExpressionEvaluator *evaluator) {
-    auto *mem = aggregation_.get_allocator().GetMemoryResource();
-    utils::pmr::vector<TypedValue> group_by(mem);
-    group_by.reserve(self_.group_by_.size());
-    for (Expression *expression : self_.group_by_) {
-      group_by.emplace_back(expression->Accept(*evaluator));
+  void CalculateAverages(ExecutionContext &context) {
+    for (size_t pos = 0; pos < self_.aggregations_.size(); ++pos) {
+      if (self_.aggregations_[pos].op != Aggregation::Op::AVG) continue;
+      for (auto &kv : aggregation_) {
+        AggregationValue &agg_value = kv.second;
+        auto count = agg_value.counts_[pos];
+        auto *pull_memory = context.evaluation_context.memory;
+        if (count > 0) {
+          agg_value.values_[pos] = agg_value.values_[pos] / TypedValue(static_cast<double>(count), pull_memory);
+        }
+      }
     }
-    auto &agg_value = aggregation_.try_emplace(std::move(group_by), mem).first->second;
-    EnsureInitialized(frame, &agg_value);
-    Update(evaluator, &agg_value);
   }
 
   /** Ensures the new AggregationValue has been initialized. This means

From 9214c715e255b6beea7c4af0d3aaab3d639af085 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 17:05:03 +0100
Subject: [PATCH 54/90] Address review comment

---
 src/query/v2/plan/operator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index f70b4b6b1..c62c7f1bb 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1320,7 +1320,7 @@ class AggregateCursor : public Cursor {
     auto remember_values_it = aggregation_it_->second.remember_.begin();
     for (const Symbol &remember_sym : self_.remember_) frame[remember_sym] = *remember_values_it++;
 
-    aggregation_it_++;
+    ++aggregation_it_;
     return true;
   }
 

From fd047e7303d988f7f5ad9071acdf1f55ba99cd04 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Fri, 27 Jan 2023 17:10:21 +0100
Subject: [PATCH 55/90] Unify even more logic

---
 src/query/v2/plan/operator.cpp | 22 ----------------------
 1 file changed, 22 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index c62c7f1bb..168266ab4 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1500,28 +1500,6 @@ class AggregateCursor : public Cursor {
     for (const Symbol &remember_sym : self_.remember_) agg_value->remember_.push_back(frame[remember_sym]);
   }
 
-  /** Ensures the new AggregationValue has been initialized. This means
-   * that the value vectors are filled with an appropriate number of Nulls,
-   * counts are set to 0 and remember values are remembered.
-   */
-  void EnsureInitialized(FrameWithValidity &frame, AggregateCursor::AggregationValue *agg_value) const {
-    if (!agg_value->values_.empty()) {
-      frame.MakeInvalid();
-      return;
-    }
-
-    for (const auto &agg_elem : self_.aggregations_) {
-      auto *mem = agg_value->values_.get_allocator().GetMemoryResource();
-      agg_value->values_.emplace_back(DefaultAggregationOpValue(agg_elem, mem));
-    }
-    agg_value->counts_.resize(self_.aggregations_.size(), 0);
-
-    for (const Symbol &remember_sym : self_.remember_) {
-      agg_value->remember_.push_back(frame[remember_sym]);
-    }
-    frame.MakeInvalid();
-  }
-
   /** Updates the given AggregationValue with new data. Assumes that
    * the AggregationValue has been initialized */
   void Update(ExpressionEvaluator *evaluator, AggregateCursor::AggregationValue *agg_value) {

From 436e41f71ff5f5bb5ce6ca635d1251500641b466 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Mon, 30 Jan 2023 13:06:05 +0100
Subject: [PATCH 56/90] Init POC of ScanByPrimaryKey multiframe

---
 src/query/v2/interpreter.cpp   |   3 +-
 src/query/v2/plan/operator.cpp | 157 ++++++++++++++++++++++++++++++++-
 2 files changed, 157 insertions(+), 3 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index aa220d764..fe7a17bf7 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -812,7 +812,8 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
                                                                 const std::vector<Symbol> &output_symbols,
                                                                 std::map<std::string, TypedValue> *summary) {
-  auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
+  // auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
+  auto should_pull_multiple = true;
   if (should_pull_multiple) {
     return PullMultiple(stream, n, output_symbols, summary);
   }
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 50e5d4fca..c9bd2e9d0 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -603,8 +603,6 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
         filter_expressions_(filter_expressions),
         primary_key_(primary_key) {}
 
-  enum class State : int8_t { INITIALIZING, COMPLETED };
-
   using VertexAccessor = accessors::VertexAccessor;
 
   std::optional<VertexAccessor> MakeRequestSingleFrame(Frame &frame, RequestRouterInterface &request_router,
@@ -637,6 +635,83 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     return VertexAccessor(vertex, properties, &request_router);
   }
 
+  // TODO (gvolfing) optinal vs empty vector for signaling failure?
+  bool MakeRequestSingleFrameTwo(Frame &frame, RequestRouterInterface &request_router, ExecutionContext &context) {
+    // Evaluate the expressions that hold the PrimaryKey.
+    ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
+                                  storage::v3::View::NEW);
+
+    std::vector<msgs::Value> pk;
+    for (auto *primary_property : primary_key_) {
+      pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+    }
+
+    msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
+
+    msgs::GetPropertiesRequest req = {.vertex_ids = {std::make_pair(label, pk)}};
+    auto get_prop_result = std::invoke([&context, &request_router, &req]() mutable {
+      SCOPED_REQUEST_WAIT_PROFILE;
+      return request_router.GetProperties(req);
+    });
+    MG_ASSERT(get_prop_result.size() <= 1);
+
+    // {
+    //   SCOPED_REQUEST_WAIT_PROFILE;
+    //   std::optional<std::string> request_label = std::nullopt;
+    //   if (label_.has_value()) {
+    //     request_label = context.request_router->LabelToName(*label_);
+    //   }
+    //   current_batch_ = context.request_router->ScanVertices(request_label);
+    // }
+    // current_vertex_it_ = current_batch_.begin();
+    // return !current_batch_.empty(
+
+    if (get_prop_result.empty()) {
+      // return std::nullopt;
+      return false;
+    }
+
+    auto properties = get_prop_result[0].props;
+    // TODO (gvolfing) figure out labels when relevant.
+    msgs::Vertex vertex = {.id = get_prop_result[0].vertex, .labels = {}};
+
+    current_batch_ = {VertexAccessor(vertex, properties, &request_router)};
+    current_vertex_it_ = current_batch_.begin();
+    return current_batch_.empty();
+  }
+
+  //   std::vector<VertexAccessor> MakeRequest(Frame &frame, RequestRouterInterface &request_router,
+  //                                                      ExecutionContext &context) {
+  //   std::vector<VertexAccessor> ret;
+  //   // Evaluate the expressions that hold the PrimaryKey.
+  //   ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
+  //                                 storage::v3::View::NEW);
+
+  //   std::vector<msgs::Value> pk;
+  //   for (auto *primary_property : primary_key_) {
+  //     pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+  //   }
+
+  //   msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
+
+  //   msgs::GetPropertiesRequest req = {.vertex_ids = {std::make_pair(label, pk)}};
+  //   auto get_prop_result = std::invoke([&context, &request_router, &req]() mutable {
+  //     SCOPED_REQUEST_WAIT_PROFILE;
+  //     return request_router.GetProperties(req);
+  //   });
+  //   MG_ASSERT(get_prop_result.size() <= 1);
+
+  //   if (get_prop_result.empty()) {
+  //     return ret;
+  //   }
+  //   auto properties = get_prop_result[0].props;
+  //   // TODO (gvolfing) figure out labels when relevant.
+  //   msgs::Vertex vertex = {.id = get_prop_result[0].vertex, .labels = {}};
+  //   auto va = VertexAccessor(vertex, properties, &request_router);
+  //   ret.push_back(va);
+  //   return ret;
+  // }
+
   bool Pull(Frame &frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
@@ -655,17 +730,95 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     return false;
   }
 
+  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP(op_name_);
+
+    if (!own_multi_frame_.has_value()) {
+      own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
+                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+      own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+      own_frames_it_ = own_frames_consumer_->begin();
+    }
+
+    auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+    auto populated_any = false;
+
+    while (true) {
+      switch (state_) {
+        case State::PullInput: {
+          if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
+            state_ = State::Exhausted;
+            return populated_any;
+          }
+          own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+          own_frames_it_ = own_frames_consumer_->begin();
+          state_ = State::FetchVertices;
+          break;
+        }
+        case State::FetchVertices: {
+          if (own_frames_it_ == own_frames_consumer_->end()) {
+            state_ = State::PullInput;
+            continue;
+          }
+          if (!filter_expressions_->empty() || current_batch_.empty()) {
+            // MakeRequest(context);
+            MakeRequestSingleFrameTwo(*own_frames_it_, *context.request_router, context);
+          } else {
+            // We can reuse the vertices as they don't depend on any value from the frames
+            current_vertex_it_ = current_batch_.begin();
+          }
+          state_ = State::PopulateOutput;
+          break;
+        }
+        case State::PopulateOutput: {
+          if (!output_multi_frame.HasInvalidFrame()) {
+            return populated_any;
+          }
+          if (current_vertex_it_ == current_batch_.end()) {
+            own_frames_it_->MakeInvalid();
+            ++own_frames_it_;
+            state_ = State::FetchVertices;
+            continue;
+          }
+
+          for (auto output_frame_it = output_frames_populator.begin();
+               output_frame_it != output_frames_populator.end() && current_vertex_it_ != current_batch_.end();
+               ++output_frame_it) {
+            auto &output_frame = *output_frame_it;
+            output_frame = *own_frames_it_;
+            output_frame[output_symbol_] = TypedValue(*current_vertex_it_);
+            current_vertex_it_++;
+            populated_any = true;
+          }
+          break;
+        }
+        case State::Exhausted: {
+          return populated_any;
+        }
+      }
+    }
+    return populated_any;
+  };
+
   void Reset() override { input_cursor_->Reset(); }
 
   void Shutdown() override { input_cursor_->Shutdown(); }
 
  private:
+  enum class State { PullInput, FetchVertices, PopulateOutput, Exhausted };
+
+  State state_{State::PullInput};
   const Symbol output_symbol_;
   const UniqueCursorPtr input_cursor_;
   const char *op_name_;
+  std::vector<VertexAccessor> current_batch_;
+  std::vector<VertexAccessor>::iterator current_vertex_it_{current_batch_.begin()};
   storage::v3::LabelId label_;
   std::optional<std::vector<Expression *>> filter_expressions_;
   std::vector<Expression *> primary_key_;
+  std::optional<MultiFrame> own_multi_frame_;
+  std::optional<ValidFramesConsumer> own_frames_consumer_;
+  ValidFramesConsumer::Iterator own_frames_it_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From da28a29c7ff95820346eb4228dc5a0c763efcfa6 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Tue, 31 Jan 2023 17:09:41 +0100
Subject: [PATCH 57/90] Pass properties when creating vertices

---
 src/query/v2/plan/operator.cpp                | 56 +++++++++++--------
 src/query/v2/request_router.hpp               |  4 +-
 tests/unit/mock_helpers.hpp                   |  2 +-
 .../unit/query_v2_create_node_multiframe.cpp  |  2 +-
 tests/unit/query_v2_expression_evaluator.cpp  |  2 +-
 5 files changed, 38 insertions(+), 28 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index aa9743878..2c2f07116 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -227,22 +227,27 @@ class DistributedCreateNodeCursor : public Cursor {
     ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, nullptr,
                                   storage::v3::View::NEW);
     if (const auto *node_info_properties = std::get_if<PropertiesMapList>(&node_info_.properties)) {
-      for (const auto &[key, value_expression] : *node_info_properties) {
+      for (const auto &[property, value_expression] : *node_info_properties) {
         TypedValue val = value_expression->Accept(evaluator);
-        if (context.request_router->IsPrimaryKey(primary_label, key)) {
-          rqst.primary_key.push_back(TypedValueToValue(val));
-          pk.push_back(TypedValueToValue(val));
+        auto msgs_value = TypedValueToValue(val);
+        if (context.request_router->IsPrimaryProperty(primary_label, property)) {
+          rqst.primary_key.push_back(msgs_value);
+          pk.push_back(std::move(msgs_value));
+        } else {
+          rqst.properties.emplace_back(property, std::move(msgs_value));
         }
       }
     } else {
       auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(node_info_.properties)).ValueMap();
-      for (const auto &[key, value] : property_map) {
-        auto key_str = std::string(key);
-        auto property_id = context.request_router->NameToProperty(key_str);
-        if (context.request_router->IsPrimaryKey(primary_label, property_id)) {
-          rqst.primary_key.push_back(TypedValueToValue(value));
-          pk.push_back(TypedValueToValue(value));
-        }
+      for (const auto &[property, typed_value] : property_map) {
+        auto property_str = std::string(property);
+        auto property_id = context.request_router->NameToProperty(property_str);
+        auto msgs_value = TypedValueToValue(typed_value);
+        if (context.request_router->IsPrimaryProperty(primary_label, property_id)) {
+          rqst.primary_key.push_back(msgs_value);
+          pk.push_back(std::move(msgs_value));
+        } else
+          rqst.properties.emplace_back(property_id, std::move(msgs_value));
       }
     }
 
@@ -280,22 +285,27 @@ class DistributedCreateNodeCursor : public Cursor {
       ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, nullptr,
                                     storage::v3::View::NEW);
       if (const auto *node_info_properties = std::get_if<PropertiesMapList>(&node_info_.properties)) {
-        for (const auto &[key, value_expression] : *node_info_properties) {
+        for (const auto &[property, value_expression] : *node_info_properties) {
           TypedValue val = value_expression->Accept(evaluator);
-          if (context.request_router->IsPrimaryKey(primary_label, key)) {
-            rqst.primary_key.push_back(TypedValueToValue(val));
-            pk.push_back(TypedValueToValue(val));
+          auto msgs_value = TypedValueToValue(val);
+          if (context.request_router->IsPrimaryProperty(primary_label, property)) {
+            rqst.primary_key.push_back(msgs_value);
+            pk.push_back(std::move(msgs_value));
+          } else {
+            rqst.properties.emplace_back(property, std::move(msgs_value));
           }
         }
       } else {
         auto property_map = evaluator.Visit(*std::get<ParameterLookup *>(node_info_.properties)).ValueMap();
-        for (const auto &[key, value] : property_map) {
-          auto key_str = std::string(key);
-          auto property_id = context.request_router->NameToProperty(key_str);
-          if (context.request_router->IsPrimaryKey(primary_label, property_id)) {
-            rqst.primary_key.push_back(TypedValueToValue(value));
-            pk.push_back(TypedValueToValue(value));
-          }
+        for (const auto &[property, typed_value] : property_map) {
+          auto property_str = std::string(property);
+          auto property_id = context.request_router->NameToProperty(property_str);
+          auto msgs_value = TypedValueToValue(typed_value);
+          if (context.request_router->IsPrimaryProperty(primary_label, property_id)) {
+            rqst.primary_key.push_back(msgs_value);
+            pk.push_back(std::move(msgs_value));
+          } else
+            rqst.properties.emplace_back(property_id, std::move(msgs_value));
         }
       }
 
@@ -2820,7 +2830,7 @@ class DistributedCreateExpandCursor : public Cursor {
       const auto set_vertex = [&context](const auto &vertex, auto &vertex_id) {
         vertex_id.first = vertex.PrimaryLabel();
         for (const auto &[key, val] : vertex.Properties()) {
-          if (context.request_router->IsPrimaryKey(vertex_id.first.id, key)) {
+          if (context.request_router->IsPrimaryProperty(vertex_id.first.id, key)) {
             vertex_id.second.push_back(val);
           }
         }
diff --git a/src/query/v2/request_router.hpp b/src/query/v2/request_router.hpp
index bf8c93566..a8326d900 100644
--- a/src/query/v2/request_router.hpp
+++ b/src/query/v2/request_router.hpp
@@ -117,7 +117,7 @@ class RequestRouterInterface {
   virtual std::optional<storage::v3::EdgeTypeId> MaybeNameToEdgeType(const std::string &name) const = 0;
   virtual std::optional<storage::v3::LabelId> MaybeNameToLabel(const std::string &name) const = 0;
   virtual bool IsPrimaryLabel(storage::v3::LabelId label) const = 0;
-  virtual bool IsPrimaryKey(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const = 0;
+  virtual bool IsPrimaryProperty(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const = 0;
 
   virtual std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) = 0;
   virtual void InstallSimulatorTicker(std::function<bool()> tick_simulator) = 0;
@@ -231,7 +231,7 @@ class RequestRouter : public RequestRouterInterface {
     return edge_types_.IdToName(id.AsUint());
   }
 
-  bool IsPrimaryKey(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const override {
+  bool IsPrimaryProperty(storage::v3::LabelId primary_label, storage::v3::PropertyId property) const override {
     const auto schema_it = shards_map_.schemas.find(primary_label);
     MG_ASSERT(schema_it != shards_map_.schemas.end(), "Invalid primary label id: {}", primary_label.AsUint());
 
diff --git a/tests/unit/mock_helpers.hpp b/tests/unit/mock_helpers.hpp
index 15f264cac..5ce73538a 100644
--- a/tests/unit/mock_helpers.hpp
+++ b/tests/unit/mock_helpers.hpp
@@ -41,7 +41,7 @@ class MockedRequestRouter : public RequestRouterInterface {
   MOCK_METHOD(std::optional<storage::v3::EdgeTypeId>, MaybeNameToEdgeType, (const std::string &), (const));
   MOCK_METHOD(std::optional<storage::v3::LabelId>, MaybeNameToLabel, (const std::string &), (const));
   MOCK_METHOD(bool, IsPrimaryLabel, (storage::v3::LabelId), (const));
-  MOCK_METHOD(bool, IsPrimaryKey, (storage::v3::LabelId, storage::v3::PropertyId), (const));
+  MOCK_METHOD(bool, IsPrimaryProperty, (storage::v3::LabelId, storage::v3::PropertyId), (const));
   MOCK_METHOD((std::optional<std::pair<uint64_t, uint64_t>>), AllocateInitialEdgeIds, (io::Address));
   MOCK_METHOD(void, InstallSimulatorTicker, (std::function<bool()>));
   MOCK_METHOD(const std::vector<coordinator::SchemaProperty> &, GetSchemaForLabel, (storage::v3::LabelId), (const));
diff --git a/tests/unit/query_v2_create_node_multiframe.cpp b/tests/unit/query_v2_create_node_multiframe.cpp
index b298d2781..a2d5b161d 100644
--- a/tests/unit/query_v2_create_node_multiframe.cpp
+++ b/tests/unit/query_v2_create_node_multiframe.cpp
@@ -58,7 +58,7 @@ TEST(CreateNodeTest, CreateNodeCursor) {
   MockedRequestRouter router;
   EXPECT_CALL(router, CreateVertices(_)).Times(1).WillOnce(Return(std::vector<msgs::CreateVerticesResponse>{}));
   EXPECT_CALL(router, IsPrimaryLabel(_)).WillRepeatedly(Return(true));
-  EXPECT_CALL(router, IsPrimaryKey(_, _)).WillRepeatedly(Return(true));
+  EXPECT_CALL(router, IsPrimaryProperty(_, _)).WillRepeatedly(Return(true));
   auto context = MakeContext(ast, symbol_table, &router, &id_alloc);
   auto multi_frame = CreateMultiFrame(context.symbol_table.max_position());
   cursor->PullMultiple(multi_frame, context);
diff --git a/tests/unit/query_v2_expression_evaluator.cpp b/tests/unit/query_v2_expression_evaluator.cpp
index 0000e62b2..6b1c23816 100644
--- a/tests/unit/query_v2_expression_evaluator.cpp
+++ b/tests/unit/query_v2_expression_evaluator.cpp
@@ -123,7 +123,7 @@ class MockedRequestRouter : public RequestRouterInterface {
 
   bool IsPrimaryLabel(LabelId label) const override { return true; }
 
-  bool IsPrimaryKey(LabelId primary_label, PropertyId property) const override { return true; }
+  bool IsPrimaryProperty(LabelId primary_label, PropertyId property) const override { return true; }
 
   std::optional<std::pair<uint64_t, uint64_t>> AllocateInitialEdgeIds(io::Address coordinator_address) override {
     return {};

From 60b71cc2c107eb63882cd3c1cc9616cf204b32ea Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Tue, 31 Jan 2023 17:30:31 +0100
Subject: [PATCH 58/90] Rework ScanByPrimaryKey operator - multiframe

---
 src/query/v2/plan/operator.cpp | 158 ++++++++++++++-------------------
 1 file changed, 65 insertions(+), 93 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index ad1674d29..20425de6a 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -635,83 +635,43 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     return VertexAccessor(vertex, properties, &request_router);
   }
 
-  // TODO (gvolfing) optinal vs empty vector for signaling failure?
-  bool MakeRequestSingleFrameTwo(Frame &frame, RequestRouterInterface &request_router, ExecutionContext &context) {
-    // Evaluate the expressions that hold the PrimaryKey.
-    ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
-                                  storage::v3::View::NEW);
+  void MakeRequestMultiFrame(MultiFrame &multi_frame, RequestRouterInterface &request_router,
+                             ExecutionContext &context) {
+    msgs::GetPropertiesRequest req;
+    const msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
 
-    std::vector<msgs::Value> pk;
-    for (auto *primary_property : primary_key_) {
-      pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+    std::unordered_set<msgs::VertexId> used_vertex_ids;
+
+    for (auto &frame : multi_frame.GetValidFramesModifier()) {
+      ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
+                                    storage::v3::View::NEW);
+
+      std::vector<msgs::Value> pk;
+      for (auto *primary_property : primary_key_) {
+        pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+      }
+
+      auto vertex_id = std::make_pair(label, std::move(pk));
+      auto [it, inserted] = used_vertex_ids.emplace(std::move(vertex_id));
+      if (inserted) {
+        req.vertex_ids.emplace_back(*it);
+      }
     }
 
-    msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
-
-    msgs::GetPropertiesRequest req = {.vertex_ids = {std::make_pair(label, pk)}};
     auto get_prop_result = std::invoke([&context, &request_router, &req]() mutable {
       SCOPED_REQUEST_WAIT_PROFILE;
       return request_router.GetProperties(req);
     });
-    MG_ASSERT(get_prop_result.size() <= 1);
 
-    // {
-    //   SCOPED_REQUEST_WAIT_PROFILE;
-    //   std::optional<std::string> request_label = std::nullopt;
-    //   if (label_.has_value()) {
-    //     request_label = context.request_router->LabelToName(*label_);
-    //   }
-    //   current_batch_ = context.request_router->ScanVertices(request_label);
-    // }
-    // current_vertex_it_ = current_batch_.begin();
-    // return !current_batch_.empty(
+    for (auto &result : get_prop_result) {
+      auto properties = result.props;
+      // TODO (gvolfing) figure out labels when relevant.
+      msgs::Vertex vertex = {.id = result.vertex, .labels = {}};
 
-    if (get_prop_result.empty()) {
-      // return std::nullopt;
-      return false;
+      id_to_accessor_mapping_.emplace(result.vertex, VertexAccessor(vertex, properties, &request_router));
     }
-
-    auto properties = get_prop_result[0].props;
-    // TODO (gvolfing) figure out labels when relevant.
-    msgs::Vertex vertex = {.id = get_prop_result[0].vertex, .labels = {}};
-
-    current_batch_ = {VertexAccessor(vertex, properties, &request_router)};
-    current_vertex_it_ = current_batch_.begin();
-    return current_batch_.empty();
   }
 
-  //   std::vector<VertexAccessor> MakeRequest(Frame &frame, RequestRouterInterface &request_router,
-  //                                                      ExecutionContext &context) {
-  //   std::vector<VertexAccessor> ret;
-  //   // Evaluate the expressions that hold the PrimaryKey.
-  //   ExpressionEvaluator evaluator(&frame, context.symbol_table, context.evaluation_context, context.request_router,
-  //                                 storage::v3::View::NEW);
-
-  //   std::vector<msgs::Value> pk;
-  //   for (auto *primary_property : primary_key_) {
-  //     pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
-  //   }
-
-  //   msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
-
-  //   msgs::GetPropertiesRequest req = {.vertex_ids = {std::make_pair(label, pk)}};
-  //   auto get_prop_result = std::invoke([&context, &request_router, &req]() mutable {
-  //     SCOPED_REQUEST_WAIT_PROFILE;
-  //     return request_router.GetProperties(req);
-  //   });
-  //   MG_ASSERT(get_prop_result.size() <= 1);
-
-  //   if (get_prop_result.empty()) {
-  //     return ret;
-  //   }
-  //   auto properties = get_prop_result[0].props;
-  //   // TODO (gvolfing) figure out labels when relevant.
-  //   msgs::Vertex vertex = {.id = get_prop_result[0].vertex, .labels = {}};
-  //   auto va = VertexAccessor(vertex, properties, &request_router);
-  //   ret.push_back(va);
-  //   return ret;
-  // }
-
   bool Pull(Frame &frame, ExecutionContext &context) override {
     SCOPED_PROFILE_OP(op_name_);
 
@@ -730,15 +690,19 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     return false;
   }
 
-  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
-    SCOPED_PROFILE_OP(op_name_);
-
+  void EnsureOwnMultiFrameIsGood(MultiFrame &output_multi_frame) {
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
                                           kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
       own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
       own_frames_it_ = own_frames_consumer_->begin();
     }
+    MG_ASSERT(output_multi_frame.GetFirstFrame().elems().size() == own_multi_frame_->GetFirstFrame().elems().size());
+  }
+
+  bool PullMultiple(MultiFrame &output_multi_frame, ExecutionContext &context) override {
+    SCOPED_PROFILE_OP(op_name_);
+    EnsureOwnMultiFrameIsGood(output_multi_frame);
 
     auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
     auto populated_any = false;
@@ -746,49 +710,58 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     while (true) {
       switch (state_) {
         case State::PullInput: {
+          id_to_accessor_mapping_.clear();
           if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
             state_ = State::Exhausted;
             return populated_any;
           }
           own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
           own_frames_it_ = own_frames_consumer_->begin();
-          state_ = State::FetchVertices;
-          break;
-        }
-        case State::FetchVertices: {
+
           if (own_frames_it_ == own_frames_consumer_->end()) {
-            state_ = State::PullInput;
             continue;
           }
-          if (!filter_expressions_->empty() || current_batch_.empty()) {
-            // MakeRequest(context);
-            MakeRequestSingleFrameTwo(*own_frames_it_, *context.request_router, context);
-          } else {
-            // We can reuse the vertices as they don't depend on any value from the frames
-            current_vertex_it_ = current_batch_.begin();
-          }
+
+          MakeRequestMultiFrame(*own_multi_frame_, *context.request_router, context);
+
           state_ = State::PopulateOutput;
           break;
         }
         case State::PopulateOutput: {
           if (!output_multi_frame.HasInvalidFrame()) {
+            if (own_frames_it_ == own_frames_consumer_->end()) {
+              id_to_accessor_mapping_.clear();
+            }
             return populated_any;
           }
-          if (current_vertex_it_ == current_batch_.end()) {
-            own_frames_it_->MakeInvalid();
-            ++own_frames_it_;
-            state_ = State::FetchVertices;
+
+          if (own_frames_it_ == own_frames_consumer_->end()) {
+            state_ = State::PullInput;
             continue;
           }
 
           for (auto output_frame_it = output_frames_populator.begin();
-               output_frame_it != output_frames_populator.end() && current_vertex_it_ != current_batch_.end();
-               ++output_frame_it) {
+               output_frame_it != output_frames_populator.end() && own_frames_it_ != own_frames_consumer_->end();
+               ++own_frames_it_) {
             auto &output_frame = *output_frame_it;
-            output_frame = *own_frames_it_;
-            output_frame[output_symbol_] = TypedValue(*current_vertex_it_);
-            current_vertex_it_++;
-            populated_any = true;
+
+            ExpressionEvaluator evaluator(&*own_frames_it_, context.symbol_table, context.evaluation_context,
+                                          context.request_router, storage::v3::View::NEW);
+
+            std::vector<msgs::Value> pk;
+            for (auto *primary_property : primary_key_) {
+              pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+            }
+
+            const msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
+            auto vertex_id = std::make_pair(label, std::move(pk));
+
+            if (const auto it = id_to_accessor_mapping_.find(vertex_id); it != id_to_accessor_mapping_.end()) {
+              output_frame = *own_frames_it_;
+              output_frame[output_symbol_] = TypedValue(it->second);
+              populated_any = true;
+              ++output_frame_it;
+            }
           }
           break;
         }
@@ -805,20 +778,19 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
   void Shutdown() override { input_cursor_->Shutdown(); }
 
  private:
-  enum class State { PullInput, FetchVertices, PopulateOutput, Exhausted };
+  enum class State { PullInput, PopulateOutput, Exhausted };
 
   State state_{State::PullInput};
   const Symbol output_symbol_;
   const UniqueCursorPtr input_cursor_;
   const char *op_name_;
-  std::vector<VertexAccessor> current_batch_;
-  std::vector<VertexAccessor>::iterator current_vertex_it_{current_batch_.begin()};
   storage::v3::LabelId label_;
   std::optional<std::vector<Expression *>> filter_expressions_;
   std::vector<Expression *> primary_key_;
   std::optional<MultiFrame> own_multi_frame_;
   std::optional<ValidFramesConsumer> own_frames_consumer_;
   ValidFramesConsumer::Iterator own_frames_it_;
+  std::unordered_map<msgs::VertexId, VertexAccessor> id_to_accessor_mapping_;
 };
 
 ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol, storage::v3::View view)

From 7d63236f871ec60d94c1dff4794ef028783456a6 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Tue, 31 Jan 2023 17:36:52 +0100
Subject: [PATCH 59/90] Set the default pull-mechanism back to single-pull

---
 src/query/v2/interpreter.cpp | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index fe7a17bf7..aa220d764 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -812,8 +812,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
                                                                 const std::vector<Symbol> &output_symbols,
                                                                 std::map<std::string, TypedValue> *summary) {
-  // auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
-  auto should_pull_multiple = true;
+  auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
   if (should_pull_multiple) {
     return PullMultiple(stream, n, output_symbols, summary);
   }

From 4be4a86d0a932a2242c15967844d9ce3eb1c1676 Mon Sep 17 00:00:00 2001
From: gvolfing <107616712+gvolfing@users.noreply.github.com>
Date: Wed, 1 Feb 2023 11:39:48 +0100
Subject: [PATCH 60/90] Apply suggestions from code review
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

Co-authored-by: János Benjamin Antal <antaljanosbenjamin@users.noreply.github.com>
---
 src/query/v2/plan/operator.cpp | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 20425de6a..4a512fcd6 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -664,11 +664,10 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
     });
 
     for (auto &result : get_prop_result) {
-      auto properties = result.props;
       // TODO (gvolfing) figure out labels when relevant.
       msgs::Vertex vertex = {.id = result.vertex, .labels = {}};
 
-      id_to_accessor_mapping_.emplace(result.vertex, VertexAccessor(vertex, properties, &request_router));
+      id_to_accessor_mapping_.emplace(result.vertex, VertexAccessor(std::move(vertex), std::move(result.properties), &request_router));
     }
   }
 

From bf93b53e7dcec4bbad54fe17968ce2d16b499a66 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 1 Feb 2023 12:36:27 +0100
Subject: [PATCH 61/90] Fix compile error due to wrong aggregate field name

---
 src/query/v2/plan/operator.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 4a512fcd6..86b804db2 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -667,7 +667,8 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
       // TODO (gvolfing) figure out labels when relevant.
       msgs::Vertex vertex = {.id = result.vertex, .labels = {}};
 
-      id_to_accessor_mapping_.emplace(result.vertex, VertexAccessor(std::move(vertex), std::move(result.properties), &request_router));
+      id_to_accessor_mapping_.emplace(result.vertex,
+                                      VertexAccessor(std::move(vertex), std::move(result.props), &request_router));
     }
   }
 

From a38401130e44951a5a4263a88223630d2238e4c2 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 13:24:58 +0100
Subject: [PATCH 62/90] Set vertex id in `Expand` properly

---
 src/query/v2/plan/operator.cpp | 19 ++++---------------
 1 file changed, 4 insertions(+), 15 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index a4b893c5c..e885ed029 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -2949,27 +2949,16 @@ class DistributedCreateExpandCursor : public Cursor {
       const auto &v1 = v1_value.ValueVertex();
       const auto &v2 = OtherVertex(frame);
 
-      // Set src and dest vertices
-      // TODO(jbajic) Currently we are only handling scenario where vertices
-      // are matched
-      const auto set_vertex = [&context](const auto &vertex, auto &vertex_id) {
-        vertex_id.first = vertex.PrimaryLabel();
-        for (const auto &[key, val] : vertex.Properties()) {
-          if (context.request_router->IsPrimaryProperty(vertex_id.first.id, key)) {
-            vertex_id.second.push_back(val);
-          }
-        }
-      };
       std::invoke([&]() {
         switch (edge_info.direction) {
           case EdgeAtom::Direction::IN: {
-            set_vertex(v2, request.src_vertex);
-            set_vertex(v1, request.dest_vertex);
+            request.src_vertex = v2.Id();
+            request.dest_vertex = v1.Id();
             break;
           }
           case EdgeAtom::Direction::OUT: {
-            set_vertex(v1, request.src_vertex);
-            set_vertex(v2, request.dest_vertex);
+            request.src_vertex = v1.Id();
+            request.dest_vertex = v2.Id();
             break;
           }
           case EdgeAtom::Direction::BOTH:

From b136cd71d2f078f85d221fe54b3f102b6157f789 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 14:22:47 +0100
Subject: [PATCH 63/90] Fix `DistributedCreatedNodeCursor` in case of `UNWIND`

---
 src/query/v2/plan/operator.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index e885ed029..2e16b3d9d 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -218,6 +218,7 @@ class DistributedCreateNodeCursor : public Cursor {
   }
 
   std::vector<msgs::NewVertex> NodeCreationInfoToRequest(ExecutionContext &context, Frame &frame) {
+    primary_keys_.clear();
     std::vector<msgs::NewVertex> requests;
     msgs::PrimaryKey pk;
     msgs::NewVertex rqst;
@@ -273,6 +274,7 @@ class DistributedCreateNodeCursor : public Cursor {
   }
 
   std::vector<msgs::NewVertex> NodeCreationInfoToRequests(ExecutionContext &context, MultiFrame &multi_frame) {
+    primary_keys_.clear();
     std::vector<msgs::NewVertex> requests;
     auto multi_frame_modifier = multi_frame.GetValidFramesModifier();
     for (auto &frame : multi_frame_modifier) {

From 7be66f0c540ed0d926e0751736e2d6ff90e8d916 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 14:24:04 +0100
Subject: [PATCH 64/90] Add unwind based dataset creator

---
 tests/mgbench/dataset_creator_unwind.py       | 139 ++++++++++++++++++
 .../accesscontrol_large.shard_configuration   |   6 +-
 .../accesscontrol_medium.shard_configuration  |   6 +-
 .../accesscontrol_small.shard_configuration   |   6 +-
 4 files changed, 154 insertions(+), 3 deletions(-)
 create mode 100644 tests/mgbench/dataset_creator_unwind.py

diff --git a/tests/mgbench/dataset_creator_unwind.py b/tests/mgbench/dataset_creator_unwind.py
new file mode 100644
index 000000000..c9f9a12df
--- /dev/null
+++ b/tests/mgbench/dataset_creator_unwind.py
@@ -0,0 +1,139 @@
+# Copyright 2022 Memgraph Ltd.
+#
+# Use of this software is governed by the Business Source License
+# included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
+# License, and you may not use this file except in compliance with the Business Source License.
+#
+# As of the Change Date specified in that file, in accordance with
+# the Business Source License, use of this software will be governed
+# by the Apache License, Version 2.0, included in the file
+# licenses/APL.txt.
+
+import argparse
+import random
+
+import helpers
+
+# Explaination of datasets:
+#   - empty_only_index: contains index; contains no data
+#   - small: contains index; contains data (small dataset)
+#
+# Datamodel is as follow:
+#
+#                               ┌──────────────┐
+#                               │ Permission   │
+#         ┌────────────────┐    │  Schema:uuid │   ┌────────────┐
+#         │:IS_FOR_IDENTITY├────┤  Index:name  ├───┤:IS_FOR_FILE│
+#         └┬───────────────┘    └──────────────┘   └────────────┤
+#          │                                                    │
+#   ┌──────▼──────────────┐                                  ┌──▼────────────────┐
+#   │  Identity           │                                  │ File              │
+#   │   Schema:uuid       │                                  │  Schema:uuid      │
+#   │   Index:email       │                                  │  Index:name       │
+#   └─────────────────────┘                                  │  Index:platformId │
+#                                                            └───────────────────┘
+#
+#   - File: attributes: ["uuid", "name", "platformId"]
+#   - Permission: attributes: ["uuid", "name"]
+#   - Identity: attributes: ["uuid", "email"]
+#
+# Indexes:
+#   - File: [File(uuid), File(platformId), File(name)]
+#   - Permission: [Permission(uuid), Permission(name)]
+#   - Identity: [Identity(uuid), Identity(email)]
+#
+# Edges:
+#   - (:Permission)-[:IS_FOR_FILE]->(:File)
+#   - (:Permission)-[:IS_FOR_IDENTITYR]->(:Identity)
+#
+# AccessControl specific: uuid is the schema
+
+
+def main():
+    parser = argparse.ArgumentParser()
+    parser.add_argument("--number_of_identities", type=int, default=10)
+    parser.add_argument("--number_of_files", type=int, default=10)
+    parser.add_argument("--percentage_of_permissions", type=float, default=1.0)
+    parser.add_argument("--filename", default="dataset.cypher")
+
+    args = parser.parse_args()
+
+    number_of_identities = args.number_of_identities
+    number_of_files = args.number_of_files
+    percentage_of_permissions = args.percentage_of_permissions
+    filename = args.filename
+
+    assert number_of_identities >= 0
+    assert number_of_files >= 0
+    assert percentage_of_permissions > 0.0 and percentage_of_permissions <= 1.0
+    assert filename != ""
+
+    with open(filename, "w") as f:
+        f.write("MATCH (n) DETACH DELETE n;\n")
+
+        # Create the indexes
+        f.write("CREATE INDEX ON :File;\n")
+        f.write("CREATE INDEX ON :Permission;\n")
+        f.write("CREATE INDEX ON :Identity;\n")
+        f.write("CREATE INDEX ON :File(platformId);\n")
+        f.write("CREATE INDEX ON :File(name);\n")
+        f.write("CREATE INDEX ON :Permission(name);\n")
+        f.write("CREATE INDEX ON :Identity(email);\n")
+
+        # Create extra index: in distributed, this will be the schema
+        f.write("CREATE INDEX ON :File(uuid);\n")
+        f.write("CREATE INDEX ON :Permission(uuid);\n")
+        f.write("CREATE INDEX ON :Identity(uuid);\n")
+
+        uuid = 1
+
+        # Create the nodes File
+        f.write("UNWIND [")
+        for index in range(0, number_of_files):
+            if index != 0:
+                f.write(",")
+            f.write(f'\n  {{uuid: {uuid}, platformId: "platform_id", name: "name_file_{uuid}"}}')
+            uuid += 1
+        f.write("\n] AS props CREATE (:File {uuid: props.uuid, platformId: props.platformId, name: props.name});\n")
+
+        identities = []
+        f.write("UNWIND [")
+        # Create the nodes Identity
+        for index in range(0, number_of_identities):
+            if index != 0:
+                f.write(",")
+            f.write(f'\n  {{uuid: {uuid}, name: "mail_{uuid}@something.com"}}')
+            uuid += 1
+        f.write("\n] AS props CREATE (:Identity {uuid: props.uuid, name: props.name});\n")
+
+        f.write("UNWIND [")
+        wrote_anything = False
+        for outer_index in range(0, number_of_files):
+            for inner_index in range(0, number_of_identities):
+
+                file_uuid = outer_index + 1
+                identity_uuid = number_of_files + inner_index + 1
+
+                if random.random() <= percentage_of_permissions:
+
+                    if wrote_anything:
+                        f.write(",")
+
+                    f.write(
+                        f'\n  {{permUuid: {uuid}, permName: "name_permission_{uuid}", fileUuid: {file_uuid}, identityUuid: {identity_uuid}}}'
+                    )
+                    wrote_anything = True
+                    uuid += 1
+        f.write(
+            """
+\n] AS props
+MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid})
+CREATE (permission:Permission {uuid: props.permUuid, name: props.permName})
+CREATE (permission)-[: IS_FOR_FILE]->(file)
+CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);
+"""
+        )
+
+
+if __name__ == "__main__":
+    main()
diff --git a/tests/mgbench/splitfiles/accesscontrol_large.shard_configuration b/tests/mgbench/splitfiles/accesscontrol_large.shard_configuration
index 34dca66be..d2138ec93 100644
--- a/tests/mgbench/splitfiles/accesscontrol_large.shard_configuration
+++ b/tests/mgbench/splitfiles/accesscontrol_large.shard_configuration
@@ -1,8 +1,12 @@
-4
+8
 uuid
 email
 name
 platformId
+permUuid
+permName
+fileUuid
+identityUuid
 2
 IS_FOR_IDENTITY
 IS_FOR_FILE
diff --git a/tests/mgbench/splitfiles/accesscontrol_medium.shard_configuration b/tests/mgbench/splitfiles/accesscontrol_medium.shard_configuration
index a807e783f..f05ee8993 100644
--- a/tests/mgbench/splitfiles/accesscontrol_medium.shard_configuration
+++ b/tests/mgbench/splitfiles/accesscontrol_medium.shard_configuration
@@ -1,8 +1,12 @@
-4
+8
 uuid
 email
 name
 platformId
+permUuid
+permName
+fileUuid
+identityUuid
 2
 IS_FOR_IDENTITY
 IS_FOR_FILE
diff --git a/tests/mgbench/splitfiles/accesscontrol_small.shard_configuration b/tests/mgbench/splitfiles/accesscontrol_small.shard_configuration
index 9c11b6258..2cce1ccef 100644
--- a/tests/mgbench/splitfiles/accesscontrol_small.shard_configuration
+++ b/tests/mgbench/splitfiles/accesscontrol_small.shard_configuration
@@ -1,8 +1,12 @@
-4
+8
 uuid
 email
 name
 platformId
+permUuid
+permName
+fileUuid
+identityUuid
 2
 IS_FOR_IDENTITY
 IS_FOR_FILE

From 24ae6069f0dabe9cfb468c4ff857f2ec34041d0b Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 14:31:56 +0100
Subject: [PATCH 65/90] Split edge creation into batches

---
 tests/mgbench/dataset_creator_unwind.py | 20 +++++++++++++++-----
 1 file changed, 15 insertions(+), 5 deletions(-)

diff --git a/tests/mgbench/dataset_creator_unwind.py b/tests/mgbench/dataset_creator_unwind.py
index c9f9a12df..4fe537275 100644
--- a/tests/mgbench/dataset_creator_unwind.py
+++ b/tests/mgbench/dataset_creator_unwind.py
@@ -107,7 +107,7 @@ def main():
         f.write("\n] AS props CREATE (:Identity {uuid: props.uuid, name: props.name});\n")
 
         f.write("UNWIND [")
-        wrote_anything = False
+        created = 0
         for outer_index in range(0, number_of_files):
             for inner_index in range(0, number_of_identities):
 
@@ -116,17 +116,27 @@ def main():
 
                 if random.random() <= percentage_of_permissions:
 
-                    if wrote_anything:
+                    if created > 0:
                         f.write(",")
 
                     f.write(
                         f'\n  {{permUuid: {uuid}, permName: "name_permission_{uuid}", fileUuid: {file_uuid}, identityUuid: {identity_uuid}}}'
                     )
-                    wrote_anything = True
+                    created += 1
                     uuid += 1
+
+                    if created == 5000:
+                        f.write(
+                            """\n] AS props
+MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid})
+CREATE (permission:Permission {uuid: props.permUuid, name: props.permName})
+CREATE (permission)-[: IS_FOR_FILE]->(file)
+CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);
+UNWIND ["""
+                        )
+                        created = 0
         f.write(
-            """
-\n] AS props
+            """\n] AS props
 MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid})
 CREATE (permission:Permission {uuid: props.permUuid, name: props.permName})
 CREATE (permission)-[: IS_FOR_FILE]->(file)

From 50327254e0e5f88e97b1bad83d2a28295ae30003 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 14:59:50 +0100
Subject: [PATCH 66/90] Make queries into a single line

---
 tests/mgbench/dataset_creator_unwind.py | 24 +++++++-----------------
 1 file changed, 7 insertions(+), 17 deletions(-)

diff --git a/tests/mgbench/dataset_creator_unwind.py b/tests/mgbench/dataset_creator_unwind.py
index 4fe537275..84434ba6f 100644
--- a/tests/mgbench/dataset_creator_unwind.py
+++ b/tests/mgbench/dataset_creator_unwind.py
@@ -92,9 +92,9 @@ def main():
         for index in range(0, number_of_files):
             if index != 0:
                 f.write(",")
-            f.write(f'\n  {{uuid: {uuid}, platformId: "platform_id", name: "name_file_{uuid}"}}')
+            f.write(f' {{uuid: {uuid}, platformId: "platform_id", name: "name_file_{uuid}"}}')
             uuid += 1
-        f.write("\n] AS props CREATE (:File {uuid: props.uuid, platformId: props.platformId, name: props.name});\n")
+        f.write("] AS props CREATE (:File {uuid: props.uuid, platformId: props.platformId, name: props.name});\n")
 
         identities = []
         f.write("UNWIND [")
@@ -102,9 +102,9 @@ def main():
         for index in range(0, number_of_identities):
             if index != 0:
                 f.write(",")
-            f.write(f'\n  {{uuid: {uuid}, name: "mail_{uuid}@something.com"}}')
+            f.write(f' {{uuid: {uuid}, name: "mail_{uuid}@something.com"}}')
             uuid += 1
-        f.write("\n] AS props CREATE (:Identity {uuid: props.uuid, name: props.name});\n")
+        f.write("] AS props CREATE (:Identity {uuid: props.uuid, name: props.name});\n")
 
         f.write("UNWIND [")
         created = 0
@@ -120,28 +120,18 @@ def main():
                         f.write(",")
 
                     f.write(
-                        f'\n  {{permUuid: {uuid}, permName: "name_permission_{uuid}", fileUuid: {file_uuid}, identityUuid: {identity_uuid}}}'
+                        f' {{permUuid: {uuid}, permName: "name_permission_{uuid}", fileUuid: {file_uuid}, identityUuid: {identity_uuid}}}'
                     )
                     created += 1
                     uuid += 1
 
                     if created == 5000:
                         f.write(
-                            """\n] AS props
-MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid})
-CREATE (permission:Permission {uuid: props.permUuid, name: props.permName})
-CREATE (permission)-[: IS_FOR_FILE]->(file)
-CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);
-UNWIND ["""
+                            "] AS props MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid}) CREATE (permission:Permission {uuid: props.permUuid, name: props.permName}) CREATE (permission)-[: IS_FOR_FILE]->(file) CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);\nUNWIND ["
                         )
                         created = 0
         f.write(
-            """\n] AS props
-MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid})
-CREATE (permission:Permission {uuid: props.permUuid, name: props.permName})
-CREATE (permission)-[: IS_FOR_FILE]->(file)
-CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);
-"""
+            "] AS props MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid}) CREATE (permission:Permission {uuid: props.permUuid, name: props.permName}) CREATE (permission)-[: IS_FOR_FILE]->(file) CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);"
         )
 
 

From c9a0c15c16ebb786625566019fc5c2a8c60c47aa Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 16:17:06 +0100
Subject: [PATCH 67/90] Make frames invalid on consumption

---
 src/query/v2/plan/operator.cpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 2e16b3d9d..e24e4d786 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1586,6 +1586,7 @@ class AggregateCursor : public Cursor {
         ExpressionEvaluator evaluator(&frame, context->symbol_table, context->evaluation_context,
                                       context->request_router, storage::v3::View::NEW);
         ProcessOne(frame, &evaluator);
+        frame.MakeInvalid();
       }
     }
 

From 41183b328b49d4c70e363da1480141818e217c54 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 1 Feb 2023 17:19:02 +0100
Subject: [PATCH 68/90] Invalidate consumed frames in `ScanByPrimaryKey`

---
 src/query/v2/plan/operator.cpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index e24e4d786..4ed9e00e4 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -774,6 +774,7 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
               populated_any = true;
               ++output_frame_it;
             }
+            own_frames_it_->MakeInvalid();
           }
           break;
         }

From 8e315875f2f5aa08c60cc27dee29df5195399220 Mon Sep 17 00:00:00 2001
From: gvolfing <107616712+gvolfing@users.noreply.github.com>
Date: Thu, 2 Feb 2023 07:44:47 +0100
Subject: [PATCH 69/90] Update src/query/v2/plan/operator.cpp
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

Co-authored-by: János Benjamin Antal <antaljanosbenjamin@users.noreply.github.com>
---
 src/query/v2/plan/operator.cpp | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 86b804db2..1139c0fbd 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -761,7 +761,8 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
               output_frame[output_symbol_] = TypedValue(it->second);
               populated_any = true;
               ++output_frame_it;
-            }
+            }           
+            own_frames_it_->MakeInvalid();
           }
           break;
         }

From 2219dee6f606a00b0d9a5fe17b3984fda1b04547 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Mon, 6 Feb 2023 12:49:32 +0100
Subject: [PATCH 70/90] Add initial impl for
 EdgeUniquenessFilterCursor::PullMultiple

---
 src/query/v2/interpreter.cpp   |   3 +-
 src/query/v2/plan/operator.cpp | 126 +++++++++++++++++++++++++++++----
 src/query/v2/plan/operator.lcp |   7 ++
 3 files changed, 122 insertions(+), 14 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index aa220d764..40c3fced4 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -812,7 +812,8 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
                                                                 const std::vector<Symbol> &output_symbols,
                                                                 std::map<std::string, TypedValue> *summary) {
-  auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
+  // auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
+  auto should_pull_multiple = false;
   if (should_pull_multiple) {
     return PullMultiple(stream, n, output_symbols, summary);
   }
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index aa9743878..8bd3544cf 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1207,28 +1207,128 @@ bool ContainsSameEdge(const TypedValue &a, const TypedValue &b) {
 
   return a.ValueEdge() == b.ValueEdge();
 }
+
+bool IsExpansionOk(Frame &frame, const Symbol &expand_symbol, const std::vector<Symbol> &previous_symbols) {
+  const auto &expand_value = frame[expand_symbol];
+  for (const auto &previous_symbol : previous_symbols) {
+    const auto &previous_value = frame[previous_symbol];
+    // This shouldn't raise a TypedValueException, because the planner
+    // makes sure these are all of the expected type. In case they are not
+    // an error should be raised long before this code is executed.
+    if (ContainsSameEdge(previous_value, expand_value)) return false;
+  }
+  return true;
+}
+
 }  // namespace
 
 bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::Pull(Frame &frame, ExecutionContext &context) {
   SCOPED_PROFILE_OP("EdgeUniquenessFilter");
-
-  auto expansion_ok = [&]() {
-    const auto &expand_value = frame[self_.expand_symbol_];
-    for (const auto &previous_symbol : self_.previous_symbols_) {
-      const auto &previous_value = frame[previous_symbol];
-      // This shouldn't raise a TypedValueException, because the planner
-      // makes sure these are all of the expected type. In case they are not
-      // an error should be raised long before this code is executed.
-      if (ContainsSameEdge(previous_value, expand_value)) return false;
-    }
-    return true;
-  };
+  // // TODO (gvolfing) Make the simple Pull method use the function instead of the lambda as well.
+  // auto expansion_ok = [&]() {
+  //   const auto &expand_value = frame[self_.expand_symbol_];
+  //   for (const auto &previous_symbol : self_.previous_symbols_) {
+  //     const auto &previous_value = frame[previous_symbol];
+  //     // This shouldn't raise a TypedValueException, because the planner
+  //     // makes sure these are all of the expected type. In case they are not
+  //     // an error should be raised long before this code is executed.
+  //     if (ContainsSameEdge(previous_value, expand_value)) return false;
+  //   }
+  //   return true;
+  // };
 
   while (input_cursor_->Pull(frame, context))
-    if (expansion_ok()) return true;
+    if (IsExpansionOk(frame, self_.expand_symbol_, self_.previous_symbols_)) return true;
   return false;
 }
 
+bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::PullMultiple(MultiFrame &output_multi_frame,
+                                                                    ExecutionContext &context) {
+  SCOPED_PROFILE_OP("EdgeUniquenessFilterMF");
+
+  if (!own_multi_frame_.has_value()) {
+    own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(), kNumberOfFramesInMultiframe,
+                                        output_multi_frame.GetMemoryResource()));
+    own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+    own_frames_it_ = own_frames_consumer_->begin();
+  }
+  MG_ASSERT(output_multi_frame.GetFirstFrame().elems().size() == own_multi_frame_->GetFirstFrame().elems().size());
+
+  auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
+  auto populated_any = false;
+
+  while (true) {
+    switch (state_) {
+      case State::PullInput: {
+        if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
+          state_ = State::Exhausted;
+          return populated_any;
+        }
+        own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
+        own_frames_it_ = own_frames_consumer_->begin();
+
+        if (own_frames_it_ == own_frames_consumer_->end()) {
+          continue;
+        }
+
+        state_ = State::PopulateOutput;
+        break;
+      }
+      case State::PopulateOutput: {
+        if (!output_multi_frame.HasInvalidFrame()) {
+          return populated_any;
+        }
+
+        if (own_frames_it_ == own_frames_consumer_->end()) {
+          state_ = State::PullInput;
+          continue;
+        }
+
+        for (auto output_frame_it = output_frames_populator.begin();
+             output_frame_it != output_frames_populator.end() && own_frames_it_ != own_frames_consumer_->end();
+             ++own_frames_it_) {
+          auto &output_frame = *output_frame_it;
+
+          if (IsExpansionOk(*own_frames_it_, self_.expand_symbol_, self_.previous_symbols_)) {
+            output_frame = *own_frames_it_;
+            populated_any = true;
+          } else {
+            own_frames_it_->MakeInvalid();
+          }
+          ++output_frame_it;
+
+          /////////////////////////////////////////////////
+          /*
+          ExpressionEvaluator evaluator(&*own_frames_it_, context.symbol_table, context.evaluation_context,
+                                        context.request_router, storage::v3::View::NEW);
+
+          std::vector<msgs::Value> pk;
+          for (auto *primary_property : primary_key_) {
+            pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
+          }
+
+          const msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
+          auto vertex_id = std::make_pair(label, std::move(pk));
+
+          if (const auto it = id_to_accessor_mapping_.find(vertex_id); it != id_to_accessor_mapping_.end()) {
+            output_frame = *own_frames_it_;
+            output_frame[output_symbol_] = TypedValue(it->second);
+            populated_any = true;
+            ++output_frame_it;
+          }
+          own_frames_it_->MakeInvalid();
+          */
+        }
+        break;
+      }
+      case State::Exhausted: {
+        return populated_any;
+      }
+    }
+  }
+  return populated_any;
+}
+
 void EdgeUniquenessFilter::EdgeUniquenessFilterCursor::Shutdown() { input_cursor_->Shutdown(); }
 
 void EdgeUniquenessFilter::EdgeUniquenessFilterCursor::Reset() { input_cursor_->Reset(); }
diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index 4f34cc061..ae02b5931 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -1570,12 +1570,19 @@ edge lists).")
      EdgeUniquenessFilterCursor(const EdgeUniquenessFilter &,
                                 utils::MemoryResource *);
      bool Pull(Frame &, ExecutionContext &) override;
+     bool PullMultiple(MultiFrame &, ExecutionContext &) override;
      void Shutdown() override;
      void Reset() override;
 
     private:
      const EdgeUniquenessFilter &self_;
      const UniqueCursorPtr input_cursor_;
+     enum class State { PullInput, PopulateOutput, Exhausted };
+
+     State state_{State::PullInput};
+     std::optional<MultiFrame> own_multi_frame_;
+     std::optional<ValidFramesConsumer> own_frames_consumer_;
+     ValidFramesConsumer::Iterator own_frames_it_;
    };
    cpp<#)
   (:serialize (:slk))

From 7e99f32adb080bea62149d4ec200a5b7ea479236 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Mon, 6 Feb 2023 15:47:18 +0100
Subject: [PATCH 71/90] Remove uncommented, useless code

---
 src/query/v2/interpreter.cpp   |  3 +--
 src/query/v2/plan/operator.cpp | 37 +---------------------------------
 2 files changed, 2 insertions(+), 38 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 40c3fced4..aa220d764 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -812,8 +812,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
                                                                 const std::vector<Symbol> &output_symbols,
                                                                 std::map<std::string, TypedValue> *summary) {
-  // auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
-  auto should_pull_multiple = false;
+  auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
   if (should_pull_multiple) {
     return PullMultiple(stream, n, output_symbols, summary);
   }
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 5397d6735..beca89b13 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -761,7 +761,7 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
               output_frame[output_symbol_] = TypedValue(it->second);
               populated_any = true;
               ++output_frame_it;
-            }           
+            }
             own_frames_it_->MakeInvalid();
           }
           break;
@@ -1350,19 +1350,6 @@ bool IsExpansionOk(Frame &frame, const Symbol &expand_symbol, const std::vector<
 
 bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::Pull(Frame &frame, ExecutionContext &context) {
   SCOPED_PROFILE_OP("EdgeUniquenessFilter");
-  // // TODO (gvolfing) Make the simple Pull method use the function instead of the lambda as well.
-  // auto expansion_ok = [&]() {
-  //   const auto &expand_value = frame[self_.expand_symbol_];
-  //   for (const auto &previous_symbol : self_.previous_symbols_) {
-  //     const auto &previous_value = frame[previous_symbol];
-  //     // This shouldn't raise a TypedValueException, because the planner
-  //     // makes sure these are all of the expected type. In case they are not
-  //     // an error should be raised long before this code is executed.
-  //     if (ContainsSameEdge(previous_value, expand_value)) return false;
-  //   }
-  //   return true;
-  // };
-
   while (input_cursor_->Pull(frame, context))
     if (IsExpansionOk(frame, self_.expand_symbol_, self_.previous_symbols_)) return true;
   return false;
@@ -1422,28 +1409,6 @@ bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::PullMultiple(MultiFrame &
             own_frames_it_->MakeInvalid();
           }
           ++output_frame_it;
-
-          /////////////////////////////////////////////////
-          /*
-          ExpressionEvaluator evaluator(&*own_frames_it_, context.symbol_table, context.evaluation_context,
-                                        context.request_router, storage::v3::View::NEW);
-
-          std::vector<msgs::Value> pk;
-          for (auto *primary_property : primary_key_) {
-            pk.push_back(TypedValueToValue(primary_property->Accept(evaluator)));
-          }
-
-          const msgs::Label label = {.id = msgs::LabelId::FromUint(label_.AsUint())};
-          auto vertex_id = std::make_pair(label, std::move(pk));
-
-          if (const auto it = id_to_accessor_mapping_.find(vertex_id); it != id_to_accessor_mapping_.end()) {
-            output_frame = *own_frames_it_;
-            output_frame[output_symbol_] = TypedValue(it->second);
-            populated_any = true;
-            ++output_frame_it;
-          }
-          own_frames_it_->MakeInvalid();
-          */
         }
         break;
       }

From ac59e7f7e092fd357926f5faf5d845e8af19ca2f Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Mon, 6 Feb 2023 15:54:07 +0100
Subject: [PATCH 72/90] Move loop variables incrementation into the same place

---
 src/query/v2/plan/operator.cpp | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index beca89b13..400ee4f53 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1399,7 +1399,7 @@ bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::PullMultiple(MultiFrame &
 
         for (auto output_frame_it = output_frames_populator.begin();
              output_frame_it != output_frames_populator.end() && own_frames_it_ != own_frames_consumer_->end();
-             ++own_frames_it_) {
+             ++own_frames_it_, ++output_frame_it) {
           auto &output_frame = *output_frame_it;
 
           if (IsExpansionOk(*own_frames_it_, self_.expand_symbol_, self_.previous_symbols_)) {
@@ -1408,7 +1408,6 @@ bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::PullMultiple(MultiFrame &
           } else {
             own_frames_it_->MakeInvalid();
           }
-          ++output_frame_it;
         }
         break;
       }

From 2b01f2280c6b42408fdd0418251babd77d9e03b9 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Mon, 6 Feb 2023 16:14:21 +0100
Subject: [PATCH 73/90] Add `TODO` about failing query

---
 src/query/v2/plan/operator.cpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index c1df73549..d53c2f0c9 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -904,6 +904,7 @@ ScanByPrimaryKey::ScanByPrimaryKey(const std::shared_ptr<LogicalOperator> &input
                                    storage::v3::LabelId label, std::vector<query::v2::Expression *> primary_key,
                                    storage::v3::View view)
     : ScanAll(input, output_symbol, view), label_(label), primary_key_(primary_key) {
+  // TODO(antaljanosbenjamin): MATCH (p:Permission) WHERE p.uuid <999 RETURN p;
   MG_ASSERT(primary_key.front());
 }
 

From 4bad8c0d1e7e03b093bb15627d7cac43d3d75efd Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Mon, 6 Feb 2023 16:14:39 +0100
Subject: [PATCH 74/90] Filter edges on types

---
 src/query/v2/plan/operator.cpp | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index d53c2f0c9..c908abb59 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -3105,6 +3105,9 @@ class DistributedExpandCursor : public Cursor {
       auto &vertex = vertex_value.ValueVertex();
       msgs::ExpandOneRequest request;
       request.direction = DirectionToMsgsDirection(self_.common_.direction);
+      std::transform(self_.common_.edge_types.begin(), self_.common_.edge_types.end(),
+                     std::back_inserter(request.edge_types),
+                     [](const storage::v3::EdgeTypeId edge_type_id) { return msgs::EdgeType{edge_type_id}; });
       // to not fetch any properties of the edges
       request.edge_properties.emplace();
       request.src_vertices.push_back(vertex.Id());
@@ -3245,6 +3248,9 @@ class DistributedExpandCursor : public Cursor {
 
     msgs::ExpandOneRequest request;
     request.direction = DirectionToMsgsDirection(self_.common_.direction);
+    std::transform(self_.common_.edge_types.begin(), self_.common_.edge_types.end(),
+                   std::back_inserter(request.edge_types),
+                   [](const storage::v3::EdgeTypeId edge_type_id) { return msgs::EdgeType{edge_type_id}; });
     // to not fetch any properties of the edges
     request.edge_properties.emplace();
     for (const auto &frame : own_multi_frame_->GetValidFramesReader()) {

From b26c7d09ef640d80c2c37342a558717676290a7c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Mon, 6 Feb 2023 16:39:42 +0100
Subject: [PATCH 75/90] Ignore not value equality property filters for
 `ScanByPrimaryKey`

---
 src/query/v2/plan/operator.cpp             | 1 -
 src/query/v2/plan/rewrite/index_lookup.hpp | 3 +++
 2 files changed, 3 insertions(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index c908abb59..2ad88cfa9 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -904,7 +904,6 @@ ScanByPrimaryKey::ScanByPrimaryKey(const std::shared_ptr<LogicalOperator> &input
                                    storage::v3::LabelId label, std::vector<query::v2::Expression *> primary_key,
                                    storage::v3::View view)
     : ScanAll(input, output_symbol, view), label_(label), primary_key_(primary_key) {
-  // TODO(antaljanosbenjamin): MATCH (p:Permission) WHERE p.uuid <999 RETURN p;
   MG_ASSERT(primary_key.front());
 }
 
diff --git a/src/query/v2/plan/rewrite/index_lookup.hpp b/src/query/v2/plan/rewrite/index_lookup.hpp
index 17996d952..0b9b9cb97 100644
--- a/src/query/v2/plan/rewrite/index_lookup.hpp
+++ b/src/query/v2/plan/rewrite/index_lookup.hpp
@@ -597,6 +597,9 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
                      [](const auto &schema_elem) { return schema_elem.property_id; });
 
       for (const auto &property_filter : property_filters) {
+        if (property_filter.property_filter->type_ != PropertyFilter::Type::EQUAL) {
+          continue;
+        }
         const auto &property_id = db_->NameToProperty(property_filter.property_filter->property_.name);
         if (std::find(schema_properties.begin(), schema_properties.end(), property_id) != schema_properties.end()) {
           pk_temp.emplace_back(std::make_pair(property_filter.expression, property_filter));

From 37f19867b088ee2a06cdafe36341eb84a5af37f2 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Tue, 7 Feb 2023 08:25:50 +0100
Subject: [PATCH 76/90] Make EdgeUniquenessFilterCursor impl simpler

---
 src/query/v2/plan/operator.cpp | 62 +++++-----------------------------
 src/query/v2/plan/operator.lcp |  6 ----
 2 files changed, 9 insertions(+), 59 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 400ee4f53..ec5413981 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1358,61 +1358,17 @@ bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::Pull(Frame &frame, Execut
 bool EdgeUniquenessFilter::EdgeUniquenessFilterCursor::PullMultiple(MultiFrame &output_multi_frame,
                                                                     ExecutionContext &context) {
   SCOPED_PROFILE_OP("EdgeUniquenessFilterMF");
-
-  if (!own_multi_frame_.has_value()) {
-    own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(), kNumberOfFramesInMultiframe,
-                                        output_multi_frame.GetMemoryResource()));
-    own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
-    own_frames_it_ = own_frames_consumer_->begin();
-  }
-  MG_ASSERT(output_multi_frame.GetFirstFrame().elems().size() == own_multi_frame_->GetFirstFrame().elems().size());
-
-  auto output_frames_populator = output_multi_frame.GetInvalidFramesPopulator();
   auto populated_any = false;
 
-  while (true) {
-    switch (state_) {
-      case State::PullInput: {
-        if (!input_cursor_->PullMultiple(*own_multi_frame_, context)) {
-          state_ = State::Exhausted;
-          return populated_any;
-        }
-        own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
-        own_frames_it_ = own_frames_consumer_->begin();
-
-        if (own_frames_it_ == own_frames_consumer_->end()) {
-          continue;
-        }
-
-        state_ = State::PopulateOutput;
-        break;
-      }
-      case State::PopulateOutput: {
-        if (!output_multi_frame.HasInvalidFrame()) {
-          return populated_any;
-        }
-
-        if (own_frames_it_ == own_frames_consumer_->end()) {
-          state_ = State::PullInput;
-          continue;
-        }
-
-        for (auto output_frame_it = output_frames_populator.begin();
-             output_frame_it != output_frames_populator.end() && own_frames_it_ != own_frames_consumer_->end();
-             ++own_frames_it_, ++output_frame_it) {
-          auto &output_frame = *output_frame_it;
-
-          if (IsExpansionOk(*own_frames_it_, self_.expand_symbol_, self_.previous_symbols_)) {
-            output_frame = *own_frames_it_;
-            populated_any = true;
-          } else {
-            own_frames_it_->MakeInvalid();
-          }
-        }
-        break;
-      }
-      case State::Exhausted: {
-        return populated_any;
+  while (output_multi_frame.HasInvalidFrame()) {
+    if (!input_cursor_->PullMultiple(output_multi_frame, context)) {
+      return populated_any;
+    }
+    for (auto &frame : output_multi_frame.GetValidFramesConsumer()) {
+      if (IsExpansionOk(frame, self_.expand_symbol_, self_.previous_symbols_)) {
+        populated_any = true;
+      } else {
+        frame.MakeInvalid();
       }
     }
   }
diff --git a/src/query/v2/plan/operator.lcp b/src/query/v2/plan/operator.lcp
index ae02b5931..110ba8a33 100644
--- a/src/query/v2/plan/operator.lcp
+++ b/src/query/v2/plan/operator.lcp
@@ -1577,12 +1577,6 @@ edge lists).")
     private:
      const EdgeUniquenessFilter &self_;
      const UniqueCursorPtr input_cursor_;
-     enum class State { PullInput, PopulateOutput, Exhausted };
-
-     State state_{State::PullInput};
-     std::optional<MultiFrame> own_multi_frame_;
-     std::optional<ValidFramesConsumer> own_frames_consumer_;
-     ValidFramesConsumer::Iterator own_frames_it_;
    };
    cpp<#)
   (:serialize (:slk))

From 292a55f4ff59aa6e4a45569bfed650ac6e60bf3c Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 8 Feb 2023 11:28:19 +0100
Subject: [PATCH 77/90] Add new line at the end of dataset file

---
 tests/mgbench/dataset_creator_unwind.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/mgbench/dataset_creator_unwind.py b/tests/mgbench/dataset_creator_unwind.py
index 84434ba6f..564a4d018 100644
--- a/tests/mgbench/dataset_creator_unwind.py
+++ b/tests/mgbench/dataset_creator_unwind.py
@@ -131,7 +131,7 @@ def main():
                         )
                         created = 0
         f.write(
-            "] AS props MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid}) CREATE (permission:Permission {uuid: props.permUuid, name: props.permName}) CREATE (permission)-[: IS_FOR_FILE]->(file) CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);"
+            "] AS props MATCH (file:File {uuid:props.fileUuid}), (identity:Identity {uuid: props.identityUuid}) CREATE (permission:Permission {uuid: props.permUuid, name: props.permName}) CREATE (permission)-[: IS_FOR_FILE]->(file) CREATE (permission)-[: IS_FOR_IDENTITY]->(identity);\n"
         )
 
 

From 25226cca920d8c93f603713909f8b266da4623f9 Mon Sep 17 00:00:00 2001
From: gvolfing <107616712+gvolfing@users.noreply.github.com>
Date: Wed, 8 Feb 2023 11:41:43 +0100
Subject: [PATCH 78/90] Update src/query/v2/plan/operator.cpp

Co-authored-by: Jure Bajic <jure.bajic@memgraph.com>
---
 src/query/v2/plan/operator.cpp | 9 +++------
 1 file changed, 3 insertions(+), 6 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index ec5413981..6fa9c4db9 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1335,15 +1335,12 @@ bool ContainsSameEdge(const TypedValue &a, const TypedValue &b) {
 }
 
 bool IsExpansionOk(Frame &frame, const Symbol &expand_symbol, const std::vector<Symbol> &previous_symbols) {
-  const auto &expand_value = frame[expand_symbol];
-  for (const auto &previous_symbol : previous_symbols) {
-    const auto &previous_value = frame[previous_symbol];
     // This shouldn't raise a TypedValueException, because the planner
     // makes sure these are all of the expected type. In case they are not
     // an error should be raised long before this code is executed.
-    if (ContainsSameEdge(previous_value, expand_value)) return false;
-  }
-  return true;
+   return std::ranges::all_of(previous_symbols, [&expand_value = frame[expand_symbol]](const auto& previous_symbol) {
+     return ContainsSameEdge(previous_value, expand_value);
+   });
 }
 
 }  // namespace

From 657279949aece28624900b012365adca8fa9d9a2 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 8 Feb 2023 12:13:46 +0100
Subject: [PATCH 79/90] Fix compile error

---
 src/query/v2/plan/operator.cpp | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 6fa9c4db9..4337add0d 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1335,12 +1335,14 @@ bool ContainsSameEdge(const TypedValue &a, const TypedValue &b) {
 }
 
 bool IsExpansionOk(Frame &frame, const Symbol &expand_symbol, const std::vector<Symbol> &previous_symbols) {
-    // This shouldn't raise a TypedValueException, because the planner
-    // makes sure these are all of the expected type. In case they are not
-    // an error should be raised long before this code is executed.
-   return std::ranges::all_of(previous_symbols, [&expand_value = frame[expand_symbol]](const auto& previous_symbol) {
-     return ContainsSameEdge(previous_value, expand_value);
-   });
+  // This shouldn't raise a TypedValueException, because the planner
+  // makes sure these are all of the expected type. In case they are not
+  // an error should be raised long before this code is executed.
+  return std::ranges::all_of(previous_symbols,
+                             [&frame, &expand_value = frame[expand_symbol]](const auto &previous_symbol) {
+                               const auto &previous_value = frame[previous_symbol];
+                               return ContainsSameEdge(previous_value, expand_value);
+                             });
 }
 
 }  // namespace

From 096d1ce5f4d4e609614c3795fb22265abb692408 Mon Sep 17 00:00:00 2001
From: gvolfing <gabor.volfinger@memgraph.io>
Date: Wed, 8 Feb 2023 12:57:21 +0100
Subject: [PATCH 80/90] Invert boolean logic when checking for unique edges

---
 src/query/v2/plan/operator.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 4337add0d..73de9ee28 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -1341,7 +1341,7 @@ bool IsExpansionOk(Frame &frame, const Symbol &expand_symbol, const std::vector<
   return std::ranges::all_of(previous_symbols,
                              [&frame, &expand_value = frame[expand_symbol]](const auto &previous_symbol) {
                                const auto &previous_value = frame[previous_symbol];
-                               return ContainsSameEdge(previous_value, expand_value);
+                               return !ContainsSameEdge(previous_value, expand_value);
                              });
 }
 

From a9a388ce44affe027f27f33bea5fc9cdf16398ca Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 8 Feb 2023 13:52:51 +0100
Subject: [PATCH 81/90] Use parametrized queries for vertex creation

---
 tests/mgbench/datasets.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/mgbench/datasets.py b/tests/mgbench/datasets.py
index 3a5806629..83ae28a2e 100644
--- a/tests/mgbench/datasets.py
+++ b/tests/mgbench/datasets.py
@@ -353,7 +353,7 @@ class AccessControl(Dataset):
 
     def benchmark__create__vertex(self):
         self.next_value_idx += 1
-        query = (f"CREATE (:File {{uuid: {self.next_value_idx}}});", {})
+        query = ("CREATE (:File {uuid: $uuid})", {"uuid": self.next_value_idx})
         return query
 
     def benchmark__create__edges(self):

From 12bc78ca2d663eac4cd84f9d6eb09572d090ff67 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 12:41:17 +0100
Subject: [PATCH 82/90] Add command line flag to determine `MultiFrame` size

---
 src/query/v2/interpreter.cpp   | 2 +-
 src/query/v2/multiframe.cpp    | 2 ++
 src/query/v2/multiframe.hpp    | 5 ++++-
 src/query/v2/plan/operator.cpp | 8 ++++----
 4 files changed, 11 insertions(+), 6 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index aa220d764..6f07b598e 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -688,7 +688,7 @@ PullPlan::PullPlan(const std::shared_ptr<CachedPlan> plan, const Parameters &par
     : plan_(plan),
       cursor_(plan->plan().MakeCursor(execution_memory)),
       frame_(plan->symbol_table().max_position(), execution_memory),
-      multi_frame_(plan->symbol_table().max_position(), kNumberOfFramesInMultiframe, execution_memory),
+      multi_frame_(plan->symbol_table().max_position(), FLAGS_default_multi_frame_size, execution_memory),
       memory_limit_(memory_limit) {
   ctx_.db_accessor = dba;
   ctx_.symbol_table = plan->symbol_table();
diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index 477ef6c0c..264990bb7 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -17,6 +17,8 @@
 #include "query/v2/bindings/frame.hpp"
 #include "utils/pmr/vector.hpp"
 
+DEFINE_uint64(default_multi_frame_size, 100, "Default size of MultiFrame");
+
 namespace memgraph::query::v2 {
 
 static_assert(std::forward_iterator<ValidFramesReader::Iterator>);
diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index 6958ffbe8..396092c2b 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -13,10 +13,13 @@
 
 #include <iterator>
 
+#include <gflags/gflags.h>
+
 #include "query/v2/bindings/frame.hpp"
 
+DECLARE_uint64(default_multi_frame_size);
+
 namespace memgraph::query::v2 {
-constexpr uint64_t kNumberOfFramesInMultiframe = 1000;  // TODO have it configurable
 
 class ValidFramesConsumer;
 class ValidFramesModifier;
diff --git a/src/query/v2/plan/operator.cpp b/src/query/v2/plan/operator.cpp
index 1841668bb..993d24282 100644
--- a/src/query/v2/plan/operator.cpp
+++ b/src/query/v2/plan/operator.cpp
@@ -509,7 +509,7 @@ class DistributedScanAllAndFilterCursor : public Cursor {
 
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
-                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+                                          FLAGS_default_multi_frame_size, output_multi_frame.GetMemoryResource()));
       own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
       own_frames_it_ = own_frames_consumer_->begin();
     }
@@ -705,7 +705,7 @@ class DistributedScanByPrimaryKeyCursor : public Cursor {
   void EnsureOwnMultiFrameIsGood(MultiFrame &output_multi_frame) {
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
-                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+                                          FLAGS_default_multi_frame_size, output_multi_frame.GetMemoryResource()));
       own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
       own_frames_it_ = own_frames_consumer_->begin();
     }
@@ -2213,7 +2213,7 @@ class UnwindCursor : public Cursor {
 
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
-                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+                                          FLAGS_default_multi_frame_size, output_multi_frame.GetMemoryResource()));
       own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
       own_frames_it_ = own_frames_consumer_->begin();
     }
@@ -3382,7 +3382,7 @@ class DistributedExpandCursor : public Cursor {
   void EnsureOwnMultiFrameIsGood(MultiFrame &output_multi_frame) {
     if (!own_multi_frame_.has_value()) {
       own_multi_frame_.emplace(MultiFrame(output_multi_frame.GetFirstFrame().elems().size(),
-                                          kNumberOfFramesInMultiframe, output_multi_frame.GetMemoryResource()));
+                                          FLAGS_default_multi_frame_size, output_multi_frame.GetMemoryResource()));
       own_frames_consumer_.emplace(own_multi_frame_->GetValidFramesConsumer());
       own_frames_it_ = own_frames_consumer_->begin();
     }

From 563035645cb5859f51ab4f667885213858eaaa45 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 12:41:41 +0100
Subject: [PATCH 83/90] Add command line flag to opt for using `MultiFrame`

---
 src/query/v2/interpreter.cpp | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 6f07b598e..d85297909 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -64,6 +64,8 @@
 #include "utils/tsc.hpp"
 #include "utils/variant_helpers.hpp"
 
+DEFINE_bool(use_multi_frame, false, "Whether to use MultiFrame or not");
+
 namespace EventCounter {
 extern Event ReadQuery;
 extern Event WriteQuery;
@@ -812,8 +814,7 @@ std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::PullMultiple(AnyStrea
 std::optional<plan::ProfilingStatsWithTotalTime> PullPlan::Pull(AnyStream *stream, std::optional<int> n,
                                                                 const std::vector<Symbol> &output_symbols,
                                                                 std::map<std::string, TypedValue> *summary) {
-  auto should_pull_multiple = false;  // TODO on the long term, we will only use PullMultiple
-  if (should_pull_multiple) {
+  if (FLAGS_use_multi_frame) {
     return PullMultiple(stream, n, output_symbols, summary);
   }
   // Set up temporary memory for a single Pull. Initial memory comes from the

From b678e6a63b15a8255465f9d0627fbf75805ef979 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 12:42:02 +0100
Subject: [PATCH 84/90] Handle bool flags properly in benchmark runner

---
 tests/mgbench/runners.py | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)

diff --git a/tests/mgbench/runners.py b/tests/mgbench/runners.py
index 2b69a811f..cf89d7f67 100644
--- a/tests/mgbench/runners.py
+++ b/tests/mgbench/runners.py
@@ -68,6 +68,15 @@ class Memgraph:
         self._cleanup()
         atexit.unregister(self._cleanup)
 
+    # Returns None if string_value is not true or false, casing doesn't matter
+    def _get_bool_value(self, string_value):
+        lower_string_value = string_value.lower()
+        if lower_string_value == "true":
+            return True
+        if lower_string_value == "false":
+            return False
+        return None
+
     def _get_args(self, **kwargs):
         data_directory = os.path.join(self._directory.name, "memgraph")
         if self._memgraph_version >= (0, 50, 0):
@@ -83,7 +92,13 @@ class Memgraph:
             args_list = self._extra_args.split(" ")
             assert len(args_list) % 2 == 0
             for i in range(0, len(args_list), 2):
-                kwargs[args_list[i]] = args_list[i + 1]
+                key = args_list[i]
+                value = args_list[i + 1]
+                maybe_bool_value = self._get_bool_value(value)
+                if maybe_bool_value is not None:
+                    kwargs[key] = maybe_bool_value
+                else:
+                    kwargs[key] = value
 
         return _convert_args_to_flags(self._memgraph_binary, **kwargs)
 

From 53f95ed1a7c0313928554c803ca97a7843652517 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 13:17:15 +0100
Subject: [PATCH 85/90] Format python file

---
 tests/mgbench/compare_results.py | 65 ++++++++++++++------------------
 1 file changed, 29 insertions(+), 36 deletions(-)

diff --git a/tests/mgbench/compare_results.py b/tests/mgbench/compare_results.py
index 2179bb408..f10ec8bab 100755
--- a/tests/mgbench/compare_results.py
+++ b/tests/mgbench/compare_results.py
@@ -14,7 +14,6 @@
 import argparse
 import json
 
-
 FIELDS = [
     {
         "name": "throughput",
@@ -85,39 +84,32 @@ def compare_results(results_from, results_to, fields):
                 if group == "__import__":
                     continue
                 for scenario, summary_to in scenarios.items():
-                    summary_from = recursive_get(
-                        results_from, dataset, variant, group, scenario,
-                        value={})
-                    if len(summary_from) > 0 and \
-                            summary_to["count"] != summary_from["count"] or \
-                            summary_to["num_workers"] != \
-                            summary_from["num_workers"]:
+                    summary_from = recursive_get(results_from, dataset, variant, group, scenario, value={})
+                    if (
+                        len(summary_from) > 0
+                        and summary_to["count"] != summary_from["count"]
+                        or summary_to["num_workers"] != summary_from["num_workers"]
+                    ):
                         raise Exception("Incompatible results!")
-                    testcode = "/".join([dataset, variant, group, scenario,
-                                         "{:02d}".format(
-                                             summary_to["num_workers"])])
+                    testcode = "/".join([dataset, variant, group, scenario, "{:02d}".format(summary_to["num_workers"])])
                     row = {}
                     performance_changed = False
                     for field in fields:
                         key = field["name"]
                         if key in summary_to:
-                            row[key] = compute_diff(
-                                summary_from.get(key, None),
-                                summary_to[key])
+                            row[key] = compute_diff(summary_from.get(key, None), summary_to[key])
                         elif key in summary_to["database"]:
                             row[key] = compute_diff(
-                                recursive_get(summary_from, "database", key,
-                                              value=None),
-                                summary_to["database"][key])
+                                recursive_get(summary_from, "database", key, value=None), summary_to["database"][key]
+                            )
                         else:
                             row[key] = compute_diff(
-                                recursive_get(summary_from, "metadata", key,
-                                              "average", value=None),
-                                summary_to["metadata"][key]["average"])
-                        if "diff" not in row[key] or \
-                                ("diff_treshold" in field and
-                                 abs(row[key]["diff"]) >=
-                                 field["diff_treshold"]):
+                                recursive_get(summary_from, "metadata", key, "average", value=None),
+                                summary_to["metadata"][key]["average"],
+                            )
+                        if "diff" not in row[key] or (
+                            "diff_treshold" in field and abs(row[key]["diff"]) >= field["diff_treshold"]
+                        ):
                             performance_changed = True
                     if performance_changed:
                         ret[testcode] = row
@@ -130,8 +122,9 @@ def generate_remarkup(fields, data):
         ret += "<table>\n"
         ret += "  <tr>\n"
         ret += "    <th>Testcode</th>\n"
-        ret += "\n".join(map(lambda x: "    <th>{}</th>".format(
-            x["name"].replace("_", " ").capitalize()), fields)) + "\n"
+        ret += (
+            "\n".join(map(lambda x: "    <th>{}</th>".format(x["name"].replace("_", " ").capitalize()), fields)) + "\n"
+        )
         ret += "  </tr>\n"
         for testcode in sorted(data.keys()):
             ret += "  <tr>\n"
@@ -147,12 +140,9 @@ def generate_remarkup(fields, data):
                     else:
                         color = "red"
                     sign = "{{icon {} color={}}}".format(arrow, color)
-                    ret += "    <td>{:.3f}{} //({:+.2%})// {}</td>\n".format(
-                        value, field["unit"], diff, sign)
+                    ret += "    <td>{:.3f}{} //({:+.2%})// {}</td>\n".format(value, field["unit"], diff, sign)
                 else:
-                    ret += "    <td>{:.3f}{} //(new)// " \
-                           "{{icon plus color=blue}}</td>\n".format(
-                               value, field["unit"])
+                    ret += "    <td>{:.3f}{} //(new)// " "{{icon plus color=blue}}</td>\n".format(value, field["unit"])
             ret += "  </tr>\n"
         ret += "</table>\n"
     else:
@@ -161,11 +151,14 @@ def generate_remarkup(fields, data):
 
 
 if __name__ == "__main__":
-    parser = argparse.ArgumentParser(
-        description="Compare results of multiple benchmark runs.")
-    parser.add_argument("--compare", action="append", nargs=2,
-                        metavar=("from", "to"),
-                        help="compare results between `from` and `to` files")
+    parser = argparse.ArgumentParser(description="Compare results of multiple benchmark runs.")
+    parser.add_argument(
+        "--compare",
+        action="append",
+        nargs=2,
+        metavar=("from", "to"),
+        help="compare results between `from` and `to` files",
+    )
     parser.add_argument("--output", default="", help="output file name")
     args = parser.parse_args()
 

From 2b3141879bc84dac2c37b430bbca9eb58e014bd8 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 13:17:46 +0100
Subject: [PATCH 86/90] Make the output table nicer for comparing results

---
 tests/mgbench/compare_results.py | 35 ++++++++++++++++++++------------
 1 file changed, 22 insertions(+), 13 deletions(-)

diff --git a/tests/mgbench/compare_results.py b/tests/mgbench/compare_results.py
index f10ec8bab..46da74270 100755
--- a/tests/mgbench/compare_results.py
+++ b/tests/mgbench/compare_results.py
@@ -123,26 +123,35 @@ def generate_remarkup(fields, data):
         ret += "  <tr>\n"
         ret += "    <th>Testcode</th>\n"
         ret += (
-            "\n".join(map(lambda x: "    <th>{}</th>".format(x["name"].replace("_", " ").capitalize()), fields)) + "\n"
+            "\n".join(
+                map(
+                    lambda x: "    <th>{}</th>".format(x["name"].replace("_", " ").capitalize()),
+                    fields,
+                )
+            )
+            + "\n"
         )
         ret += "  </tr>\n"
         for testcode in sorted(data.keys()):
             ret += "  <tr>\n"
             ret += "    <td>{}</td>\n".format(testcode)
             for field in fields:
-                result = data[testcode][field["name"]]
-                value = result["value"] * field["scaling"]
-                if "diff" in result:
-                    diff = result["diff"]
-                    arrow = "arrow-up" if diff >= 0 else "arrow-down"
-                    if not (field["positive_diff_better"] ^ (diff >= 0)):
-                        color = "green"
+                result = data[testcode].get(field["name"])
+                if result != None:
+                    value = result["value"] * field["scaling"]
+                    if "diff" in result:
+                        diff = result["diff"]
+                        arrow = "arrow-up" if diff >= 0 else "arrow-down"
+                        if not (field["positive_diff_better"] ^ (diff >= 0)):
+                            color = "green"
+                        else:
+                            color = "red"
+                        sign = "{{icon {} color={}}}".format(arrow, color)
+                        ret += '    <td bgcolor="{}">{:.3f}{} ({:+.2%})</td>\n'.format(
+                            color, value, field["unit"], diff
+                        )
                     else:
-                        color = "red"
-                    sign = "{{icon {} color={}}}".format(arrow, color)
-                    ret += "    <td>{:.3f}{} //({:+.2%})// {}</td>\n".format(value, field["unit"], diff, sign)
-                else:
-                    ret += "    <td>{:.3f}{} //(new)// " "{{icon plus color=blue}}</td>\n".format(value, field["unit"])
+                        ret += '<td bgcolor="blue">{:.3f}{} //(new)// </td>\n'.format(value, field["unit"])
             ret += "  </tr>\n"
         ret += "</table>\n"
     else:

From 74f53369c063c993812b55c82c633b73244bb3b4 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Thu, 9 Feb 2023 12:44:34 +0100
Subject: [PATCH 87/90] Add two more queries to simple benchmark

---
 tests/mgbench/datasets.py | 20 +++++++++++++++++++-
 1 file changed, 19 insertions(+), 1 deletion(-)

diff --git a/tests/mgbench/datasets.py b/tests/mgbench/datasets.py
index 83ae28a2e..319b67c17 100644
--- a/tests/mgbench/datasets.py
+++ b/tests/mgbench/datasets.py
@@ -379,6 +379,24 @@ class AccessControl(Dataset):
         return query
 
     def benchmark__match__match_all_vertices_with_edges(self):
-        self.next_value_idx += 1
         query = ("MATCH (permission:Permission)-[e:IS_FOR_FILE]->(file:File) RETURN *", {})
         return query
+
+    def benchmark__match__match_users_with_permission_for_files(self):
+        file_uuid_1 = self._get_random_uuid("File")
+        file_uuid_2 = self._get_random_uuid("File")
+        min_file_uuid = min(file_uuid_1, file_uuid_2)
+        max_file_uuid = max(file_uuid_1, file_uuid_2)
+        query = (
+            "MATCH (f:File)<-[ff:IS_FOR_FILE]-(p:Permission)-[fi:IS_FOR_IDENTITY]->(i:Identity) WHERE f.uuid >= $min_file_uuid  AND f.uuid <= $max_file_uuid RETURN *",
+            {"min_file_uuid": min_file_uuid, "max_file_uuid": max_file_uuid},
+        )
+        return query
+
+    def benchmark__match__match_users_with_permission_for_specific_file(self):
+        file_uuid = self._get_random_uuid("File")
+        query = (
+            "MATCH (f:File {uuid: $file_uuid})<-[ff:IS_FOR_FILE]-(p:Permission)-[fi:IS_FOR_IDENTITY]->(i:Identity) RETURN *",
+            {"file_uuid": file_uuid},
+        )
+        return query

From a17010ed16e241ff0b7566b3e9d1bec2de04e49a Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 15 Feb 2023 08:37:45 +0100
Subject: [PATCH 88/90] Supress clang-tidy warnings

---
 src/query/v2/interpreter.cpp | 1 +
 src/query/v2/multiframe.cpp  | 1 +
 2 files changed, 2 insertions(+)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index d85297909..6a414d5d9 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -64,6 +64,7 @@
 #include "utils/tsc.hpp"
 #include "utils/variant_helpers.hpp"
 
+// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
 DEFINE_bool(use_multi_frame, false, "Whether to use MultiFrame or not");
 
 namespace EventCounter {
diff --git a/src/query/v2/multiframe.cpp b/src/query/v2/multiframe.cpp
index 264990bb7..8bbba08bf 100644
--- a/src/query/v2/multiframe.cpp
+++ b/src/query/v2/multiframe.cpp
@@ -17,6 +17,7 @@
 #include "query/v2/bindings/frame.hpp"
 #include "utils/pmr/vector.hpp"
 
+// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
 DEFINE_uint64(default_multi_frame_size, 100, "Default size of MultiFrame");
 
 namespace memgraph::query::v2 {

From 3b0d531343a201f1009ccd635f9c252dfb82bce3 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 15 Feb 2023 09:07:48 +0100
Subject: [PATCH 89/90] Supress clang-tidy warning

---
 src/query/v2/multiframe.hpp | 1 +
 1 file changed, 1 insertion(+)

diff --git a/src/query/v2/multiframe.hpp b/src/query/v2/multiframe.hpp
index 396092c2b..2092ec4a2 100644
--- a/src/query/v2/multiframe.hpp
+++ b/src/query/v2/multiframe.hpp
@@ -17,6 +17,7 @@
 
 #include "query/v2/bindings/frame.hpp"
 
+// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
 DECLARE_uint64(default_multi_frame_size);
 
 namespace memgraph::query::v2 {

From a2ce9c43965163829e6a21f7ee5d5c68ec36f63f Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= <benjamin.antal@memgraph.io>
Date: Wed, 15 Feb 2023 13:59:31 +0100
Subject: [PATCH 90/90] Trigger CI

---
 src/query/v2/interpreter.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/query/v2/interpreter.cpp b/src/query/v2/interpreter.cpp
index 6a414d5d9..5fcd433a1 100644
--- a/src/query/v2/interpreter.cpp
+++ b/src/query/v2/interpreter.cpp
@@ -68,6 +68,7 @@
 DEFINE_bool(use_multi_frame, false, "Whether to use MultiFrame or not");
 
 namespace EventCounter {
+
 extern Event ReadQuery;
 extern Event WriteQuery;
 extern Event ReadWriteQuery;
@@ -77,6 +78,7 @@ extern const Event LabelPropertyIndexCreated;
 
 extern const Event StreamsCreated;
 extern const Event TriggersCreated;
+
 }  // namespace EventCounter
 
 namespace memgraph::query::v2 {