Replace query::GraphView with storage::View

Reviewers: mferencevic, ipaljak

Reviewed By: mferencevic

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D2356
This commit is contained in:
Teon Banek 2019-09-05 16:32:53 +02:00
parent ec22026fed
commit f02617f2bd
18 changed files with 189 additions and 196 deletions

View File

@ -85,12 +85,12 @@ clause. In the query `CREATE (n)` only one node is created, while in the query
`CreateNode` logical operator the input is either a `ScanAll` operator, or a
`Once` operator.
### GraphView
### storage::View
In the previous section, [Cypher Execution
Semantics](#cypher-execution-semantics), we mentioned how the preceding
clauses should not see changes made in subsequent ones. For that reason, some
operators take a `GraphView` enum value. This value determines which state of
operators take a `storage::View` enum value. This value determines which state of
the graph an operator sees.
Consider the query `MATCH (n)--(m) WHERE n.x = 0 SET m.x = 1`. Naive streaming
@ -101,9 +101,9 @@ is not how Cypher works. To handle this issue properly, Memgraph designed the
`VertexAccessor` class that tracks two versions of data: one that was visible
before the current transaction+command, and the optional other that was
created in the current transaction+command. The `MATCH` clause will be planned
as `ScanAll` and `Expand` operations using `GraphView::OLD` value. This will
ensure modifications performed in the same query do not affect it. The same
applies to edges and the `EdgeAccessor` class.
as `ScanAll` and `Expand` operations using `storage::View::OLD` value. This
will ensure modifications performed in the same query do not affect it. The
same applies to edges and the `EdgeAccessor` class.
### Existing Record Detection

View File

@ -82,18 +82,18 @@ bool TypedValueCompare(const TypedValue &a, const TypedValue &b) {
} // namespace impl
template <typename TAccessor>
void SwitchAccessor(TAccessor &accessor, GraphView graph_view) {
switch (graph_view) {
case GraphView::NEW:
void SwitchAccessor(TAccessor &accessor, storage::View view) {
switch (view) {
case storage::View::NEW:
accessor.SwitchNew();
break;
case GraphView::OLD:
case storage::View::OLD:
accessor.SwitchOld();
break;
}
}
template void SwitchAccessor<>(VertexAccessor &accessor, GraphView graph_view);
template void SwitchAccessor<>(EdgeAccessor &accessor, GraphView graph_view);
template void SwitchAccessor<>(VertexAccessor &accessor, storage::View view);
template void SwitchAccessor<>(EdgeAccessor &accessor, storage::View view);
} // namespace query

View File

@ -12,14 +12,10 @@
#include "query/frontend/semantic/symbol.hpp"
#include "query/typed_value.hpp"
#include "storage/common/types/types.hpp"
#include "storage/v2/view.hpp"
namespace query {
/// Indicates that some part of query execution should see the OLD graph state
/// (the latest state before the current transaction+command), or NEW (state as
/// changed by the current transaction+command).
enum class GraphView { OLD, NEW };
/// Recursively reconstruct all the accessors in the given TypedValue.
///
/// @throw ReconstructionException if any reconstruction failed.
@ -72,7 +68,7 @@ class TypedValueVectorCompare final {
/// Switch the given [Vertex/Edge]Accessor to the desired state.
template <class TAccessor>
void SwitchAccessor(TAccessor &accessor, GraphView graph_view);
void SwitchAccessor(TAccessor &accessor, storage::View view);
/// Raise QueryRuntimeException if the value for symbol isn't of expected type.
inline void ExpectType(const Symbol &symbol, const TypedValue &value,

View File

@ -23,12 +23,12 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
public:
ExpressionEvaluator(Frame *frame, const SymbolTable &symbol_table,
const EvaluationContext &ctx,
database::GraphDbAccessor *dba, GraphView graph_view)
database::GraphDbAccessor *dba, storage::View view)
: frame_(frame),
symbol_table_(&symbol_table),
ctx_(&ctx),
dba_(dba),
graph_view_(graph_view) {}
view_(view) {}
using ExpressionVisitor<TypedValue>::Visit;
@ -552,29 +552,25 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
switch (value.type()) {
case TypedValue::Type::Vertex: {
auto &vertex = value.ValueVertex();
switch (graph_view_) {
case GraphView::NEW:
switch (view_) {
case storage::View::NEW:
vertex.SwitchNew();
break;
case GraphView::OLD:
case storage::View::OLD:
vertex.SwitchOld();
break;
default:
LOG(FATAL) << "Unhandled GraphView enum";
}
break;
}
case TypedValue::Type::Edge: {
auto &edge = value.ValueEdge();
switch (graph_view_) {
case GraphView::NEW:
switch (view_) {
case storage::View::NEW:
edge.SwitchNew();
break;
case GraphView::OLD:
case storage::View::OLD:
edge.SwitchOld();
break;
default:
LOG(FATAL) << "Unhandled GraphView enum";
}
break;
}
@ -589,15 +585,13 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
break;
}
case TypedValue::Type::Path:
switch (graph_view_) {
case GraphView::NEW:
switch (view_) {
case storage::View::NEW:
value.ValuePath().SwitchNew();
break;
case GraphView::OLD:
case storage::View::OLD:
value.ValuePath().SwitchOld();
break;
default:
LOG(FATAL) << "Unhandled GraphView enum";
}
case TypedValue::Type::Null:
case TypedValue::Type::Bool:
@ -613,7 +607,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
const EvaluationContext *ctx_;
database::GraphDbAccessor *dba_;
// which switching approach should be used when evaluating
const GraphView graph_view_;
storage::View view_;
};
/// A helper function for evaluating an expression that's an int.

View File

@ -135,7 +135,7 @@ Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
.count();
evaluation_context.parameters = parameters;
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
db_accessor, storage::View::OLD);
AuthQuery::Action action = auth_query->action_;
std::string username = auth_query->user_;
@ -431,7 +431,7 @@ Callback HandleStreamQuery(StreamQuery *stream_query,
.count();
evaluation_context.parameters = parameters;
ExpressionEvaluator eval(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
db_accessor, storage::View::OLD);
std::string stream_name = stream_query->stream_name_;
auto stream_uri =

View File

@ -128,7 +128,7 @@ VertexAccessor &CreateLocalVertex(const NodeCreationInfo &node_info,
// setting properties on new nodes.
ExpressionEvaluator evaluator(frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
// TODO: PropsSetChecked allocates a PropertyValue, make it use context.memory
// when we update PropertyValue with custom allocator.
for (auto &kv : node_info.properties)
@ -227,7 +227,7 @@ bool CreateExpand::CreateExpandCursor::Pull(Frame &frame,
// latest accesors.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
// E.g. we pickup new properties: `CREATE (n {p: 42}) -[:r {ep: n.p}]-> ()`
v1.SwitchNew();
@ -321,17 +321,17 @@ class ScanAllCursor : public Cursor {
};
ScanAll::ScanAll(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, GraphView graph_view)
Symbol output_symbol, storage::View view)
: input_(input ? input : std::make_shared<Once>()),
output_symbol_(output_symbol),
graph_view_(graph_view) {}
view_(view) {}
ACCEPT_WITH_INPUT(ScanAll)
UniqueCursorPtr ScanAll::MakeCursor(utils::MemoryResource *mem) const {
auto vertices = [this](Frame &, ExecutionContext &context) {
auto *db = context.db_accessor;
return std::make_optional(db->Vertices(graph_view_ == GraphView::NEW));
return std::make_optional(db->Vertices(view_ == storage::View::NEW));
};
return MakeUniqueCursorPtr<ScanAllCursor<decltype(vertices)>>(
mem, output_symbol_, input_->MakeCursor(mem), std::move(vertices));
@ -345,8 +345,8 @@ std::vector<Symbol> ScanAll::ModifiedSymbols(const SymbolTable &table) const {
ScanAllByLabel::ScanAllByLabel(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, storage::Label label,
GraphView graph_view)
: ScanAll(input, output_symbol, graph_view), label_(label) {}
storage::View view)
: ScanAll(input, output_symbol, view), label_(label) {}
ACCEPT_WITH_INPUT(ScanAllByLabel)
@ -354,7 +354,7 @@ UniqueCursorPtr ScanAllByLabel::MakeCursor(utils::MemoryResource *mem) const {
auto vertices = [this](Frame &, ExecutionContext &context) {
auto *db = context.db_accessor;
return std::make_optional(
db->Vertices(label_, graph_view_ == GraphView::NEW));
db->Vertices(label_, view_ == storage::View::NEW));
};
return MakeUniqueCursorPtr<ScanAllCursor<decltype(vertices)>>(
mem, output_symbol_, input_->MakeCursor(mem), std::move(vertices));
@ -364,8 +364,8 @@ ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
storage::Label label, storage::Property property,
const std::string &property_name, std::optional<Bound> lower_bound,
std::optional<Bound> upper_bound, GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
std::optional<Bound> upper_bound, storage::View view)
: ScanAll(input, output_symbol, view),
label_(label),
property_(property),
property_name_(property_name),
@ -384,7 +384,7 @@ UniqueCursorPtr ScanAllByLabelPropertyRange::MakeCursor(
auto *db = context.db_accessor;
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, graph_view_);
context.db_accessor, view_);
auto convert =
[&evaluator](
const auto &bound) -> std::optional<utils::Bound<PropertyValue>> {
@ -406,7 +406,7 @@ UniqueCursorPtr ScanAllByLabelPropertyRange::MakeCursor(
if (maybe_upper && maybe_upper->value().IsNull()) return std::nullopt;
return std::make_optional(db->Vertices(label_, property_, maybe_lower,
maybe_upper,
graph_view_ == GraphView::NEW));
view_ == storage::View::NEW));
};
return MakeUniqueCursorPtr<ScanAllCursor<decltype(vertices)>>(
mem, output_symbol_, input_->MakeCursor(mem), std::move(vertices));
@ -416,8 +416,8 @@ ScanAllByLabelPropertyValue::ScanAllByLabelPropertyValue(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
storage::Label label, storage::Property property,
const std::string &property_name, Expression *expression,
GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
storage::View view)
: ScanAll(input, output_symbol, view),
label_(label),
property_(property),
property_name_(property_name),
@ -435,7 +435,7 @@ UniqueCursorPtr ScanAllByLabelPropertyValue::MakeCursor(
auto *db = context.db_accessor;
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, graph_view_);
context.db_accessor, view_);
auto value = expression_->Accept(evaluator);
if (value.IsNull()) return std::nullopt;
if (!value.IsPropertyValue()) {
@ -444,7 +444,7 @@ UniqueCursorPtr ScanAllByLabelPropertyValue::MakeCursor(
}
return std::make_optional(db->Vertices(label_, property_,
PropertyValue(value),
graph_view_ == GraphView::NEW));
view_ == storage::View::NEW));
};
return MakeUniqueCursorPtr<ScanAllCursor<decltype(vertices)>>(
mem, output_symbol_, input_->MakeCursor(mem), std::move(vertices));
@ -464,11 +464,11 @@ Expand::Expand(const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction,
const std::vector<storage::EdgeType> &edge_types,
bool existing_node, GraphView graph_view)
bool existing_node, storage::View view)
: input_(input ? input : std::make_shared<Once>()),
input_symbol_(input_symbol),
common_{node_symbol, edge_symbol, direction, edge_types, existing_node},
graph_view_(graph_view) {}
view_(view) {}
ACCEPT_WITH_INPUT(Expand)
@ -560,7 +560,7 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame, ExecutionContext &context) {
ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
auto &vertex = vertex_value.ValueVertex();
SwitchAccessor(vertex, self_.graph_view_);
SwitchAccessor(vertex, self_.view_);
auto direction = self_.common_.direction;
if (direction == EdgeAtom::Direction::IN ||
@ -707,7 +707,7 @@ class ExpandVariableCursor : public Cursor {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
while (true) {
if (Expand(frame, context)) return true;
@ -779,12 +779,12 @@ class ExpandVariableCursor : public Cursor {
ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
auto &vertex = vertex_value.ValueVertex();
SwitchAccessor(vertex, GraphView::OLD);
SwitchAccessor(vertex, storage::View::OLD);
// Evaluate the upper and lower bounds.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
auto calc_bound = [&evaluator](auto &bound) {
auto value = EvaluateInt(&evaluator, bound, "Variable expansion bound");
if (value < 0)
@ -798,7 +798,7 @@ class ExpandVariableCursor : public Cursor {
: std::numeric_limits<int64_t>::max();
if (upper_bound_ > 0) {
SwitchAccessor(vertex, GraphView::OLD);
SwitchAccessor(vertex, storage::View::OLD);
auto *memory = edges_.get_allocator().GetMemoryResource();
edges_.emplace_back(ExpandFromVertex(vertex, self_.common_.direction,
self_.common_.edge_types, memory));
@ -847,7 +847,7 @@ class ExpandVariableCursor : public Cursor {
bool Expand(Frame &frame, ExecutionContext &context) {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
// Some expansions might not be valid due to edge uniqueness and
// existing_node criterions, so expand in a loop until either the input
// vertex is exhausted or a valid variable-length expansion is available.
@ -916,7 +916,7 @@ class ExpandVariableCursor : public Cursor {
// we are doing depth-first search, so place the current
// edge's expansions onto the stack, if we should continue to expand
if (upper_bound_ > static_cast<int64_t>(edges_.size())) {
SwitchAccessor(current_vertex, GraphView::OLD);
SwitchAccessor(current_vertex, storage::View::OLD);
auto *memory = edges_.get_allocator().GetMemoryResource();
edges_.emplace_back(ExpandFromVertex(current_vertex,
self_.common_.direction,
@ -948,7 +948,7 @@ class STShortestPathCursor : public query::plan::Cursor {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
while (input_cursor_->Pull(frame, context)) {
const auto &source_tv = frame[self_.input_symbol_];
const auto &sink_tv = frame[self_.common_.node_symbol];
@ -1191,7 +1191,7 @@ class SingleSourceShortestPathCursor : public query::plan::Cursor {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
// for the given (edge, vertex) pair checks if they satisfy the
// "where" condition. if so, places them in the to_visit_ structure.
@ -1352,7 +1352,7 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
auto create_state = [this](VertexAccessor vertex, int depth) {
return std::make_pair(vertex, upper_bound_set_ ? depth : 0);
};
@ -1363,8 +1363,8 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
auto expand_pair = [this, &evaluator, &frame, &create_state](
EdgeAccessor edge, VertexAccessor vertex,
double weight, int depth) {
SwitchAccessor(edge, GraphView::OLD);
SwitchAccessor(vertex, GraphView::OLD);
SwitchAccessor(edge, storage::View::OLD);
SwitchAccessor(vertex, storage::View::OLD);
auto *memory = evaluator.GetMemoryResource();
if (self_.filter_lambda_.expression) {
@ -1428,7 +1428,7 @@ class ExpandWeightedShortestPathCursor : public query::plan::Cursor {
// Skip expansion for such nodes.
if (node.IsNull()) continue;
}
SwitchAccessor(vertex, GraphView::OLD);
SwitchAccessor(vertex, storage::View::OLD);
if (self_.upper_bound_) {
upper_bound_ =
EvaluateInt(&evaluator, self_.upper_bound_,
@ -1727,7 +1727,7 @@ bool Filter::FilterCursor::Pull(Frame &frame, ExecutionContext &context) {
// nodes and edges.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::OLD);
storage::View::OLD);
while (input_cursor_->Pull(frame, context)) {
if (EvaluateFilter(evaluator, self_.expression_)) return true;
}
@ -1773,7 +1773,7 @@ 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.db_accessor, GraphView::NEW);
context.db_accessor, storage::View::NEW);
for (auto named_expr : self_.named_expressions_)
named_expr->Accept(evaluator);
@ -1814,7 +1814,7 @@ bool Delete::DeleteCursor::Pull(Frame &frame, ExecutionContext &context) {
// delete newly added nodes and edges.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
auto *pull_memory = context.evaluation_context.memory;
// collect expressions results so edges can get deleted before vertices
// this is necessary because an edge that gets deleted could block vertex
@ -1895,7 +1895,7 @@ bool SetProperty::SetPropertyCursor::Pull(Frame &frame,
// Set, just like Create needs to see the latest changes.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
TypedValue lhs = self_.lhs_->expression_->Accept(evaluator);
TypedValue rhs = self_.rhs_->Accept(evaluator);
@ -2007,7 +2007,7 @@ bool SetProperties::SetPropertiesCursor::Pull(Frame &frame,
// Set, just like Create needs to see the latest changes.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
TypedValue rhs = self_.rhs_->Accept(evaluator);
switch (lhs.type()) {
@ -2106,7 +2106,7 @@ bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
// Remove, just like Delete needs to see the latest changes.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context, context.db_accessor,
GraphView::NEW);
storage::View::NEW);
TypedValue lhs = self_.lhs_->expression_->Accept(evaluator);
switch (lhs.type()) {
@ -2470,7 +2470,7 @@ class AggregateCursor : public Cursor {
void ProcessAll(Frame *frame, ExecutionContext *context) {
ExpressionEvaluator evaluator(frame, context->symbol_table,
context->evaluation_context,
context->db_accessor, GraphView::NEW);
context->db_accessor, storage::View::NEW);
while (input_cursor_->Pull(*frame, *context)) {
ProcessOne(*frame, &evaluator);
}
@ -2703,7 +2703,7 @@ bool Skip::SkipCursor::Pull(Frame &frame, ExecutionContext &context) {
// parameter is not important.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
TypedValue to_skip = self_.expression_->Accept(evaluator);
if (to_skip.type() != TypedValue::Type::Int)
throw QueryRuntimeException(
@ -2764,7 +2764,7 @@ bool Limit::LimitCursor::Pull(Frame &frame, ExecutionContext &context) {
// important.
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
TypedValue limit = self_.expression_->Accept(evaluator);
if (limit.type() != TypedValue::Type::Int)
throw QueryRuntimeException(
@ -2830,7 +2830,7 @@ class OrderByCursor : public Cursor {
if (!did_pull_all_) {
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
auto *mem = cache_.get_allocator().GetMemoryResource();
while (input_cursor_->Pull(frame, context)) {
// collect the order_by elements
@ -3102,7 +3102,7 @@ class UnwindCursor : public Cursor {
// successful pull from input, initialize value and iterator
ExpressionEvaluator evaluator(&frame, context.symbol_table,
context.evaluation_context,
context.db_accessor, GraphView::OLD);
context.db_accessor, storage::View::OLD);
TypedValue input_value = self_.input_expression_->Accept(evaluator);
if (input_value.type() != TypedValue::Type::List)
throw QueryRuntimeException(

View File

@ -519,14 +519,14 @@ chained in cases when longer paths need creating.
:slk-save #'slk-save-operator-pointer
:slk-load #'slk-load-operator-pointer)
(output-symbol "Symbol" :scope :public)
(graph-view "GraphView" :scope :public
:documentation
"Controls which graph state is used to produce vertices.
(view "::storage::View" :scope :public
:documentation
"Controls which graph state is used to produce vertices.
If @c GraphView::OLD, @c ScanAll will produce vertices visible in the previous
graph state, before modifications done by current transaction & command. With
@c GraphView::NEW, all vertices will be produced the current transaction sees
along with their modifications."))
If @c storage::View::OLD, @c ScanAll will produce vertices visible in the
previous graph state, before modifications done by current transaction &
command. With @c storage::View::NEW, all vertices will be produced the current
transaction sees along with their modifications."))
(:documentation
"Operator which iterates over all the nodes currently in the database.
@ -547,7 +547,7 @@ with a constructor argument.
#>cpp
ScanAll() {}
ScanAll(const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
GraphView graph_view = GraphView::OLD);
storage::View view = storage::View::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override;
@ -575,7 +575,7 @@ given label.
ScanAllByLabel() {}
ScanAllByLabel(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, storage::Label label,
GraphView graph_view = GraphView::OLD);
storage::View view = storage::View::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override;
cpp<#)
@ -675,7 +675,7 @@ property value which is inside a range (inclusive or exlusive).
* @param property Property from which the value will be looked up from.
* @param lower_bound Optional lower @c Bound.
* @param upper_bound Optional upper @c Bound.
* @param graph_view GraphView used when obtaining vertices.
* @param view storage::View used when obtaining vertices.
*/
ScanAllByLabelPropertyRange(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, storage::Label label,
@ -683,7 +683,7 @@ property value which is inside a range (inclusive or exlusive).
const std::string &property_name,
std::optional<Bound> lower_bound,
std::optional<Bound> upper_bound,
GraphView graph_view = GraphView::OLD);
storage::View view = storage::View::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override;
@ -716,14 +716,14 @@ property value.
* @param label Label which the vertex must have.
* @param property Property from which the value will be looked up from.
* @param expression Expression producing the value of the vertex property.
* @param graph_view GraphView used when obtaining vertices.
* @param view storage::View used when obtaining vertices.
*/
ScanAllByLabelPropertyValue(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, storage::Label label,
storage::Property property,
const std::string &property_name,
Expression *expression,
GraphView graph_view = GraphView::OLD);
storage::View view = storage::View::OLD);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
UniqueCursorPtr MakeCursor(utils::MemoryResource *) const override;
@ -757,9 +757,9 @@ that has already been expanded and should be just validated in the frame."))
:slk-load #'slk-load-operator-pointer)
(input-symbol "Symbol" :scope :public)
(common "ExpandCommon" :scope :public)
(graph-view "GraphView" :scope :public
:documentation
"State from which the input node should get expanded."))
(view "::storage::View" :scope :public
:documentation
"State from which the input node should get expanded."))
(:documentation
"Expansion operator. For a node existing in the frame it
expands one edge and one node and places them on the frame.
@ -786,7 +786,7 @@ pulled.")
Expand(const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Direction direction,
const std::vector<storage::EdgeType> &edge_types, bool existing_node,
GraphView graph_view);
storage::View view);
Expand() {}

View File

@ -90,8 +90,7 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
if (expand.common_.existing_node) {
return true;
}
ScanAll dst_scan(expand.input(), expand.common_.node_symbol,
expand.graph_view_);
ScanAll dst_scan(expand.input(), expand.common_.node_symbol, expand.view_);
auto indexed_scan =
GenScanByIndex(dst_scan, FLAGS_query_vertex_count_to_expand_existing);
if (indexed_scan) {
@ -499,7 +498,7 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
const std::optional<int64_t> &max_vertex_count = std::nullopt) {
const auto &input = scan.input();
const auto &node_symbol = scan.output_symbol_;
const auto &graph_view = scan.graph_view_;
const auto &view = scan.view_;
const auto labels = filters_.FilteredLabels(node_symbol);
if (labels.empty()) {
// Without labels, we cannot generate any indexed ScanAll.
@ -532,7 +531,7 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
return std::make_unique<ScanAllByLabelPropertyRange>(
input, node_symbol, GetLabel(found_index->label),
GetProperty(prop_filter.property_), prop_filter.property_.name,
prop_filter.lower_bound_, prop_filter.upper_bound_, graph_view);
prop_filter.lower_bound_, prop_filter.upper_bound_, view);
} else if (prop_filter.type_ == PropertyFilter::Type::REGEX_MATCH) {
// Generate index scan using the empty string as a lower bound.
Expression *empty_string = ast_storage_->Create<PrimitiveLiteral>("");
@ -540,14 +539,14 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
return std::make_unique<ScanAllByLabelPropertyRange>(
input, node_symbol, GetLabel(found_index->label),
GetProperty(prop_filter.property_), prop_filter.property_.name,
std::make_optional(lower_bound), std::nullopt, graph_view);
std::make_optional(lower_bound), std::nullopt, view);
} else {
CHECK(prop_filter.value_) << "Property filter should either have "
"bounds or a value expression.";
return std::make_unique<ScanAllByLabelPropertyValue>(
input, node_symbol, GetLabel(found_index->label),
GetProperty(prop_filter.property_), prop_filter.property_.name,
prop_filter.value_, graph_view);
prop_filter.value_, view);
}
}
auto label = FindBestLabelIndex(labels);
@ -562,7 +561,7 @@ class IndexLookupRewriter final : public HierarchicalLogicalOperatorVisitor {
filter_exprs_for_removal_.insert(removed_expressions.begin(),
removed_expressions.end());
return std::make_unique<ScanAllByLabel>(input, node_symbol, GetLabel(label),
graph_view);
view);
}
};

View File

@ -51,7 +51,7 @@ struct MatchContext {
// to during generation.
std::unordered_set<Symbol> &bound_symbols;
// Determines whether the match should see the new graph state or not.
GraphView graph_view = GraphView::OLD;
storage::View view = storage::View::OLD;
// All the newly established symbols in match.
std::vector<Symbol> new_symbols{};
};
@ -388,7 +388,7 @@ class RuleBasedPlanner {
if (bound_symbols.insert(node1_symbol).second) {
// We have just bound this symbol, so generate ScanAll which fills it.
last_op = std::make_unique<ScanAll>(std::move(last_op), node1_symbol,
match_context.graph_view);
match_context.view);
match_context.new_symbols.emplace_back(node1_symbol);
last_op = impl::GenFilters(std::move(last_op), bound_symbols, filters,
storage);
@ -470,8 +470,9 @@ class RuleBasedPlanner {
}
// TODO: Pass weight lambda.
CHECK(match_context.graph_view == GraphView::OLD)
<< "ExpandVariable should only be planned with GraphView::OLD";
CHECK(match_context.view == storage::View::OLD)
<< "ExpandVariable should only be planned with "
"storage::View::OLD";
last_op = std::make_unique<ExpandVariable>(
std::move(last_op), node1_symbol, node_symbol, edge_symbol,
edge->type_, expansion.direction, edge_types,
@ -481,7 +482,7 @@ class RuleBasedPlanner {
last_op = std::make_unique<Expand>(
std::move(last_op), node1_symbol, node_symbol, edge_symbol,
expansion.direction, edge_types, existing_node,
match_context.graph_view);
match_context.view);
}
// Bind the expanded edge and node.
@ -535,7 +536,7 @@ class RuleBasedPlanner {
// version when generating the create part.
std::unordered_set<Symbol> bound_symbols_copy(context_->bound_symbols);
MatchContext match_ctx{matching, *context_->symbol_table,
bound_symbols_copy, GraphView::NEW};
bound_symbols_copy, storage::View::NEW};
auto on_match = PlanMatching(match_ctx, nullptr);
// Use the original bound_symbols, so we fill it with new symbols.
auto on_create =

View File

@ -2,6 +2,7 @@
namespace storage {
/// Indicator for obtaining the state before or after a transaction & command.
enum class View {
OLD,
NEW,

View File

@ -37,7 +37,7 @@ static void MapLiteral(benchmark::State &state) {
evaluation_context.properties =
query::NamesToProperties(ast.properties_, &dba);
query::ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
&dba, query::GraphView::NEW);
&dba, storage::View::NEW);
while (state.KeepRunning()) {
benchmark::DoNotOptimize(expr->Accept(evaluator));
}
@ -68,7 +68,7 @@ static void AdditionOperator(benchmark::State &state) {
}
query::EvaluationContext evaluation_context{memory.get()};
query::ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
&dba, query::GraphView::NEW);
&dba, storage::View::NEW);
while (state.KeepRunning()) {
benchmark::DoNotOptimize(expr->Accept(evaluator));
}

View File

@ -242,7 +242,7 @@ TEST_F(PrintToJsonTest, Expand) {
EdgeAtom::Direction::BOTH,
std::vector<storage::EdgeType>{dba.EdgeType("EdgeType1"),
dba.EdgeType("EdgeType2")},
false, GraphView::OLD);
false, storage::View::OLD);
Check(last_op.get(), R"(
{
@ -348,10 +348,10 @@ TEST_F(PrintToJsonTest, ConstructNamedPath) {
std::make_shared<ScanAll>(nullptr, node1_sym);
last_op = std::make_shared<Expand>(
last_op, node1_sym, node2_sym, edge1_sym, EdgeAtom::Direction::OUT,
std::vector<storage::EdgeType>{}, false, GraphView::OLD);
std::vector<storage::EdgeType>{}, false, storage::View::OLD);
last_op = std::make_shared<Expand>(
last_op, node2_sym, node3_sym, edge2_sym, EdgeAtom::Direction::OUT,
std::vector<storage::EdgeType>{}, false, GraphView::OLD);
std::vector<storage::EdgeType>{}, false, storage::View::OLD);
last_op = std::make_shared<ConstructNamedPath>(
last_op, GetSymbol("path"),
std::vector<Symbol>{node1_sym, edge1_sym, node2_sym, edge2_sym,
@ -436,7 +436,7 @@ TEST_F(PrintToJsonTest, Delete) {
last_op = std::make_shared<Expand>(
last_op, node_sym, GetSymbol("node2"), GetSymbol("edge"),
EdgeAtom::Direction::BOTH, std::vector<storage::EdgeType>{}, false,
GraphView::OLD);
storage::View::OLD);
last_op = std::make_shared<plan::Delete>(
last_op, std::vector<Expression *>{IDENT("node2")}, true);
@ -585,11 +585,11 @@ TEST_F(PrintToJsonTest, EdgeUniquenessFilter) {
std::make_shared<ScanAll>(nullptr, node1_sym);
last_op = std::make_shared<Expand>(
last_op, node1_sym, node2_sym, edge1_sym, EdgeAtom::Direction::IN,
std::vector<storage::EdgeType>{}, false, GraphView::OLD);
std::vector<storage::EdgeType>{}, false, storage::View::OLD);
last_op = std::make_shared<ScanAll>(last_op, node3_sym);
last_op = std::make_shared<Expand>(
last_op, node3_sym, node4_sym, edge2_sym, EdgeAtom::Direction::OUT,
std::vector<storage::EdgeType>{}, false, GraphView::OLD);
std::vector<storage::EdgeType>{}, false, storage::View::OLD);
last_op = std::make_shared<EdgeUniquenessFilter>(
last_op, edge2_sym, std::vector<Symbol>{edge1_sym});
@ -816,7 +816,7 @@ TEST_F(PrintToJsonTest, Optional) {
std::shared_ptr<LogicalOperator> expand = std::make_shared<Expand>(
nullptr, node1_sym, node2_sym, edge_sym, EdgeAtom::Direction::OUT,
std::vector<storage::EdgeType>{}, false, GraphView::OLD);
std::vector<storage::EdgeType>{}, false, storage::View::OLD);
std::shared_ptr<LogicalOperator> last_op = std::make_shared<Optional>(
input, expand, std::vector<Symbol>{node2_sym, edge_sym});

View File

@ -167,7 +167,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
TEST_F(QueryCostEstimator, Expand) {
MakeOp<Expand>(last_op_, NextSymbol(), NextSymbol(), NextSymbol(),
EdgeAtom::Direction::IN, std::vector<storage::EdgeType>{},
false, GraphView::OLD);
false, storage::View::OLD);
EXPECT_COST(CardParam::kExpand * CostParam::kExpand);
}

View File

@ -38,7 +38,7 @@ class ExpressionEvaluatorTest : public ::testing::Test {
SymbolTable symbol_table;
Frame frame{128};
ExpressionEvaluator eval{&frame, symbol_table, ctx, &dba, GraphView::OLD};
ExpressionEvaluator eval{&frame, symbol_table, ctx, &dba, storage::View::OLD};
Identifier *CreateIdentifierWithValue(std::string name,
const TypedValue &value) {

View File

@ -42,9 +42,9 @@ TEST(QueryPlan, Accumulate) {
SymbolTable symbol_table;
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false,
storage::View::OLD);
auto one = LITERAL(1);
auto n_p = PROPERTY_LOOKUP(IDENT("n")->MapTo(n.sym_), prop);

View File

@ -83,11 +83,11 @@ struct ScanAllTuple {
ScanAllTuple MakeScanAll(AstStorage &storage, SymbolTable &symbol_table,
const std::string &identifier,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
storage::View view = storage::View::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
node->identifier_->MapTo(symbol);
auto logical_op = std::make_shared<ScanAll>(input, symbol, graph_view);
auto logical_op = std::make_shared<ScanAll>(input, symbol, view);
return ScanAllTuple{node, logical_op, symbol};
}
@ -101,12 +101,12 @@ ScanAllTuple MakeScanAllByLabel(
AstStorage &storage, SymbolTable &symbol_table,
const std::string &identifier, storage::Label label,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
storage::View view = storage::View::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
node->identifier_->MapTo(symbol);
auto logical_op =
std::make_shared<ScanAllByLabel>(input, symbol, label, graph_view);
std::make_shared<ScanAllByLabel>(input, symbol, label, view);
return ScanAllTuple{node, logical_op, symbol};
}
@ -122,13 +122,13 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
const std::string &property_name, std::optional<Bound> lower_bound,
std::optional<Bound> upper_bound,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
storage::View view = storage::View::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
node->identifier_->MapTo(symbol);
auto logical_op = std::make_shared<ScanAllByLabelPropertyRange>(
input, symbol, label, property, property_name, lower_bound, upper_bound,
graph_view);
view);
return ScanAllTuple{node, logical_op, symbol};
}
@ -143,12 +143,12 @@ ScanAllTuple MakeScanAllByLabelPropertyValue(
storage::Label label, storage::Property property,
const std::string &property_name, Expression *value,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
storage::View view = storage::View::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
node->identifier_->MapTo(symbol);
auto logical_op = std::make_shared<ScanAllByLabelPropertyValue>(
input, symbol, label, property, property_name, value, graph_view);
input, symbol, label, property, property_name, value, view);
return ScanAllTuple{node, logical_op, symbol};
}
@ -166,7 +166,7 @@ ExpandTuple MakeExpand(AstStorage &storage, SymbolTable &symbol_table,
EdgeAtom::Direction direction,
const std::vector<storage::EdgeType> &edge_types,
const std::string &node_identifier, bool existing_node,
GraphView graph_view) {
storage::View view) {
auto edge = EDGE(edge_identifier, direction);
auto edge_sym = symbol_table.CreateSymbol(edge_identifier, true);
edge->identifier_->MapTo(edge_sym);
@ -177,7 +177,7 @@ ExpandTuple MakeExpand(AstStorage &storage, SymbolTable &symbol_table,
auto op = std::make_shared<Expand>(input, input_symbol, node_sym, edge_sym,
direction, edge_types, existing_node,
graph_view);
view);
return ExpandTuple{edge, edge_sym, node, node_sym, op};
}

View File

@ -290,9 +290,9 @@ TEST(QueryPlan, Delete) {
// delete all remaining edges
{
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::NEW);
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false,
storage::View::NEW);
auto r_get = storage.Create<Identifier>("r")->MapTo(r_m.edge_sym_);
auto delete_op = std::make_shared<plan::Delete>(
r_m.op_, std::vector<Expression *>{r_get}, false);
@ -344,9 +344,9 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
SymbolTable symbol_table;
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false,
storage::View::OLD);
// getter expressions for deletion
auto n_get = storage.Create<Identifier>("n")->MapTo(n.sym_);
@ -467,7 +467,7 @@ TEST(QueryPlan, SetProperty) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::OUT, {}, "m", false, storage::View::OLD);
// set prop1 to 42 on n and r
auto prop1 = dba.Property("prop1");
@ -518,9 +518,9 @@ TEST(QueryPlan, SetProperties) {
// scan (n)-[r]->(m)
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false,
storage::View::OLD);
auto op = update ? plan::SetProperties::Op::UPDATE
: plan::SetProperties::Op::REPLACE;
@ -624,7 +624,7 @@ TEST(QueryPlan, RemoveProperty) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::OUT, {}, "m", false, storage::View::OLD);
auto n_p = PROPERTY_LOOKUP(IDENT("n")->MapTo(n.sym_), prop1);
auto set_n_p = std::make_shared<plan::RemoveProperty>(r_m.op_, prop1, n_p);
@ -702,7 +702,7 @@ TEST(QueryPlan, NodeFilterSet) {
scan_all.node_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
auto expand =
MakeExpand(storage, symbol_table, scan_all.op_, scan_all.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::BOTH, {}, "m", false, storage::View::OLD);
auto *filter_expr =
EQ(storage.Create<PropertyLookup>(scan_all.node_->identifier_,
storage.GetPropertyIx(prop.first)),
@ -744,7 +744,7 @@ TEST(QueryPlan, FilterRemove) {
scan_all.node_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
auto expand =
MakeExpand(storage, symbol_table, scan_all.op_, scan_all.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::BOTH, {}, "m", false, storage::View::OLD);
auto filter_prop = PROPERTY_LOOKUP(IDENT("n")->MapTo(scan_all.sym_), prop);
auto filter =
std::make_shared<Filter>(expand.op_, LESS(filter_prop, LITERAL(43)));
@ -808,7 +808,7 @@ TEST(QueryPlan, Merge) {
// merge_match branch
auto r_m =
MakeExpand(storage, symbol_table, std::make_shared<Once>(), n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::BOTH, {}, "m", false, storage::View::OLD);
auto m_p = PROPERTY_LOOKUP(IDENT("m")->MapTo(r_m.node_sym_), prop);
auto m_set = std::make_shared<plan::SetProperty>(r_m.op_, prop.second, m_p,
LITERAL(1));

View File

@ -48,9 +48,9 @@ TEST_F(MatchReturnFixture, MatchReturn) {
AddVertices(2);
dba_.AdvanceCommand();
auto test_pull_count = [&](GraphView graph_view) {
auto test_pull_count = [&](storage::View view) {
auto scan_all =
MakeScanAll(storage, symbol_table, "n", nullptr, graph_view);
MakeScanAll(storage, symbol_table, "n", nullptr, view);
auto output =
NEXPR("n", IDENT("n")->MapTo(scan_all.sym_))
->MapTo(symbol_table.CreateSymbol("named_expression_1", true));
@ -59,13 +59,13 @@ TEST_F(MatchReturnFixture, MatchReturn) {
return PullAll(*produce, &context);
};
EXPECT_EQ(2, test_pull_count(GraphView::NEW));
EXPECT_EQ(2, test_pull_count(GraphView::OLD));
EXPECT_EQ(2, test_pull_count(storage::View::NEW));
EXPECT_EQ(2, test_pull_count(storage::View::OLD));
dba_.InsertVertex();
EXPECT_EQ(3, test_pull_count(GraphView::NEW));
EXPECT_EQ(2, test_pull_count(GraphView::OLD));
EXPECT_EQ(3, test_pull_count(storage::View::NEW));
EXPECT_EQ(2, test_pull_count(storage::View::OLD));
dba_.AdvanceCommand();
EXPECT_EQ(3, test_pull_count(GraphView::OLD));
EXPECT_EQ(3, test_pull_count(storage::View::OLD));
}
TEST_F(MatchReturnFixture, MatchReturnPath) {
@ -398,10 +398,11 @@ class ExpandFixture : public testing::Test {
};
TEST_F(ExpandFixture, Expand) {
auto test_expand = [&](EdgeAtom::Direction direction, GraphView graph_view) {
auto test_expand = [&](EdgeAtom::Direction direction,
storage::View view) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r", direction,
{}, "m", false, graph_view);
{}, "m", false, view);
// make a named expression and a produce
auto output =
@ -418,23 +419,23 @@ TEST_F(ExpandFixture, Expand) {
v3.Reconstruct();
dba_.InsertEdge(v1, v2, edge_type);
dba_.InsertEdge(v1, v3, edge_type);
EXPECT_EQ(2, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD));
EXPECT_EQ(2, test_expand(EdgeAtom::Direction::IN, GraphView::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::NEW));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::NEW));
EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::NEW));
EXPECT_EQ(2, test_expand(EdgeAtom::Direction::OUT, storage::View::OLD));
EXPECT_EQ(2, test_expand(EdgeAtom::Direction::IN, storage::View::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::BOTH, storage::View::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, storage::View::NEW));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, storage::View::NEW));
EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, storage::View::NEW));
dba_.AdvanceCommand();
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, GraphView::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, GraphView::OLD));
EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, GraphView::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::OUT, storage::View::OLD));
EXPECT_EQ(4, test_expand(EdgeAtom::Direction::IN, storage::View::OLD));
EXPECT_EQ(8, test_expand(EdgeAtom::Direction::BOTH, storage::View::OLD));
}
TEST_F(ExpandFixture, ExpandPath) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::OUT, {}, "m", false, storage::View::OLD);
Symbol path_sym = symbol_table.CreateSymbol("path", true);
auto path = std::make_shared<ConstructNamedPath>(
r_m.op_, path_sym,
@ -533,8 +534,7 @@ class QueryPlanExpandVariable : public testing::Test {
int layer, EdgeAtom::Direction direction,
const std::vector<storage::EdgeType> &edge_types,
std::optional<size_t> lower, std::optional<size_t> upper, Symbol edge_sym,
const std::string &node_to, GraphView graph_view,
bool is_reverse = false) {
const std::string &node_to, storage::View view, bool is_reverse = false) {
auto n_from = MakeScanAll(storage, symbol_table, node_from, input_op);
auto filter_op = std::make_shared<Filter>(
n_from.op_,
@ -551,8 +551,8 @@ class QueryPlanExpandVariable : public testing::Test {
auto convert = [this](std::optional<size_t> bound) {
return bound ? LITERAL(static_cast<int64_t>(bound.value())) : nullptr;
};
CHECK(graph_view == GraphView::OLD)
<< "ExpandVariable should only be planned with GraphView::OLD";
CHECK(view == storage::View::OLD)
<< "ExpandVariable should only be planned with storage::View::OLD";
return std::make_shared<ExpandVariable>(
filter_op, n_from.sym_, n_to_sym, edge_sym,
@ -565,7 +565,7 @@ class QueryPlanExpandVariable : public testing::Test {
} else
return std::make_shared<Expand>(filter_op, n_from.sym_, n_to_sym,
edge_sym, direction, edge_types, false,
graph_view);
view);
}
/* Creates an edge (in the frame and symbol table). Returns the symbol. */
@ -630,7 +630,7 @@ TEST_F(QueryPlanExpandVariable, OneVariableExpansion) {
auto e = Edge("r", direction);
return GetEdgeListSizes(
AddMatch<ExpandVariable>(nullptr, "n", layer, direction, {}, lower,
upper, e, "m", GraphView::OLD, reverse),
upper, e, "m", storage::View::OLD, reverse),
e);
};
@ -691,20 +691,22 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessSingleAndVariableExpansion) {
if (single_expansion_before) {
symbols.push_back(Edge("r0", direction));
last_op = AddMatch<Expand>(last_op, "n0", layer, direction, {}, lower,
upper, symbols.back(), "m0", GraphView::OLD);
last_op =
AddMatch<Expand>(last_op, "n0", layer, direction, {}, lower, upper,
symbols.back(), "m0", storage::View::OLD);
}
auto var_length_sym = Edge("r1", direction);
symbols.push_back(var_length_sym);
last_op =
AddMatch<ExpandVariable>(last_op, "n1", layer, direction, {}, lower,
upper, var_length_sym, "m1", GraphView::OLD);
last_op = AddMatch<ExpandVariable>(last_op, "n1", layer, direction, {},
lower, upper, var_length_sym, "m1",
storage::View::OLD);
if (!single_expansion_before) {
symbols.push_back(Edge("r2", direction));
last_op = AddMatch<Expand>(last_op, "n2", layer, direction, {}, lower,
upper, symbols.back(), "m2", GraphView::OLD);
last_op =
AddMatch<Expand>(last_op, "n2", layer, direction, {}, lower, upper,
symbols.back(), "m2", storage::View::OLD);
}
if (add_uniqueness_check) {
@ -734,11 +736,11 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessTwoVariableExpansions) {
auto e1 = Edge("r1", direction);
auto first =
AddMatch<ExpandVariable>(nullptr, "n1", layer, direction, {}, lower,
upper, e1, "m1", GraphView::OLD);
upper, e1, "m1", storage::View::OLD);
auto e2 = Edge("r2", direction);
auto last_op =
AddMatch<ExpandVariable>(first, "n2", layer, direction, {}, lower,
upper, e2, "m2", GraphView::OLD);
upper, e2, "m2", storage::View::OLD);
if (add_uniqueness_check) {
last_op = std::make_shared<EdgeUniquenessFilter>(
last_op, e2, std::vector<Symbol>{e1});
@ -757,7 +759,7 @@ TEST_F(QueryPlanExpandVariable, NamedPath) {
auto e = Edge("r", EdgeAtom::Direction::OUT);
auto expand =
AddMatch<ExpandVariable>(nullptr, "n", 0, EdgeAtom::Direction::OUT, {}, 2,
2, e, "m", GraphView::OLD);
2, e, "m", storage::View::OLD);
auto find_symbol = [this](const std::string &name) {
for (const auto &sym : symbol_table.table())
if (sym.second.name() == name) return sym.second;
@ -1154,7 +1156,7 @@ TEST(QueryPlan, ExpandOptional) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m =
MakeExpand(storage, symbol_table, nullptr, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::OUT, {}, "m", false, storage::View::OLD);
auto optional = std::make_shared<plan::Optional>(
n.op_, r_m.op_, std::vector<Symbol>{r_m.edge_sym_, r_m.node_sym_});
@ -1225,7 +1227,7 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
// MATCH (n) -[r]-> (m)
auto r_m =
MakeExpand(storage, symbol_table, with, with_n_sym, "r",
EdgeAtom::Direction::OUT, {}, "m", false, GraphView::OLD);
EdgeAtom::Direction::OUT, {}, "m", false, storage::View::OLD);
// RETURN m
auto m_ne = NEXPR("m", IDENT("m")->MapTo(r_m.node_sym_))
->MapTo(symbol_table.CreateSymbol("m", true));
@ -1273,7 +1275,7 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
node->identifier_->MapTo(with_n_sym);
auto expand = std::make_shared<plan::Expand>(
m.op_, m.sym_, with_n_sym, edge_sym, edge_direction,
std::vector<storage::EdgeType>{}, true, GraphView::OLD);
std::vector<storage::EdgeType>{}, true, storage::View::OLD);
// RETURN m
auto m_ne = NEXPR("m", IDENT("m")->MapTo(m.sym_))
->MapTo(symbol_table.CreateSymbol("m", true));
@ -1303,11 +1305,11 @@ TEST(QueryPlan, ExpandExistingNode) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_n = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {}, "n", with_existing,
GraphView::OLD);
storage::View::OLD);
if (with_existing)
r_n.op_ = std::make_shared<Expand>(
n.op_, n.sym_, n.sym_, r_n.edge_sym_, r_n.edge_->direction_,
std::vector<storage::EdgeType>{}, with_existing, GraphView::OLD);
std::vector<storage::EdgeType>{}, with_existing, storage::View::OLD);
// make a named expression and a produce
auto output =
@ -1339,7 +1341,7 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_ =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "_", false, GraphView::OLD);
EdgeAtom::Direction::BOTH, {}, "_", false, storage::View::OLD);
auto context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(1, PullAll(*r_.op_, &context));
}
@ -1388,7 +1390,7 @@ TEST(QueryPlan, EdgeFilter) {
const auto &edge_type = edge_types[0];
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {edge_type}, "m", false,
GraphView::OLD);
storage::View::OLD);
r_m.edge_->edge_types_.push_back(
storage.GetEdgeTypeIx(dba.EdgeTypeName(edge_type)));
r_m.edge_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
@ -1434,7 +1436,7 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {type_1, type_2}, "m", false,
GraphView::OLD);
storage::View::OLD);
// make a named expression and a produce
auto output =
@ -1489,13 +1491,13 @@ TEST(QueryPlan, EdgeUniquenessFilter) {
SymbolTable symbol_table;
auto n1 = MakeScanAll(storage, symbol_table, "n1");
auto r1_n2 =
MakeExpand(storage, symbol_table, n1.op_, n1.sym_, "r1",
EdgeAtom::Direction::OUT, {}, "n2", false, GraphView::OLD);
auto r1_n2 = MakeExpand(storage, symbol_table, n1.op_, n1.sym_, "r1",
EdgeAtom::Direction::OUT, {}, "n2", false,
storage::View::OLD);
std::shared_ptr<LogicalOperator> last_op = r1_n2.op_;
auto r2_n3 =
MakeExpand(storage, symbol_table, last_op, r1_n2.node_sym_, "r2",
EdgeAtom::Direction::OUT, {}, "n3", false, GraphView::OLD);
auto r2_n3 = MakeExpand(storage, symbol_table, last_op, r1_n2.node_sym_,
"r2", EdgeAtom::Direction::OUT, {}, "n3", false,
storage::View::OLD);
last_op = r2_n3.op_;
if (edge_uniqueness)
last_op = std::make_shared<EdgeUniquenessFilter>(