From e5437080c5df58a16067d4d662aecabe96c3df79 Mon Sep 17 00:00:00 2001
From: Jure Bajic <jure.bajic@memgraph.com>
Date: Fri, 21 Oct 2022 12:33:40 +0200
Subject: [PATCH] Handle OrderBy in ScanVertices request (#594)

- Refactor shard_rsm and move function into expr.hpp/expr.cpp and request_helper.hpp/request_helper.cpp
---
 src/query/v2/requests.hpp            |   1 +
 src/storage/v3/CMakeLists.txt        |   2 +
 src/storage/v3/expr.cpp              | 210 +++++++++++++++++
 src/storage/v3/expr.hpp              |  55 +++++
 src/storage/v3/request_helper.cpp    |  82 +++++++
 src/storage/v3/request_helper.hpp    | 116 +++++++++
 src/storage/v3/shard_rsm.cpp         | 340 ++++++++-------------------
 tests/simulation/shard_rsm.cpp       |  80 ++++++-
 tools/gdb-plugins/pretty_printers.py | 204 +++++++++++++---
 9 files changed, 808 insertions(+), 282 deletions(-)
 create mode 100644 src/storage/v3/expr.cpp
 create mode 100644 src/storage/v3/expr.hpp
 create mode 100644 src/storage/v3/request_helper.cpp
 create mode 100644 src/storage/v3/request_helper.hpp

diff --git a/src/query/v2/requests.hpp b/src/query/v2/requests.hpp
index 12fa2ea95..49ae346cb 100644
--- a/src/query/v2/requests.hpp
+++ b/src/query/v2/requests.hpp
@@ -345,6 +345,7 @@ struct ScanVerticesRequest {
   // expression whose result is returned for every vertex
   std::vector<std::string> vertex_expressions;
   std::optional<size_t> batch_limit;
+  std::vector<OrderBy> order_bys;
   StorageView storage_view{StorageView::NEW};
 
   std::optional<Label> label;
diff --git a/src/storage/v3/CMakeLists.txt b/src/storage/v3/CMakeLists.txt
index 657897b0c..0f359c702 100644
--- a/src/storage/v3/CMakeLists.txt
+++ b/src/storage/v3/CMakeLists.txt
@@ -19,6 +19,8 @@ set(storage_v3_src_files
     storage.cpp
     shard_rsm.cpp
     bindings/typed_value.cpp
+    expr.cpp
+    request_helper.cpp
     storage.cpp)
 
 # ######################
diff --git a/src/storage/v3/expr.cpp b/src/storage/v3/expr.cpp
new file mode 100644
index 000000000..1d6739433
--- /dev/null
+++ b/src/storage/v3/expr.cpp
@@ -0,0 +1,210 @@
+// 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 "storage/v3/expr.hpp"
+
+#include <vector>
+
+#include "db_accessor.hpp"
+#include "opencypher/parser.hpp"
+#include "query/v2/requests.hpp"
+#include "storage/v3/bindings/ast/ast.hpp"
+#include "storage/v3/bindings/bindings.hpp"
+#include "storage/v3/bindings/cypher_main_visitor.hpp"
+#include "storage/v3/bindings/db_accessor.hpp"
+#include "storage/v3/bindings/eval.hpp"
+#include "storage/v3/bindings/frame.hpp"
+#include "storage/v3/bindings/symbol_generator.hpp"
+#include "storage/v3/bindings/symbol_table.hpp"
+#include "storage/v3/bindings/typed_value.hpp"
+#include "storage/v3/value_conversions.hpp"
+
+namespace memgraph::storage::v3 {
+
+msgs::Value ConstructValueVertex(const VertexAccessor &acc, View view) {
+  // Get the vertex id
+  auto prim_label = acc.PrimaryLabel(view).GetValue();
+  memgraph::msgs::Label value_label{.id = prim_label};
+
+  auto prim_key = conversions::ConvertValueVector(acc.PrimaryKey(view).GetValue());
+  memgraph::msgs::VertexId vertex_id = std::make_pair(value_label, prim_key);
+
+  // Get the labels
+  auto vertex_labels = acc.Labels(view).GetValue();
+  std::vector<memgraph::msgs::Label> value_labels;
+  value_labels.reserve(vertex_labels.size());
+
+  std::transform(vertex_labels.begin(), vertex_labels.end(), std::back_inserter(value_labels),
+                 [](const auto &label) { return msgs::Label{.id = label}; });
+
+  return msgs::Value({.id = vertex_id, .labels = value_labels});
+}
+
+msgs::Value ConstructValueEdge(const EdgeAccessor &acc, View view) {
+  msgs::EdgeType type = {.id = acc.EdgeType()};
+  msgs::EdgeId gid = {.gid = acc.Gid().AsUint()};
+
+  msgs::Label src_prim_label = {.id = acc.FromVertex().primary_label};
+  memgraph::msgs::VertexId src_vertex =
+      std::make_pair(src_prim_label, conversions::ConvertValueVector(acc.FromVertex().primary_key));
+
+  msgs::Label dst_prim_label = {.id = acc.ToVertex().primary_label};
+  msgs::VertexId dst_vertex =
+      std::make_pair(dst_prim_label, conversions::ConvertValueVector(acc.ToVertex().primary_key));
+
+  auto properties = acc.Properties(view);
+
+  std::vector<std::pair<PropertyId, msgs::Value>> present_properties;
+  if (properties.HasValue()) {
+    auto props = properties.GetValue();
+    std::vector<std::pair<PropertyId, msgs::Value>> present_properties;
+    present_properties.reserve(props.size());
+
+    std::transform(props.begin(), props.end(), std::back_inserter(present_properties),
+                   [](std::pair<const PropertyId, PropertyValue> &prop) {
+                     return std::make_pair(prop.first, conversions::FromPropertyValueToValue(std::move(prop.second)));
+                   });
+  }
+
+  return msgs::Value(msgs::Edge{.src = std::move(src_vertex),
+                                .dst = std::move(dst_vertex),
+                                .properties = std::move(present_properties),
+                                .id = gid,
+                                .type = type});
+}
+
+msgs::Value FromTypedValueToValue(TypedValue &&tv) {
+  switch (tv.type()) {
+    case TypedValue::Type::Bool:
+      return msgs::Value(tv.ValueBool());
+    case TypedValue::Type::Double:
+      return msgs::Value(tv.ValueDouble());
+    case TypedValue::Type::Int:
+      return msgs::Value(tv.ValueInt());
+    case TypedValue::Type::List: {
+      std::vector<msgs::Value> list;
+      auto &tv_list = tv.ValueList();
+      list.reserve(tv_list.size());
+      std::transform(tv_list.begin(), tv_list.end(), std::back_inserter(list),
+                     [](auto &elem) { return FromTypedValueToValue(std::move(elem)); });
+      return msgs::Value(list);
+    }
+    case TypedValue::Type::Map: {
+      std::map<std::string, msgs::Value> map;
+      for (auto &[key, val] : tv.ValueMap()) {
+        map.emplace(key, FromTypedValueToValue(std::move(val)));
+      }
+      return msgs::Value(map);
+    }
+    case TypedValue::Type::Null:
+      return {};
+    case TypedValue::Type::String:
+      return msgs::Value((std::string(tv.ValueString())));
+    case TypedValue::Type::Vertex:
+      return ConstructValueVertex(tv.ValueVertex(), View::OLD);
+    case TypedValue::Type::Edge:
+      return ConstructValueEdge(tv.ValueEdge(), View::OLD);
+
+    // TBD -> we need to specify temporal types, not a priority.
+    case TypedValue::Type::Date:
+    case TypedValue::Type::LocalTime:
+    case TypedValue::Type::LocalDateTime:
+    case TypedValue::Type::Duration:
+    case TypedValue::Type::Path: {
+      MG_ASSERT(false, "This conversion between TypedValue and Value is not implemented yet!");
+      break;
+    }
+  }
+  return {};
+}
+
+std::vector<msgs::Value> ConvertToValueVectorFromTypedValueVector(
+    std::vector<memgraph::storage::v3::TypedValue> &&vec) {
+  std::vector<msgs::Value> ret;
+  ret.reserve(vec.size());
+
+  std::transform(vec.begin(), vec.end(), std::back_inserter(ret),
+                 [](auto &elem) { return FromTypedValueToValue(std::move(elem)); });
+  return ret;
+}
+
+std::vector<PropertyId> NamesToProperties(const std::vector<std::string> &property_names, DbAccessor &dba) {
+  std::vector<PropertyId> properties;
+  properties.reserve(property_names.size());
+
+  for (const auto &name : property_names) {
+    properties.push_back(dba.NameToProperty(name));
+  }
+  return properties;
+}
+
+std::vector<memgraph::storage::v3::LabelId> NamesToLabels(const std::vector<std::string> &label_names,
+                                                          DbAccessor &dba) {
+  std::vector<memgraph::storage::v3::LabelId> labels;
+  labels.reserve(label_names.size());
+  for (const auto &name : label_names) {
+    labels.push_back(dba.NameToLabel(name));
+  }
+  return labels;
+}
+
+std::any ParseExpression(const std::string &expr, memgraph::expr::AstStorage &storage) {
+  memgraph::frontend::opencypher::Parser<memgraph::frontend::opencypher::ParserOpTag::EXPRESSION> parser(expr);
+  ParsingContext pc;
+  CypherMainVisitor visitor(pc, &storage);
+
+  auto *ast = parser.tree();
+  return visitor.visit(ast);
+}
+
+TypedValue ComputeExpression(DbAccessor &dba, const std::optional<memgraph::storage::v3::VertexAccessor> &v_acc,
+                             const std::optional<memgraph::storage::v3::EdgeAccessor> &e_acc,
+                             const std::string &expression, std::string_view node_name, std::string_view edge_name) {
+  AstStorage storage;
+  Frame frame{1 + 1};  // 1 for the node_identifier, 1 for the edge_identifier
+  SymbolTable symbol_table;
+  EvaluationContext ctx;
+
+  ExpressionEvaluator eval{&frame, symbol_table, ctx, &dba, View::OLD};
+  auto expr = ParseExpression(expression, storage);
+
+  auto node_identifier = Identifier(std::string(node_name), false);
+  auto edge_identifier = Identifier(std::string(edge_name), false);
+
+  std::vector<Identifier *> identifiers;
+  identifiers.push_back(&node_identifier);
+  identifiers.push_back(&edge_identifier);
+
+  expr::SymbolGenerator symbol_generator(&symbol_table, identifiers);
+  (std::any_cast<Expression *>(expr))->Accept(symbol_generator);
+
+  if (node_identifier.symbol_pos_ != -1) {
+    MG_ASSERT(std::find_if(symbol_table.table().begin(), symbol_table.table().end(),
+                           [&node_name](const std::pair<int32_t, Symbol> &position_symbol_pair) {
+                             return position_symbol_pair.second.name() == node_name;
+                           }) != symbol_table.table().end());
+
+    frame[symbol_table.at(node_identifier)] = *v_acc;
+  }
+
+  if (edge_identifier.symbol_pos_ != -1) {
+    MG_ASSERT(std::find_if(symbol_table.table().begin(), symbol_table.table().end(),
+                           [&edge_name](const std::pair<int32_t, Symbol> &position_symbol_pair) {
+                             return position_symbol_pair.second.name() == edge_name;
+                           }) != symbol_table.table().end());
+
+    frame[symbol_table.at(edge_identifier)] = *e_acc;
+  }
+
+  return Eval(std::any_cast<Expression *>(expr), ctx, storage, eval, dba);
+}
+
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/expr.hpp b/src/storage/v3/expr.hpp
new file mode 100644
index 000000000..c3199abf1
--- /dev/null
+++ b/src/storage/v3/expr.hpp
@@ -0,0 +1,55 @@
+// 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 <vector>
+
+#include "db_accessor.hpp"
+#include "opencypher/parser.hpp"
+#include "query/v2/requests.hpp"
+#include "storage/v3/bindings/ast/ast.hpp"
+#include "storage/v3/bindings/bindings.hpp"
+#include "storage/v3/bindings/cypher_main_visitor.hpp"
+#include "storage/v3/bindings/db_accessor.hpp"
+#include "storage/v3/bindings/eval.hpp"
+#include "storage/v3/bindings/frame.hpp"
+#include "storage/v3/bindings/symbol_generator.hpp"
+#include "storage/v3/bindings/symbol_table.hpp"
+#include "storage/v3/bindings/typed_value.hpp"
+
+namespace memgraph::storage::v3 {
+
+memgraph::msgs::Value ConstructValueVertex(const memgraph::storage::v3::VertexAccessor &acc, View view);
+
+msgs::Value ConstructValueEdge(const EdgeAccessor &acc, View view);
+
+msgs::Value FromTypedValueToValue(TypedValue &&tv);
+
+std::vector<msgs::Value> ConvertToValueVectorFromTypedValueVector(std::vector<TypedValue> &&vec);
+
+std::vector<PropertyId> NamesToProperties(const std::vector<std::string> &property_names, DbAccessor &dba);
+
+std::vector<LabelId> NamesToLabels(const std::vector<std::string> &label_names, DbAccessor &dba);
+
+template <class TExpression>
+auto Eval(TExpression *expr, EvaluationContext &ctx, AstStorage &storage, ExpressionEvaluator &eval, DbAccessor &dba) {
+  ctx.properties = NamesToProperties(storage.properties_, dba);
+  ctx.labels = NamesToLabels(storage.labels_, dba);
+  auto value = expr->Accept(eval);
+  return value;
+}
+
+std::any ParseExpression(const std::string &expr, AstStorage &storage);
+
+TypedValue ComputeExpression(DbAccessor &dba, const std::optional<VertexAccessor> &v_acc,
+                             const std::optional<EdgeAccessor> &e_acc, const std::string &expression,
+                             std::string_view node_name, std::string_view edge_name);
+
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/request_helper.cpp b/src/storage/v3/request_helper.cpp
new file mode 100644
index 000000000..bb1c8bca4
--- /dev/null
+++ b/src/storage/v3/request_helper.cpp
@@ -0,0 +1,82 @@
+// 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 "storage/v3/request_helper.hpp"
+
+#include <vector>
+
+#include "pretty_print_ast_to_original_expression.hpp"
+#include "storage/v3/bindings/db_accessor.hpp"
+#include "storage/v3/expr.hpp"
+
+namespace memgraph::storage::v3 {
+
+std::vector<Element> OrderByElements(Shard::Accessor &acc, DbAccessor &dba, VerticesIterable &vertices_iterable,
+                                     std::vector<msgs::OrderBy> &order_bys) {
+  std::vector<Element> ordered;
+  ordered.reserve(acc.ApproximateVertexCount());
+  std::vector<Ordering> ordering;
+  ordering.reserve(order_bys.size());
+  for (const auto &order : order_bys) {
+    switch (order.direction) {
+      case memgraph::msgs::OrderingDirection::ASCENDING: {
+        ordering.push_back(Ordering::ASC);
+        break;
+      }
+      case memgraph::msgs::OrderingDirection::DESCENDING: {
+        ordering.push_back(Ordering::DESC);
+        break;
+      }
+    }
+  }
+  auto compare_typed_values = TypedValueVectorCompare(ordering);
+  for (auto it = vertices_iterable.begin(); it != vertices_iterable.end(); ++it) {
+    std::vector<TypedValue> properties_order_by;
+    properties_order_by.reserve(order_bys.size());
+
+    for (const auto &order_by : order_bys) {
+      const auto val =
+          ComputeExpression(dba, *it, std::nullopt, order_by.expression.expression, expr::identifier_node_symbol, "");
+      properties_order_by.push_back(val);
+    }
+    ordered.push_back({std::move(properties_order_by), *it});
+  }
+
+  std::sort(ordered.begin(), ordered.end(), [compare_typed_values](const auto &pair1, const auto &pair2) {
+    return compare_typed_values(pair1.properties_order_by, pair2.properties_order_by);
+  });
+  return ordered;
+}
+
+VerticesIterable::Iterator GetStartVertexIterator(VerticesIterable &vertex_iterable,
+                                                  const std::vector<PropertyValue> &start_ids, const View view) {
+  auto it = vertex_iterable.begin();
+  while (it != vertex_iterable.end()) {
+    if (const auto &vertex = *it; start_ids <= vertex.PrimaryKey(view).GetValue()) {
+      break;
+    }
+    ++it;
+  }
+  return it;
+}
+
+std::vector<Element>::const_iterator GetStartOrderedElementsIterator(const std::vector<Element> &ordered_elements,
+                                                                     const std::vector<PropertyValue> &start_ids,
+                                                                     const View view) {
+  for (auto it = ordered_elements.begin(); it != ordered_elements.end(); ++it) {
+    if (const auto &vertex = it->vertex_acc; start_ids <= vertex.PrimaryKey(view).GetValue()) {
+      return it;
+    }
+  }
+  return ordered_elements.end();
+}
+
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/request_helper.hpp b/src/storage/v3/request_helper.hpp
new file mode 100644
index 000000000..24ed40f8c
--- /dev/null
+++ b/src/storage/v3/request_helper.hpp
@@ -0,0 +1,116 @@
+// 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 <vector>
+
+#include "ast/ast.hpp"
+#include "storage/v3/bindings/typed_value.hpp"
+#include "storage/v3/shard.hpp"
+#include "storage/v3/vertex_accessor.hpp"
+
+namespace memgraph::storage::v3 {
+
+inline bool TypedValueCompare(const TypedValue &a, const TypedValue &b) {
+  // in ordering null comes after everything else
+  // at the same time Null is not less that null
+  // first deal with Null < Whatever case
+  if (a.IsNull()) return false;
+  // now deal with NotNull < Null case
+  if (b.IsNull()) return true;
+
+  // comparisons are from this point legal only between values of
+  // the  same type, or int+float combinations
+  if ((a.type() != b.type() && !(a.IsNumeric() && b.IsNumeric())))
+    throw utils::BasicException("Can't compare value of type {} to value of type {}.", a.type(), b.type());
+
+  switch (a.type()) {
+    case TypedValue::Type::Bool:
+      return !a.ValueBool() && b.ValueBool();
+    case TypedValue::Type::Int:
+      if (b.type() == TypedValue::Type::Double)
+        // NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
+        return a.ValueInt() < b.ValueDouble();
+      else
+        return a.ValueInt() < b.ValueInt();
+    case TypedValue::Type::Double:
+      if (b.type() == TypedValue::Type::Int)
+        // NOLINTNEXTLINE(bugprone-narrowing-conversions,cppcoreguidelines-narrowing-conversions)
+        return a.ValueDouble() < b.ValueInt();
+      else
+        return a.ValueDouble() < b.ValueDouble();
+    case TypedValue::Type::String:
+      // NOLINTNEXTLINE(modernize-use-nullptr)
+      return a.ValueString() < b.ValueString();
+    case TypedValue::Type::Date:
+      // NOLINTNEXTLINE(modernize-use-nullptr)
+      return a.ValueDate() < b.ValueDate();
+    case TypedValue::Type::LocalTime:
+      // NOLINTNEXTLINE(modernize-use-nullptr)
+      return a.ValueLocalTime() < b.ValueLocalTime();
+    case TypedValue::Type::LocalDateTime:
+      // NOLINTNEXTLINE(modernize-use-nullptr)
+      return a.ValueLocalDateTime() < b.ValueLocalDateTime();
+    case TypedValue::Type::Duration:
+      // NOLINTNEXTLINE(modernize-use-nullptr)
+      return a.ValueDuration() < b.ValueDuration();
+    case TypedValue::Type::List:
+    case TypedValue::Type::Map:
+    case TypedValue::Type::Vertex:
+    case TypedValue::Type::Edge:
+    case TypedValue::Type::Path:
+      throw utils::BasicException("Comparison is not defined for values of type {}.", a.type());
+    case TypedValue::Type::Null:
+      LOG_FATAL("Invalid type");
+  }
+}
+
+class TypedValueVectorCompare final {
+ public:
+  explicit TypedValueVectorCompare(const std::vector<Ordering> &ordering) : ordering_(ordering) {}
+
+  bool operator()(const std::vector<TypedValue> &c1, const std::vector<TypedValue> &c2) const {
+    // ordering is invalid if there are more elements in the collections
+    // then there are in the ordering_ vector
+    MG_ASSERT(c1.size() <= ordering_.size() && c2.size() <= ordering_.size(),
+              "Collections contain more elements then there are orderings");
+
+    auto c1_it = c1.begin();
+    auto c2_it = c2.begin();
+    auto ordering_it = ordering_.begin();
+    for (; c1_it != c1.end() && c2_it != c2.end(); c1_it++, c2_it++, ordering_it++) {
+      if (TypedValueCompare(*c1_it, *c2_it)) return *ordering_it == Ordering::ASC;
+      if (TypedValueCompare(*c2_it, *c1_it)) return *ordering_it == Ordering::DESC;
+    }
+
+    // at least one collection is exhausted
+    // c1 is less then c2 iff c1 reached the end but c2 didn't
+    return (c1_it == c1.end()) && (c2_it != c2.end());
+  }
+
+ private:
+  std::vector<Ordering> ordering_;
+};
+
+struct Element {
+  std::vector<TypedValue> properties_order_by;
+  VertexAccessor vertex_acc;
+};
+
+std::vector<Element> OrderByElements(Shard::Accessor &acc, DbAccessor &dba, VerticesIterable &vertices_iterable,
+                                     std::vector<msgs::OrderBy> &order_bys);
+
+VerticesIterable::Iterator GetStartVertexIterator(VerticesIterable &vertex_iterable,
+                                                  const std::vector<PropertyValue> &start_ids, View view);
+
+std::vector<Element>::const_iterator GetStartOrderedElementsIterator(const std::vector<Element> &ordered_elements,
+                                                                     const std::vector<PropertyValue> &start_ids,
+                                                                     View view);
+}  // namespace memgraph::storage::v3
diff --git a/src/storage/v3/shard_rsm.cpp b/src/storage/v3/shard_rsm.cpp
index f598c6c54..679f95172 100644
--- a/src/storage/v3/shard_rsm.cpp
+++ b/src/storage/v3/shard_rsm.cpp
@@ -26,10 +26,13 @@
 #include "storage/v3/bindings/symbol_generator.hpp"
 #include "storage/v3/bindings/symbol_table.hpp"
 #include "storage/v3/bindings/typed_value.hpp"
+#include "storage/v3/expr.hpp"
 #include "storage/v3/id_types.hpp"
 #include "storage/v3/key_store.hpp"
 #include "storage/v3/property_value.hpp"
+#include "storage/v3/request_helper.hpp"
 #include "storage/v3/schemas.hpp"
+#include "storage/v3/shard.hpp"
 #include "storage/v3/shard_rsm.hpp"
 #include "storage/v3/storage.hpp"
 #include "storage/v3/value_conversions.hpp"
@@ -60,10 +63,10 @@ using SpecificEdgeProperties = std::tuple<msgs::VertexId, msgs::Gid, SpecificEdg
 using SpecificEdgePropertiesVector = std::vector<SpecificEdgeProperties>;
 using AllEdgePropertiesVector = std::vector<AllEdgeProperties>;
 
-using EdgeAccessors = std::vector<memgraph::storage::v3::EdgeAccessor>;
+using EdgeAccessors = std::vector<storage::v3::EdgeAccessor>;
 
 using EdgeFiller = std::function<bool(const EdgeAccessor &edge, bool is_in_edge, msgs::ExpandOneResultRow &result_row)>;
-using EdgeUniqunessFunction = std::function<EdgeAccessors(EdgeAccessors &&, memgraph::msgs::EdgeDirection)>;
+using EdgeUniqunessFunction = std::function<EdgeAccessors(EdgeAccessors &&, msgs::EdgeDirection)>;
 
 struct VertexIdCmpr {
   bool operator()(const storage::v3::VertexId *lhs, const storage::v3::VertexId *rhs) const { return *lhs < *rhs; }
@@ -142,188 +145,7 @@ std::optional<std::map<PropertyId, Value>> CollectAllPropertiesFromAccessor(cons
   return ret;
 }
 
-msgs::Value ConstructValueVertex(const VertexAccessor &acc, View view) {
-  // Get the vertex id
-  auto prim_label = acc.PrimaryLabel(view).GetValue();
-  msgs::Label value_label{.id = prim_label};
-
-  auto prim_key = ConvertValueVector(acc.PrimaryKey(view).GetValue());
-  msgs::VertexId vertex_id = std::make_pair(value_label, prim_key);
-
-  // Get the labels
-  auto vertex_labels = acc.Labels(view).GetValue();
-  std::vector<msgs::Label> value_labels;
-  value_labels.reserve(vertex_labels.size());
-
-  std::transform(vertex_labels.begin(), vertex_labels.end(), std::back_inserter(value_labels),
-                 [](const auto &label) { return msgs::Label{.id = label}; });
-
-  return Value({.id = vertex_id, .labels = value_labels});
-}
-
-Value ConstructValueEdge(const EdgeAccessor &acc, View view) {
-  msgs::EdgeType type = {.id = acc.EdgeType()};
-  msgs::EdgeId gid = {.gid = acc.Gid().AsUint()};
-
-  Label src_prim_label = {.id = acc.FromVertex().primary_label};
-  msgs::VertexId src_vertex = std::make_pair(src_prim_label, ConvertValueVector(acc.FromVertex().primary_key));
-
-  Label dst_prim_label = {.id = acc.ToVertex().primary_label};
-  msgs::VertexId dst_vertex = std::make_pair(dst_prim_label, ConvertValueVector(acc.ToVertex().primary_key));
-
-  auto properties = acc.Properties(view);
-
-  std::vector<std::pair<PropertyId, Value>> present_properties;
-  if (properties.HasValue()) {
-    auto &props = properties.GetValue();
-    present_properties.reserve(props.size());
-
-    std::transform(props.begin(), props.end(), std::back_inserter(present_properties),
-                   [](std::pair<const PropertyId, PropertyValue> &prop) {
-                     return std::make_pair(prop.first, FromPropertyValueToValue(std::move(prop.second)));
-                   });
-  }
-
-  return Value(msgs::Edge{.src = std::move(src_vertex),
-                          .dst = std::move(dst_vertex),
-                          .properties = std::move(present_properties),
-                          .id = gid,
-                          .type = type});
-}
-
-Value FromTypedValueToValue(TypedValue &&tv) {
-  switch (tv.type()) {
-    case TypedValue::Type::Bool:
-      return Value(tv.ValueBool());
-    case TypedValue::Type::Double:
-      return Value(tv.ValueDouble());
-    case TypedValue::Type::Int:
-      return Value(tv.ValueInt());
-    case TypedValue::Type::List: {
-      std::vector<Value> list;
-      auto &tv_list = tv.ValueList();
-      list.reserve(tv_list.size());
-      std::transform(tv_list.begin(), tv_list.end(), std::back_inserter(list),
-                     [](auto &elem) { return FromTypedValueToValue(std::move(elem)); });
-      return Value(list);
-    }
-    case TypedValue::Type::Map: {
-      std::map<std::string, Value> map;
-      for (auto &[key, val] : tv.ValueMap()) {
-        map.emplace(key, FromTypedValueToValue(std::move(val)));
-      }
-      return Value(map);
-    }
-    case TypedValue::Type::Null:
-      return Value{};
-    case TypedValue::Type::String:
-      return Value((std::string(tv.ValueString())));
-    case TypedValue::Type::Vertex:
-      return ConstructValueVertex(tv.ValueVertex(), View::OLD);
-    case TypedValue::Type::Edge:
-      return ConstructValueEdge(tv.ValueEdge(), View::OLD);
-
-    // TBD -> we need to specify temporal types, not a priority.
-    case TypedValue::Type::Date:
-    case TypedValue::Type::LocalTime:
-    case TypedValue::Type::LocalDateTime:
-    case TypedValue::Type::Duration:
-    case TypedValue::Type::Path: {
-      MG_ASSERT(false, "This conversion between TypedValue and Value is not implemented yet!");
-      break;
-    }
-  }
-  return Value{};
-}
-
-std::vector<Value> ConvertToValueVectorFromTypedValueVector(std::vector<TypedValue> &&vec) {
-  std::vector<Value> ret;
-  ret.reserve(vec.size());
-
-  std::transform(vec.begin(), vec.end(), std::back_inserter(ret),
-                 [](auto &elem) { return FromTypedValueToValue(std::move(elem)); });
-  return ret;
-}
-
-std::vector<PropertyId> NamesToProperties(const std::vector<std::string> &property_names, DbAccessor &dba) {
-  std::vector<PropertyId> properties;
-  properties.reserve(property_names.size());
-
-  for (const auto &name : property_names) {
-    properties.push_back(dba.NameToProperty(name));
-  }
-  return properties;
-}
-
-std::vector<LabelId> NamesToLabels(const std::vector<std::string> &label_names, DbAccessor &dba) {
-  std::vector<LabelId> labels;
-  labels.reserve(label_names.size());
-  for (const auto &name : label_names) {
-    labels.push_back(dba.NameToLabel(name));
-  }
-  return labels;
-}
-
-template <class TExpression>
-auto Eval(TExpression *expr, EvaluationContext &ctx, AstStorage &storage, ExpressionEvaluator &eval, DbAccessor &dba) {
-  ctx.properties = NamesToProperties(storage.properties_, dba);
-  ctx.labels = NamesToLabels(storage.labels_, dba);
-  auto value = expr->Accept(eval);
-  return value;
-}
-
-std::any ParseExpression(const std::string &expr, memgraph::expr::AstStorage &storage) {
-  memgraph::frontend::opencypher::Parser<memgraph::frontend::opencypher::ParserOpTag::EXPRESSION> parser(expr);
-  ParsingContext pc;
-  CypherMainVisitor visitor(pc, &storage);
-
-  auto *ast = parser.tree();
-  return visitor.visit(ast);
-}
-
-TypedValue ComputeExpression(DbAccessor &dba, const std::optional<VertexAccessor> &v_acc,
-                             const std::optional<EdgeAccessor> &e_acc, const std::string &expression,
-                             std::string_view node_name, std::string_view edge_name) {
-  AstStorage storage;
-  Frame frame{1 + 1};  // 1 for the node_identifier, 1 for the edge_identifier
-  SymbolTable symbol_table;
-  EvaluationContext ctx;
-
-  ExpressionEvaluator eval{&frame, symbol_table, ctx, &dba, View::OLD};
-  auto expr = ParseExpression(expression, storage);
-
-  auto node_identifier = Identifier(std::string(node_name), false);
-  auto edge_identifier = Identifier(std::string(edge_name), false);
-
-  std::vector<Identifier *> identifiers;
-  identifiers.push_back(&node_identifier);
-  identifiers.push_back(&edge_identifier);
-
-  expr::SymbolGenerator symbol_generator(&symbol_table, identifiers);
-  (std::any_cast<Expression *>(expr))->Accept(symbol_generator);
-
-  if (node_identifier.symbol_pos_ != -1) {
-    MG_ASSERT(std::find_if(symbol_table.table().begin(), symbol_table.table().end(),
-                           [&node_name](const std::pair<int32_t, Symbol> &position_symbol_pair) {
-                             return position_symbol_pair.second.name() == node_name;
-                           }) != symbol_table.table().end());
-
-    frame[symbol_table.at(node_identifier)] = *v_acc;
-  }
-
-  if (edge_identifier.symbol_pos_ != -1) {
-    MG_ASSERT(std::find_if(symbol_table.table().begin(), symbol_table.table().end(),
-                           [&edge_name](const std::pair<int32_t, Symbol> &position_symbol_pair) {
-                             return position_symbol_pair.second.name() == edge_name;
-                           }) != symbol_table.table().end());
-
-    frame[symbol_table.at(edge_identifier)] = *e_acc;
-  }
-
-  return Eval(std::any_cast<Expression *>(expr), ctx, storage, eval, dba);
-}
-
-bool FilterOnVertex(DbAccessor &dba, const VertexAccessor &v_acc, const std::vector<std::string> &filters,
+bool FilterOnVertex(DbAccessor &dba, const storage::v3::VertexAccessor &v_acc, const std::vector<std::string> &filters,
                     const std::string_view node_name) {
   return std::ranges::all_of(filters, [&node_name, &dba, &v_acc](const auto &filter_expr) {
     auto res = ComputeExpression(dba, v_acc, std::nullopt, filter_expr, node_name, "");
@@ -462,6 +284,17 @@ std::optional<std::array<std::vector<EdgeAccessor>, 2>> FillUpConnectingEdges(
   return std::array<std::vector<EdgeAccessor>, 2>{in_edges, out_edges};
 }
 
+using AllEdgePropertyDataSructure = std::map<PropertyId, msgs::Value>;
+using SpecificEdgePropertyDataSructure = std::vector<msgs::Value>;
+
+using AllEdgeProperties = std::tuple<msgs::VertexId, msgs::Gid, AllEdgePropertyDataSructure>;
+using SpecificEdgeProperties = std::tuple<msgs::VertexId, msgs::Gid, SpecificEdgePropertyDataSructure>;
+
+using SpecificEdgePropertiesVector = std::vector<SpecificEdgeProperties>;
+using AllEdgePropertiesVector = std::vector<AllEdgeProperties>;
+
+using EdgeFiller = std::function<bool(const EdgeAccessor &edge, bool is_in_edge, msgs::ExpandOneResultRow &result_row)>;
+
 template <bool are_in_edges>
 bool FillEdges(const std::vector<EdgeAccessor> &edges, const msgs::ExpandOneRequest &req, msgs::ExpandOneResultRow &row,
                const EdgeFiller &edge_filler) {
@@ -525,30 +358,28 @@ EdgeUniqunessFunction InitializeEdgeUniqunessFunction(bool only_unique_neighbor_
 
   if (only_unique_neighbor_rows) {
     maybe_filter_based_on_edge_uniquness = [](EdgeAccessors &&edges,
-                                              memgraph::msgs::EdgeDirection edge_direction) -> EdgeAccessors {
-      std::function<bool(std::set<const storage::v3::VertexId *, VertexIdCmpr> &,
-                         const memgraph::storage::v3::EdgeAccessor &)>
+                                              msgs::EdgeDirection edge_direction) -> EdgeAccessors {
+      std::function<bool(std::set<const storage::v3::VertexId *, VertexIdCmpr> &, const storage::v3::EdgeAccessor &)>
           is_edge_unique;
       switch (edge_direction) {
-        case memgraph::msgs::EdgeDirection::OUT: {
+        case msgs::EdgeDirection::OUT: {
           is_edge_unique = [](std::set<const storage::v3::VertexId *, VertexIdCmpr> &other_vertex_set,
-                              const memgraph::storage::v3::EdgeAccessor &edge_acc) {
+                              const storage::v3::EdgeAccessor &edge_acc) {
             auto [it, insertion_happened] = other_vertex_set.insert(&edge_acc.ToVertex());
             return insertion_happened;
           };
           break;
         }
-        case memgraph::msgs::EdgeDirection::IN: {
+        case msgs::EdgeDirection::IN: {
           is_edge_unique = [](std::set<const storage::v3::VertexId *, VertexIdCmpr> &other_vertex_set,
-                              const memgraph::storage::v3::EdgeAccessor &edge_acc) {
+                              const storage::v3::EdgeAccessor &edge_acc) {
             auto [it, insertion_happened] = other_vertex_set.insert(&edge_acc.FromVertex());
             return insertion_happened;
           };
           break;
         }
-        case memgraph::msgs::EdgeDirection::BOTH:
-          MG_ASSERT(false,
-                    "This is should never happen, memgraph::msgs::EdgeDirection::BOTH should not be passed here.");
+        case msgs::EdgeDirection::BOTH:
+          MG_ASSERT(false, "This is should never happen, msgs::EdgeDirection::BOTH should not be passed here.");
       }
 
       EdgeAccessors ret;
@@ -563,10 +394,8 @@ EdgeUniqunessFunction InitializeEdgeUniqunessFunction(bool only_unique_neighbor_
       return ret;
     };
   } else {
-    maybe_filter_based_on_edge_uniquness = [](EdgeAccessors &&edges,
-                                              memgraph::msgs::EdgeDirection /*edge_direction*/) -> EdgeAccessors {
-      return std::move(edges);
-    };
+    maybe_filter_based_on_edge_uniquness =
+        [](EdgeAccessors &&edges, msgs::EdgeDirection /*edge_direction*/) -> EdgeAccessors { return std::move(edges); };
   }
 
   return maybe_filter_based_on_edge_uniquness;
@@ -920,65 +749,82 @@ msgs::ReadResponses ShardRsm::HandleRead(msgs::ScanVerticesRequest &&req) {
   bool action_successful = true;
 
   std::vector<msgs::ScanResultRow> results;
+  if (req.batch_limit) {
+    results.reserve(*req.batch_limit);
+  }
   std::optional<msgs::VertexId> next_start_id;
 
   const auto view = View(req.storage_view);
-  auto vertex_iterable = acc.Vertices(view);
-  bool did_reach_starting_point = false;
-  uint64_t sample_counter = 0;
-
-  const auto start_ids = ConvertPropertyVector(std::move(req.start_id.second));
   auto dba = DbAccessor{&acc};
-
-  for (auto it = vertex_iterable.begin(); it != vertex_iterable.end(); ++it) {
-    const auto &vertex = *it;
-
-    if (start_ids <= vertex.PrimaryKey(View(req.storage_view)).GetValue()) {
-      did_reach_starting_point = true;
+  const auto emplace_scan_result = [&](const VertexAccessor &vertex) {
+    std::vector<Value> expression_results;
+    // TODO(gvolfing) it should be enough to check these only once.
+    if (vertex.Properties(View(req.storage_view)).HasError()) {
+      action_successful = false;
+      spdlog::debug("Could not retrieve properties from VertexAccessor. Transaction id: {}",
+                    req.transaction_id.logical_id);
+    }
+    if (!req.filter_expressions.empty()) {
+      // NOTE - DbAccessor might get removed in the future.
+      const bool eval = FilterOnVertex(dba, vertex, req.filter_expressions, expr::identifier_node_symbol);
+      if (!eval) {
+        return;
+      }
+    }
+    if (!req.vertex_expressions.empty()) {
+      // NOTE - DbAccessor might get removed in the future.
+      expression_results = ConvertToValueVectorFromTypedValueVector(
+          EvaluateVertexExpressions(dba, vertex, req.vertex_expressions, expr::identifier_node_symbol));
     }
 
-    if (did_reach_starting_point) {
-      std::vector<Value> expression_results;
-
-      // TODO(gvolfing) it should be enough to check these only once.
-      if (vertex.Properties(View(req.storage_view)).HasError()) {
-        action_successful = false;
-        spdlog::debug("Could not retrive properties from VertexAccessor. Transaction id: {}",
-                      req.transaction_id.logical_id);
-        break;
-      }
-      if (!req.filter_expressions.empty()) {
-        // NOTE - DbAccessor might get removed in the future.
-        const bool eval = FilterOnVertex(dba, vertex, req.filter_expressions, expr::identifier_node_symbol);
-        if (!eval) {
-          continue;
-        }
-      }
-      if (!req.vertex_expressions.empty()) {
-        expression_results = ConvertToValueVectorFromTypedValueVector(
-            EvaluateVertexExpressions(dba, vertex, req.vertex_expressions, expr::identifier_node_symbol));
-      }
-
-      std::optional<std::map<PropertyId, Value>> found_props;
+    std::optional<std::map<PropertyId, Value>> found_props;
 
+    if (req.props_to_return) {
+      found_props = CollectSpecificPropertiesFromAccessor(vertex, req.props_to_return.value(), view);
+    } else {
       const auto *schema = shard_->GetSchema(shard_->PrimaryLabel());
-      if (req.props_to_return) {
-        found_props = CollectSpecificPropertiesFromAccessor(vertex, req.props_to_return.value(), view);
-      } else {
-        found_props = CollectAllPropertiesFromAccessor(vertex, view, schema);
-      }
+      found_props = CollectAllPropertiesFromAccessor(vertex, view, schema);
+    }
+
+    // TODO(gvolfing) -VERIFY-
+    // Vertex is separated from the properties in the response.
+    // Is it useful to return just a vertex without the properties?
+    if (!found_props) {
+      action_successful = false;
+    }
+
+    results.emplace_back(msgs::ScanResultRow{.vertex = ConstructValueVertex(vertex, view).vertex_v,
+                                             .props = FromMap(found_props.value()),
+                                             .evaluated_vertex_expressions = std::move(expression_results)});
+  };
+
+  const auto start_id = ConvertPropertyVector(std::move(req.start_id.second));
+  uint64_t sample_counter{0};
+  auto vertex_iterable = acc.Vertices(view);
+  if (!req.order_bys.empty()) {
+    const auto ordered = OrderByElements(acc, dba, vertex_iterable, req.order_bys);
+    // we are traversing Elements
+    auto it = GetStartOrderedElementsIterator(ordered, start_id, View(req.storage_view));
+    for (; it != ordered.end(); ++it) {
+      emplace_scan_result(it->vertex_acc);
+      ++sample_counter;
+      if (req.batch_limit && sample_counter == req.batch_limit) {
+        // Reached the maximum specified batch size.
+        // Get the next element before exiting.
+        ++it;
+        if (it != ordered.end()) {
+          const auto &next_vertex = it->vertex_acc;
+          next_start_id = ConstructValueVertex(next_vertex, view).vertex_v.id;
+        }
 
-      // TODO(gvolfing) -VERIFY-
-      // Vertex is seperated from the properties in the response.
-      // Is it useful to return just a vertex without the properties?
-      if (!found_props) {
-        action_successful = false;
         break;
       }
-
-      results.emplace_back(msgs::ScanResultRow{.vertex = ConstructValueVertex(vertex, view).vertex_v,
-                                               .props = FromMap(found_props.value()),
-                                               .evaluated_vertex_expressions = std::move(expression_results)});
+    }
+  } else {
+    // We are going through VerticesIterable::Iterator
+    auto it = GetStartVertexIterator(vertex_iterable, start_id, View(req.storage_view));
+    for (; it != vertex_iterable.end(); ++it) {
+      emplace_scan_result(*it);
 
       ++sample_counter;
       if (req.batch_limit && sample_counter == req.batch_limit) {
@@ -1059,4 +905,4 @@ msgs::ReadResponses ShardRsm::HandleRead(msgs::GetPropertiesRequest && /*req*/)
   return msgs::GetPropertiesResponse{};
 }
 
-}  //    namespace memgraph::storage::v3
+}  // namespace memgraph::storage::v3
diff --git a/tests/simulation/shard_rsm.cpp b/tests/simulation/shard_rsm.cpp
index aec26f6c1..d5f8f2775 100644
--- a/tests/simulation/shard_rsm.cpp
+++ b/tests/simulation/shard_rsm.cpp
@@ -25,6 +25,7 @@
 #include "io/simulator/simulator_transport.hpp"
 #include "query/v2/requests.hpp"
 #include "storage/v3/id_types.hpp"
+#include "storage/v3/key_store.hpp"
 #include "storage/v3/property_value.hpp"
 #include "storage/v3/shard.hpp"
 #include "storage/v3/shard_rsm.hpp"
@@ -414,7 +415,7 @@ std::tuple<size_t, std::optional<msgs::VertexId>> AttemptToScanAllWithExpression
                                                                                  msgs::VertexId start_id,
                                                                                  uint64_t batch_limit,
                                                                                  uint64_t prop_val_to_check_against) {
-  std::string filter_expr1 = "MG_SYMBOL_NODE.property = " + std::to_string(prop_val_to_check_against);
+  std::string filter_expr1 = "MG_SYMBOL_NODE.prop1 = " + std::to_string(prop_val_to_check_against);
   std::vector<std::string> filter_expressions = {filter_expr1};
 
   std::string regular_expr1 = "2+2";
@@ -445,6 +446,72 @@ std::tuple<size_t, std::optional<msgs::VertexId>> AttemptToScanAllWithExpression
   }
 }
 
+void AttemptToScanAllWithOrderByOnPrimaryProperty(ShardClient &client, msgs::VertexId start_id, uint64_t batch_limit) {
+  msgs::ScanVerticesRequest scan_req;
+  scan_req.batch_limit = batch_limit;
+  scan_req.order_bys = {{msgs::Expression{"MG_SYMBOL_NODE.prop1"}, msgs::OrderingDirection::DESCENDING}};
+  scan_req.props_to_return = std::nullopt;
+  scan_req.start_id = start_id;
+  scan_req.storage_view = msgs::StorageView::NEW;
+  scan_req.transaction_id.logical_id = GetTransactionId();
+
+  while (true) {
+    auto read_res = client.SendReadRequest(scan_req);
+    if (read_res.HasError()) {
+      continue;
+    }
+
+    auto write_response_result = read_res.GetValue();
+    auto write_response = std::get<msgs::ScanVerticesResponse>(write_response_result);
+
+    MG_ASSERT(write_response.success);
+    MG_ASSERT(write_response.results.size() == 5, "Expecting 5 results!");
+    for (int64_t i{0}; i < 5; ++i) {
+      const auto expected_primary_key = std::vector{msgs::Value(1023 - i)};
+      const auto actual_primary_key = write_response.results[i].vertex.id.second;
+      MG_ASSERT(expected_primary_key == actual_primary_key, "The order of vertices is not correct");
+    }
+    break;
+  }
+}
+
+void AttemptToScanAllWithOrderByOnSecondaryProperty(ShardClient &client, msgs::VertexId start_id,
+                                                    uint64_t batch_limit) {
+  msgs::ScanVerticesRequest scan_req;
+  scan_req.batch_limit = batch_limit;
+  scan_req.order_bys = {{msgs::Expression{"MG_SYMBOL_NODE.prop4"}, msgs::OrderingDirection::DESCENDING}};
+  scan_req.props_to_return = std::nullopt;
+  scan_req.start_id = start_id;
+  scan_req.storage_view = msgs::StorageView::NEW;
+  scan_req.transaction_id.logical_id = GetTransactionId();
+
+  while (true) {
+    auto read_res = client.SendReadRequest(scan_req);
+    if (read_res.HasError()) {
+      continue;
+    }
+
+    auto write_response_result = read_res.GetValue();
+    auto write_response = std::get<msgs::ScanVerticesResponse>(write_response_result);
+
+    MG_ASSERT(write_response.success);
+    MG_ASSERT(write_response.results.size() == 5, "Expecting 5 results!");
+    for (int64_t i{0}; i < 5; ++i) {
+      const auto expected_prop4 = std::vector{msgs::Value(1023 - i)};
+      const auto actual_prop4 = std::invoke([&write_response, i]() {
+        const auto res = std::ranges::find_if(write_response.results[i].props, [](const auto &id_value_prop_pair) {
+          return id_value_prop_pair.first.AsInt() == 4;
+        });
+        MG_ASSERT(res != write_response.results[i].props.end(), "Property does not exist!");
+        return std::vector{res->second};
+      });
+
+      MG_ASSERT(expected_prop4 == actual_prop4, "The order of vertices is not correct");
+    }
+    break;
+  }
+}
+
 void AttemptToExpandOneWithWrongEdgeType(ShardClient &client, uint64_t src_vertex_val, EdgeTypeId edge_type_id) {
   // Source vertex
   msgs::Label label = {.id = get_primary_label()};
@@ -733,7 +800,7 @@ void AttemptToExpandOneWithSpecifiedEdgeProperties(ShardClient &client, uint64_t
 
 void AttemptToExpandOneWithFilters(ShardClient &client, uint64_t src_vertex_val, EdgeTypeId edge_type_id,
                                    uint64_t edge_prop_id, uint64_t prop_val_to_check_against) {
-  std::string filter_expr1 = "MG_SYMBOL_NODE.property = " + std::to_string(prop_val_to_check_against);
+  std::string filter_expr1 = "MG_SYMBOL_NODE.prop1 = " + std::to_string(prop_val_to_check_against);
 
   // Source vertex
   msgs::Label label = {.id = get_primary_label()};
@@ -882,6 +949,9 @@ void TestScanAllOneGo(ShardClient &client) {
   auto [result_size_2, next_id_2] = AttemptToScanAllWithExpression(client, v_id, 5, unique_prop_val_2);
   MG_ASSERT(result_size_2 == 1);
 
+  AttemptToScanAllWithOrderByOnPrimaryProperty(client, v_id, 5);
+  AttemptToScanAllWithOrderByOnSecondaryProperty(client, v_id, 5);
+
   auto [result_size_with_batch, next_id_with_batch] = AttemptToScanAllWithBatchLimit(client, v_id, 5);
   auto [result_size_without_batch, next_id_without_batch] = AttemptToScanAllWithoutBatchLimit(client, v_id);
 
@@ -1033,9 +1103,9 @@ int TestMessages() {
   auto shard_ptr2 = std::make_unique<Shard>(get_primary_label(), min_prim_key, max_prim_key, schema_prop);
   auto shard_ptr3 = std::make_unique<Shard>(get_primary_label(), min_prim_key, max_prim_key, schema_prop);
 
-  shard_ptr1->StoreMapping({{1, "label"}, {2, "property"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
-  shard_ptr2->StoreMapping({{1, "label"}, {2, "property"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
-  shard_ptr3->StoreMapping({{1, "label"}, {2, "property"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
+  shard_ptr1->StoreMapping({{1, "label"}, {2, "prop1"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
+  shard_ptr2->StoreMapping({{1, "label"}, {2, "prop1"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
+  shard_ptr3->StoreMapping({{1, "label"}, {2, "prop1"}, {3, "label1"}, {4, "prop2"}, {5, "prop3"}, {6, "prop4"}});
 
   std::vector<Address> address_for_1{shard_server_2_address, shard_server_3_address};
   std::vector<Address> address_for_2{shard_server_1_address, shard_server_3_address};
diff --git a/tools/gdb-plugins/pretty_printers.py b/tools/gdb-plugins/pretty_printers.py
index af75a9724..db0222668 100644
--- a/tools/gdb-plugins/pretty_printers.py
+++ b/tools/gdb-plugins/pretty_printers.py
@@ -3,43 +3,187 @@ import gdb.printing
 
 
 def build_memgraph_pretty_printers():
-    '''Instantiate and return all memgraph pretty printer classes.'''
-    pp = gdb.printing.RegexpCollectionPrettyPrinter('memgraph')
-    pp.add_printer('memgraph::query::TypedValue', '^memgraph::query::TypedValue$', TypedValuePrinter)
+    """Instantiate and return all memgraph pretty printer classes."""
+    pp = gdb.printing.RegexpCollectionPrettyPrinter("memgraph")
+    pp.add_printer("memgraph::query::TypedValue", "^memgraph::query::TypedValue$", TypedValuePrinter)
+    pp.add_printer("memgraph::query::v2::TypedValue", "^memgraph::query::v2::TypedValue$", TypedValuePrinter2)
+    pp.add_printer("memgraph::storage::v3::TypedValue", "^memgraph::storage::v3::TypedValue$", TypedValuePrinter3)
+    pp.add_printer(
+        "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>",
+        "^memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>$",
+        TypedValuePrinter4,
+    )
     return pp
 
 
 class TypedValuePrinter(gdb.printing.PrettyPrinter):
-    '''Pretty printer for memgraph::query::TypedValue'''
+    """Pretty printer for memgraph::query::TypedValue"""
+
     def __init__(self, val):
-        super(TypedValuePrinter, self).__init__('TypedValue')
+        super(TypedValuePrinter, self).__init__("TypedValue")
         self.val = val
 
     def to_string(self):
         def _to_str(val):
-            return '{%s %s}' % (value_type, self.val[val])
-        value_type = str(self.val['type_'])
-        if value_type == 'memgraph::query::TypedValue::Type::Null':
-            return '{%s}' % value_type
-        elif value_type == 'memgraph::query::TypedValue::Type::Bool':
-            return _to_str('bool_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Int':
-            return _to_str('int_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Double':
-            return _to_str('double_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::String':
-            return _to_str('string_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::List':
-            return _to_str('list_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Map':
-            return _to_str('map_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Vertex':
-            return _to_str('vertex_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Edge':
-            return _to_str('edge_v')
-        elif value_type == 'memgraph::query::TypedValue::Type::Path':
-            return _to_str('path_v')
-        return '{%s}' % value_type
+            return "{%s %s}" % (value_type, self.val[val])
 
-gdb.printing.register_pretty_printer(None, build_memgraph_pretty_printers(),
-                                     replace=True)
+        value_type = str(self.val["type_"])
+        if value_type == "memgraph::query::TypedValue::Type::Null":
+            return "{%s}" % value_type
+        elif value_type == "memgraph::query::TypedValue::Type::Bool":
+            return _to_str("bool_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Int":
+            return _to_str("int_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Double":
+            return _to_str("double_v")
+        elif value_type == "memgraph::query::TypedValue::Type::String":
+            return _to_str("string_v")
+        elif value_type == "memgraph::query::TypedValue::Type::List":
+            return _to_str("list_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Map":
+            return _to_str("map_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Vertex":
+            return _to_str("vertex_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Edge":
+            return _to_str("edge_v")
+        elif value_type == "memgraph::query::TypedValue::Type::Path":
+            return _to_str("path_v")
+        return "{%s}" % value_type
+
+
+class TypedValuePrinter2(gdb.printing.PrettyPrinter):
+    """Pretty printer for memgraph::query::TypedValue"""
+
+    def __init__(self, val):
+        super(TypedValuePrinter2, self).__init__("TypedValue2")
+        self.val = val
+
+    def to_string(self):
+        def _to_str(val):
+            return "{%s %s}" % (value_type, self.val[val])
+
+        value_type = str(self.val["type_"])
+        if value_type == "memgraph::query::v2::TypedValue::Type::Null":
+            return "{%s}" % value_type
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Bool":
+            return _to_str("bool_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Int":
+            return _to_str("int_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Double":
+            return _to_str("double_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::String":
+            return _to_str("string_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::List":
+            return _to_str("list_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Map":
+            return _to_str("map_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Vertex":
+            return _to_str("vertex_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Edge":
+            return _to_str("edge_v")
+        elif value_type == "memgraph::query::v2::TypedValue::Type::Path":
+            return _to_str("path_v")
+        return "{%s}" % value_type
+
+
+class TypedValuePrinter3(gdb.printing.PrettyPrinter):
+    """Pretty printer for memgraph::query::TypedValue"""
+
+    def __init__(self, val):
+        super(TypedValuePrinter3, self).__init__("TypedValue3")
+        self.val = val
+
+    def to_string(self):
+        def _to_str(val):
+            return "{%s %s}" % (value_type, self.val[val])
+
+        value_type = str(self.val["type_"])
+        if value_type == "memgraph::storage::v3::TypedValue::Type::Null":
+            return "{%s}" % value_type
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Bool":
+            return _to_str("bool_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Int":
+            return _to_str("int_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Double":
+            return _to_str("double_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::String":
+            return _to_str("string_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::List":
+            return _to_str("list_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Map":
+            return _to_str("map_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Vertex":
+            return _to_str("vertex_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Edge":
+            return _to_str("edge_v")
+        elif value_type == "memgraph::storage::v3::TypedValue::Type::Path":
+            return _to_str("path_v")
+        return "{%s}" % value_type
+
+
+class TypedValuePrinter4(gdb.printing.PrettyPrinter):
+    """Pretty printer for memgraph::query::TypedValue"""
+
+    def __init__(self, val):
+        super(TypedValuePrinter4, self).__init__("TypedValue4")
+        self.val = val
+
+    def to_string(self):
+        def _to_str(val):
+            return "{%s %s}" % (value_type, self.val[val])
+
+        value_type = str(self.val["type_"])
+        if (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Null"
+        ):
+            return "{%s}" % value_type
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Bool"
+        ):
+            return _to_str("bool_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Int"
+        ):
+            return _to_str("int_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Double"
+        ):
+            return _to_str("double_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::String"
+        ):
+            return _to_str("string_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::List"
+        ):
+            return _to_str("list_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Map"
+        ):
+            return _to_str("map_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Vertex"
+        ):
+            return _to_str("vertex_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Edge"
+        ):
+            return _to_str("edge_v")
+        elif (
+            value_type
+            == "memgraph::expr::TypedValueT<memgraph::storage::v3::VertexAccessor, memgraph::storage::v3::EdgeAccessor, memgraph::storage::v3::Path>::Type::Path"
+        ):
+            return _to_str("path_v")
+        return "{%s}" % value_type
+
+
+gdb.printing.register_pretty_printer(None, build_memgraph_pretty_printers(), replace=True)