Query::Plan::ExpandBreadthFirst added

Summary:
Not complete (but review can start):
- implementation should be done
- still need to finish tests
- documentation missing

Reviewers: mislav.bradac, teon.banek, buda

Reviewed By: mislav.bradac

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D616
This commit is contained in:
florijan 2017-07-30 01:15:43 +02:00
parent da0e4a5b12
commit e45ae4c4b6
8 changed files with 581 additions and 96 deletions

View File

@ -459,6 +459,23 @@ void Expand::ExpandCursor::Reset() {
out_edges_it_.release();
}
namespace {
// Switch the given [Vertex/Edge]Accessor to the desired state.
template <typename TAccessor>
void SwitchAccessor(TAccessor &accessor, GraphView graph_view) {
switch (graph_view) {
case GraphView::NEW:
accessor.SwitchNew();
break;
case GraphView::OLD:
accessor.SwitchOld();
break;
case GraphView::AS_IS:
break;
}
}
}
bool Expand::ExpandCursor::InitEdges(Frame &frame,
const SymbolTable &symbol_table) {
if (!input_cursor_->Pull(frame, symbol_table)) return false;
@ -469,17 +486,7 @@ bool Expand::ExpandCursor::InitEdges(Frame &frame,
if (vertex_value.IsNull()) return false;
ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
auto &vertex = vertex_value.Value<VertexAccessor>();
// Switch the expansion origin vertex to the desired state.
switch (self_.graph_view_) {
case GraphView::NEW:
vertex.SwitchNew();
break;
case GraphView::OLD:
vertex.SwitchOld();
break;
case GraphView::AS_IS:
break;
}
SwitchAccessor(vertex, self_.graph_view_);
auto direction = self_.direction_;
if (direction == EdgeAtom::Direction::IN ||
@ -515,12 +522,12 @@ bool Expand::ExpandCursor::PullNode(const EdgeAccessor &new_edge,
}
}
ExpandVariable::ExpandVariable(
Symbol node_symbol, Symbol edge_symbol, EdgeAtom::Direction direction,
Expression *lower_bound,
Expression *upper_bound,
const std::shared_ptr<LogicalOperator> &input, Symbol input_symbol,
bool existing_node, bool existing_edge, GraphView graph_view)
ExpandVariable::ExpandVariable(Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction,
Expression *lower_bound, Expression *upper_bound,
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node,
bool existing_edge, GraphView graph_view)
: ExpandCommon(node_symbol, edge_symbol, direction, input, input_symbol,
existing_node, existing_edge, graph_view),
lower_bound_(lower_bound),
@ -575,6 +582,23 @@ auto ExpandFromVertex(const VertexAccessor &vertex,
return iter::chain.from_iterable(std::move(chain_elements));
}
/** A helper function for evaluating an expression that's an int.
*
* @param evaluator
* @param expr
* @param what - Name of what's getting evaluated. Used for user
* feedback (via exception) when the evaluated value is not an int.
*/
int64_t EvaluateInt(ExpressionEvaluator &evaluator, Expression *expr,
const std::string &what) {
TypedValue value = expr->Accept(evaluator);
try {
return value.Value<int64_t>();
} catch (TypedValueException &e) {
throw QueryRuntimeException(what + " must be an int");
}
}
} // annonymous namespace
class ExpandVariableCursor : public Cursor {
@ -650,30 +674,16 @@ class ExpandVariableCursor : public Cursor {
if (vertex_value.IsNull()) return false;
ExpectType(self_.input_symbol_, vertex_value, TypedValue::Type::Vertex);
auto &vertex = vertex_value.Value<VertexAccessor>();
switch (self_.graph_view_) {
case GraphView::NEW:
vertex.SwitchNew();
break;
case GraphView::OLD:
vertex.SwitchOld();
break;
case GraphView::AS_IS:
break;
}
SwitchAccessor(vertex, self_.graph_view_);
// evaluate the upper and lower bounds
ExpressionEvaluator evaluator(frame, symbol_table, db_);
auto calc_bound = [this, &evaluator](auto &bound) {
TypedValue value = bound->Accept(evaluator);
try {
auto return_value = value.Value<int64_t>();
if (return_value < 0)
throw QueryRuntimeException(
"Variable expansion bound must be positive or zero");
return return_value;
} catch (TypedValueException &e) {
throw QueryRuntimeException("Variable expansion bound must be an int");
}
auto value = EvaluateInt(evaluator, bound, "Variable expansion bound");
if (value < 0)
throw QueryRuntimeException(
"Variable expansion bound must be positive or zero");
return value;
};
lower_bound_ = self_.lower_bound_ ? calc_bound(self_.lower_bound_) : 1;
@ -838,6 +848,154 @@ std::unique_ptr<Cursor> ExpandVariable::MakeCursor(GraphDbAccessor &db) {
return std::make_unique<ExpandVariableCursor>(*this, db);
}
ExpandBreadthFirst::ExpandBreadthFirst(
Symbol node_symbol, Symbol edge_list_symbol, EdgeAtom::Direction direction,
Expression *max_depth, Symbol inner_node_symbol, Symbol inner_edge_symbol,
Expression *where, const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node, GraphView graph_view)
: node_symbol_(node_symbol),
edge_list_symbol_(edge_list_symbol),
direction_(direction),
max_depth_(max_depth),
inner_node_symbol_(inner_node_symbol),
inner_edge_symbol_(inner_edge_symbol),
where_(where),
input_(input),
input_symbol_(input_symbol),
existing_node_(existing_node),
graph_view_(graph_view) {}
ACCEPT_WITH_INPUT(ExpandBreadthFirst)
std::unique_ptr<Cursor> ExpandBreadthFirst::MakeCursor(GraphDbAccessor &db) {
return std::make_unique<ExpandBreadthFirst::Cursor>(*this, db);
}
ExpandBreadthFirst::Cursor::Cursor(const ExpandBreadthFirst &self,
GraphDbAccessor &db)
: self_(self), db_(db), input_cursor_(self_.input_->MakeCursor(db)) {}
bool ExpandBreadthFirst::Cursor::Pull(Frame &frame,
const SymbolTable &symbol_table) {
// evaulator for the filtering condition
ExpressionEvaluator evaluator(frame, symbol_table, db_, self_.graph_view_);
// for the given (edge, vertex) pair checks if they satisfy the
// "where" condition. if so, places them in the to_visit_ structure.
auto expand_pair = [this, &evaluator, &frame](EdgeAccessor edge,
VertexAccessor vertex) {
// if we already processed the given vertex it doesn't get expanded
if (processed_.find(vertex) != processed_.end()) return;
SwitchAccessor(edge, self_.graph_view_);
SwitchAccessor(vertex, self_.graph_view_);
// to evaluate the where expression we need the inner
// values on the frame
frame[self_.inner_edge_symbol_] = edge;
frame[self_.inner_node_symbol_] = vertex;
TypedValue result = self_.where_->Accept(evaluator);
switch (result.type()) {
case TypedValue::Type::Null:
// TODO review: is treating Null as false desired?
return;
case TypedValue::Type::Bool:
if (!result.Value<bool>()) return;
break;
default:
throw QueryRuntimeException(
"Expansion condition must be boolean or null");
}
to_visit_next_.emplace_back(edge, vertex);
processed_.emplace(vertex, edge);
};
// populates the to_visit_next_ structure with expansions
// from the given vertex. skips expansions that don't satisfy
// the "where" condition.
auto expand_from_vertex = [this, &expand_pair](VertexAccessor &vertex) {
if (self_.direction_ != EdgeAtom::Direction::IN)
for (const EdgeAccessor &edge : vertex.out())
expand_pair(edge, edge.to());
if (self_.direction_ != EdgeAtom::Direction::OUT)
for (const EdgeAccessor &edge : vertex.in())
expand_pair(edge, edge.from());
};
// do it all in a loop because we skip some elements
while (true) {
// if we have nothing to visit on the current depth, switch to next
if (to_visit_current_.empty()) to_visit_current_.swap(to_visit_next_);
// if current is still empty, it means both are empty, so pull from input
if (to_visit_current_.empty()) {
if (!input_cursor_->Pull(frame, symbol_table)) return false;
processed_.clear();
auto vertex_value = frame[self_.input_symbol_];
// it is possible that the vertex is Null due to optional matching
if (vertex_value.IsNull()) continue;
auto vertex = vertex_value.Value<VertexAccessor>();
SwitchAccessor(vertex, self_.graph_view_);
processed_.emplace(vertex, std::experimental::nullopt);
expand_from_vertex(vertex);
max_depth_ = EvaluateInt(evaluator, self_.max_depth_,
"Max depth in breadth-first expansion");
if (max_depth_ < 1)
throw QueryRuntimeException(
"Max depth in breadth-first expansion must be greater then zero");
// go back to loop start and see if we expanded anything
continue;
}
// take the next expansion from the queue
std::pair<EdgeAccessor, VertexAccessor> expansion =
to_visit_current_.front();
to_visit_current_.pop_front();
// create the frame value for the edges
std::vector<TypedValue> edge_list{expansion.first};
auto last_vertex = expansion.second;
while (true) {
const EdgeAccessor &last_edge = edge_list.back().Value<EdgeAccessor>();
last_vertex =
last_edge.from() == last_vertex ? last_edge.to() : last_edge.from();
// origin_vertex must be in processed
const auto &previous_edge = processed_.find(last_vertex)->second;
if (!previous_edge) break;
edge_list.push_back(previous_edge.value());
}
// expand only if what we've just expanded is less then max depth
if (static_cast<int>(edge_list.size()) < max_depth_)
expand_from_vertex(expansion.second);
// place destination node on the frame, handle existence flag
if (self_.existing_node_) {
TypedValue &node = frame[self_.node_symbol_];
// due to optional matching the existing node could be null
if (node.IsNull() || (node != expansion.second).Value<bool>()) continue;
} else
frame[self_.node_symbol_] = expansion.second;
// place edges on the frame in the correct order
std::reverse(edge_list.begin(), edge_list.end());
frame[self_.edge_list_symbol_] = std::move(edge_list);
return true;
}
}
void ExpandBreadthFirst::Cursor::Reset() {
input_cursor_->Reset();
processed_.clear();
to_visit_next_.clear();
to_visit_current_.clear();
}
Filter::Filter(const std::shared_ptr<LogicalOperator> &input,
Expression *expression)
: input_(input ? input : std::make_shared<Once>()),

View File

@ -4,11 +4,13 @@
#include <glog/logging.h>
#include <algorithm>
#include <deque>
#include <experimental/optional>
#include <memory>
#include <tuple>
#include <type_traits>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "database/graph_db_accessor.hpp"
@ -62,6 +64,7 @@ class ScanAllByLabelPropertyRange;
class ScanAllByLabelPropertyValue;
class Expand;
class ExpandVariable;
class ExpandBreadthFirst;
class Filter;
class Produce;
class Delete;
@ -87,8 +90,8 @@ class CreateIndex;
using LogicalOperatorCompositeVisitor = ::utils::CompositeVisitor<
Once, CreateNode, CreateExpand, ScanAll, ScanAllByLabel,
ScanAllByLabelPropertyRange, ScanAllByLabelPropertyValue, Expand,
ExpandVariable, Filter, Produce, Delete, SetProperty, SetProperties,
SetLabels, RemoveProperty, RemoveLabels,
ExpandVariable, ExpandBreadthFirst, Filter, Produce, Delete, SetProperty,
SetProperties, SetLabels, RemoveProperty, RemoveLabels,
ExpandUniquenessFilter<VertexAccessor>,
ExpandUniquenessFilter<EdgeAccessor>, Accumulate, AdvanceCommand, Aggregate,
Skip, Limit, OrderBy, Merge, Optional, Unwind, Distinct>;
@ -449,6 +452,32 @@ class ExpandCommon {
decltype(std::declval<VertexAccessor>().out().begin());
public:
/**
* Initializes common expansion parameters.
*
* Edge/Node existence are controlled with booleans. 'true'
* denotes that this expansion references an already
* Pulled node/edge, and should only be checked for equality
* during expansion.
*
* Expansion can be done from old or new state of the vertex
* the expansion originates from. This is controlled with a
* constructor argument.
*
* @param node_symbol Symbol pointing to the node to be expanded. This is
* where the new node will be stored.
* @param edge_symbol Symbol for the edges to be expanded. This is where
* a TypedValue containing a list of expanded edges will be stored.
* @param direction EdgeAtom::Direction determining the direction of edge
* expansion. The direction is relative to the starting vertex for each
* expansion.
* @param input Optional LogicalOperator that preceeds this one.
* @param input_symbol Symbol that points to a VertexAccessor in the Frame
* that expansion should emanate from.
* @param existing_node If or not the node to be expanded is already present
* in the Frame and should just be checked for equality.
* @param existing_edge Same like `existing_node`, but for edges.
*/
ExpandCommon(Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction,
const std::shared_ptr<LogicalOperator> &input,
@ -503,30 +532,8 @@ class ExpandCommon {
class Expand : public LogicalOperator, ExpandCommon {
public:
/**
* @brief Creates an expansion.
*
* Edge/Node existence is controlled via booleans. A true value
* simply denotes that this expansion references an already
* Pulled node/edge, and should only be checked for equalities
* during expansion.
*
* Expansion can be done from old or new state of the vertex
* the expansion originates from. This is controlled with a
* constructor argument.
*
* @param node_symbol Symbol pointing to the node to be expanded. This is
* where the new node will be stored.
* @param edge_symbol Symbol for the edge to be expanded. This is where the
* edge value will be stored.
* @param direction EdgeAtom::Direction determining the direction of edge
* expansion. The direction is relative to the starting vertex (pointed by
* `input_symbol`).
* @param input Optional LogicalOperator that preceeds this one.
* @param input_symbol Symbol that points to a VertexAccessor in the Frame
* that expansion should emanate from.
* @param existing_node If or not the node to be expanded is already present
* in the Frame and should just be checked for equality.
* @param existing_edge Same like `existing_node`, but for edges.
* Creates an expansion. All parameters are forwarded to @c ExpandCommon and
* are documented there.
*/
using ExpandCommon::ExpandCommon;
@ -577,7 +584,7 @@ class Expand : public LogicalOperator, ExpandCommon {
};
/**
* @brief Variable-length expansion operator. For a node existing in
* Variable-length expansion operator. For a node existing in
* the frame it expands a variable number of edges and places them
* (in a list-type TypedValue), as well as the final destination node,
* on the frame.
@ -602,37 +609,16 @@ class ExpandVariable : public LogicalOperator, ExpandCommon {
public:
/**
* @brief Creates a variable-length expansion.
* Creates a variable-length expansion. Most params are forwarded
* to the @c ExpandCommon constructor, and are documented there.
*
* Expansion length bounds are both inclusive (as in Neo's Cypher
* implementation).
*
* Edge/Node existence is controlled via booleans. A true value
* simply denotes that this expansion references an already
* Pulled node/edge, and should only be checked for equalities
* during expansion.
*
* Expansion can be done from old or new state of the vertex
* the expansion originates from. This is controlled with a
* constructor argument.
*
* @param node_symbol Symbol pointing to the node to be expanded. This is
* where the new node will be stored.
* @param edge_symbol Symbol for the edges to be expanded. This is where
* a TypedValue containing a list of expanded edges will be stored.
* @param direction EdgeAtom::Direction determining the direction of edge
* expansion. The direction is relative to the starting vertex for each
* expansion.
* @param lower_bound An optional indicator of the minimum number of edges
* that get expanded (inclusive).
* @param lower_bound An optional indicator of the maximum number of edges
* @param upper_bound An optional indicator of the maximum number of edges
* that get expanded (inclusive).
* @param input Optional LogicalOperator that preceeds this one.
* @param input_symbol Symbol that points to a VertexAccessor in the Frame
* that expansion should emanate from.
* @param existing_node If or not the node to be expanded is already present
* in the Frame and should just be checked for equality.
* @param existing_edge Same like `existing_node`, but for edges.
*/
ExpandVariable(Symbol node_symbol, Symbol edge_symbol,
EdgeAtom::Direction direction, Expression *lower_bound,
@ -650,6 +636,98 @@ class ExpandVariable : public LogicalOperator, ExpandCommon {
Expression *lower_bound_;
Expression *upper_bound_;
};
/**
* Variable length breadth-first expansion operator. For a node existing in the
* frame it expands a variable number of edges and places them (in a list-type
* TypedValue), as well as the final destination node, on the frame.
*
* This class does not handle node/edge filtering based on properties, labels
* and edge types. However, it does handle filtering on existing node. Edge and
* vertex uniqueness are inherent to breadth-first expansion, due to filtering
* of already visited nodes.
*
* Most params are equivalent to @c ExpandCommon, and are documented there.
* Additional param documentation follows:
*
* @param max_depth Expression that controls the maximum allowed length of the
* expansion. Inclusive.
* @param inner_node_symbol For each expansion the node expanded into is
* assigned to this symbol so it can be evaulated by the 'where' expression.
* @param inner_edge_symbol Like `inner_node_symbol`
* @Param where An expression that controls whether an expansion is accepted or
* not. It can use the `inner` node and edge symbols. The expansion succeeds
* only if the expression evaluates to `true` for the expanded-into edge and
* node. `false` and `Null` prevent expansion, while any other value results in
* an exception.
*/
class ExpandBreadthFirst : public LogicalOperator {
public:
ExpandBreadthFirst(Symbol node_symbol, Symbol edge_list_symbol,
EdgeAtom::Direction direction, Expression *max_depth,
Symbol inner_node_symbol, Symbol inner_edge_symbol,
Expression *where,
const std::shared_ptr<LogicalOperator> &input,
Symbol input_symbol, bool existing_node,
GraphView graph_view = GraphView::AS_IS);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(GraphDbAccessor &db) override;
private:
class Cursor : public query::plan::Cursor {
public:
Cursor(const ExpandBreadthFirst &self, GraphDbAccessor &db);
bool Pull(Frame &frame, const SymbolTable &symbol_table) override;
void Reset() override;
private:
const ExpandBreadthFirst &self_;
GraphDbAccessor &db_;
const std::unique_ptr<query::plan::Cursor> input_cursor_;
// maximum depth of the expansion. calculated on each pull
// from the input, the initial value is irrelevant.
int max_depth_{-1};
// maps vertices to the edge they got expanded from. it is an optional
// edge because the root does not get expanded from anything.
// contains visited vertices as well as those scheduled to be visited.
std::unordered_map<VertexAccessor,
std::experimental::optional<EdgeAccessor>>
processed_;
// edge/vertex pairs we have yet to visit, for current and next depth
std::deque<std::pair<EdgeAccessor, VertexAccessor>> to_visit_current_;
std::deque<std::pair<EdgeAccessor, VertexAccessor>> to_visit_next_;
};
// info on what's getting expanded
const Symbol node_symbol_;
const Symbol edge_list_symbol_;
const EdgeAtom::Direction direction_;
Expression *max_depth_;
// symbols for a single node and edge that are currently getting expanded
const Symbol inner_node_symbol_;
const Symbol inner_edge_symbol_;
// a filtering expression for skipping expansions during expansion
// can refer to inner node and edges
Expression *where_;
// the input op and the symbol under which the op's result
// can be found in the frame
const std::shared_ptr<LogicalOperator> input_;
const Symbol input_symbol_;
// if the node symbol already exists on the frame and this expansion
// should just match it
bool existing_node_;
// from which state the input node should get expanded
const GraphView graph_view_;
};
/**
* @brief Filter whose Pull returns true only when the given expression
* evaluates into true.

View File

@ -182,6 +182,15 @@ std::vector<TypedValue> &TypedValue::Value<std::vector<TypedValue>>() {
return list_v;
}
std::vector<TypedValue> &TypedValue::ValueList() {
return Value<std::vector<TypedValue>>();
}
const std::vector<TypedValue> &TypedValue::ValueList() const {
return Value<std::vector<TypedValue>>();
}
template <>
std::map<std::string, TypedValue>
&TypedValue::Value<std::map<std::string, TypedValue>>() {

View File

@ -130,6 +130,9 @@ class TypedValue : public TotalOrdering<TypedValue, TypedValue, TypedValue> {
template <typename T>
const T &Value() const;
std::vector<TypedValue> &ValueList();
const std::vector<TypedValue> &ValueList() const;
/** Convenience function for checking if this TypedValue is Null */
bool IsNull() const;

View File

@ -14,12 +14,14 @@
* @param db_accessor A database accessor to create the record accessors with.
*/
template <typename TAccessor, typename TIterable>
auto make_accessor_iterator(const TIterable &records, GraphDbAccessor &db_accessor) {
return iter::imap([&db_accessor](auto vlist) {
return TAccessor(*vlist, db_accessor);
// note that here we iterate over records in REVERSED order
// this is necessary for DETACH DELETE (see GraphDbAccessor)
// which deletes items from relationship collections in a
// vertex accessor
}, iter::reversed(records));
auto make_accessor_iterator(TIterable &&records, GraphDbAccessor &db_accessor) {
return iter::imap(
[&db_accessor](auto vlist) {
return TAccessor(*vlist, db_accessor);
// note that here we iterate over records in REVERSED order
// this is necessary for DETACH DELETE (see GraphDbAccessor)
// which deletes items from relationship collections in a
// vertex accessor
},
iter::reversed(std::forward<TIterable>(records)));
}

View File

@ -79,3 +79,12 @@ class VertexAccessor : public RecordAccessor<Vertex> {
};
std::ostream &operator<<(std::ostream &, const VertexAccessor &);
// hash function for the vertex accessor
namespace std {
template <> struct hash<VertexAccessor> {
size_t operator()(const VertexAccessor &v) const {
return v.temporary_id();
};
};
}

View File

@ -473,6 +473,7 @@ auto GetMerge(AstTreeStorage &storage, Pattern *pattern, OnMatch on_match,
#define COUNT(expr) \
storage.Create<query::Aggregation>((expr), query::Aggregation::Op::COUNT)
#define EQ(expr1, expr2) storage.Create<query::EqualOperator>((expr1), (expr2))
#define NEQ(expr1, expr2) storage.Create<query::NotEqualOperator>((expr1), (expr2))
#define AND(expr1, expr2) storage.Create<query::AndOperator>((expr1), (expr2))
#define OR(expr1, expr2) storage.Create<query::OrOperator>((expr1), (expr2))
// Function call

View File

@ -550,8 +550,233 @@ TEST_F(QueryPlanExpandVariable, ExistingEdges) {
(map_int{{1, 4}, {2, 12}}));
}
namespace std {
template <>
struct hash<std::pair<int, int>> {
size_t operator()(const std::pair<int, int> &p) const {
return p.first + 31 * p.second;
}
};
}
// TODO test optional + variable length
/** A test fixture for breadth first expansion */
class QueryPlanExpandBreadthFirst : public testing::Test {
protected:
Dbms dbms_;
std::unique_ptr<GraphDbAccessor> dba_ = dbms_.active();
GraphDbTypes::Property prop = dba_->property("property");
GraphDbTypes::EdgeType edge_type = dba_->edge_type("edge_type");
// make 4 vertices because we'll need to compare against them exactly
// v[0] has `prop` with the value 0
std::vector<VertexAccessor> v;
// make some edges too, in a map (from, to) vertex indices
std::unordered_map<std::pair<int, int>, EdgeAccessor> e;
AstTreeStorage storage;
SymbolTable symbol_table;
// inner edge and vertex symbols
// edge from a to b has `prop` with the value ab (all ints)
Symbol inner_edge = symbol_table.CreateSymbol("inner_edge", true);
Symbol inner_node = symbol_table.CreateSymbol("inner_node", true);
void SetUp() {
for (int i = 0; i < 4; i++) {
v.push_back(dba_->insert_vertex());
v.back().PropsSet(prop, i);
}
auto add_edge = [&](int from, int to) {
EdgeAccessor edge = dba_->insert_edge(v[from], v[to], edge_type);
edge.PropsSet(prop, from * 10 + to);
e.emplace(std::make_pair(from, to), edge);
};
add_edge(0, 1);
add_edge(2, 0);
add_edge(2, 1);
add_edge(1, 3);
add_edge(3, 2);
add_edge(2, 2);
dba_->advance_command();
for (auto &vertex : v) vertex.Reconstruct();
for (auto &edge : e) edge.second.Reconstruct();
}
// defines and performs a breadth-first expansion with the given params
// returns a vector of pairs. each pair is (vector-of-edges, vertex)
auto ExpandBF(EdgeAtom::Direction direction, int max_depth, Expression *where,
GraphView graph_view = GraphView::AS_IS,
std::experimental::optional<int> node_id = 0,
ScanAllTuple *existing_node_input = nullptr) {
// scan the nodes optionally filtering on property value
auto n =
MakeScanAll(storage, symbol_table, "n",
existing_node_input ? existing_node_input->op_ : nullptr);
auto last_op = n.op_;
if (node_id) {
last_op = std::make_shared<Filter>(
last_op,
EQ(PROPERTY_LOOKUP(n.node_->identifier_, prop), LITERAL(*node_id)));
}
// expand bf
auto node_sym = existing_node_input
? existing_node_input->sym_
: symbol_table.CreateSymbol("node", true);
auto edge_list_sym = symbol_table.CreateSymbol("edgelist_", true);
last_op = std::make_shared<ExpandBreadthFirst>(
node_sym, edge_list_sym, direction, LITERAL(max_depth), inner_node,
inner_edge, where, last_op, n.sym_, existing_node_input != nullptr,
graph_view);
Frame frame(symbol_table.max_position());
auto cursor = last_op->MakeCursor(*dba_);
std::vector<std::pair<std::vector<EdgeAccessor>, VertexAccessor>> results;
while (cursor->Pull(frame, symbol_table)) {
results.emplace_back(std::vector<EdgeAccessor>(),
frame[node_sym].Value<VertexAccessor>());
for (const TypedValue &edge : frame[edge_list_sym].ValueList())
results.back().first.emplace_back(edge.Value<EdgeAccessor>());
}
return results;
}
template <typename TAccessor>
auto GetProp(const TAccessor &accessor) {
return accessor.PropsAt(prop).template Value<int64_t>();
}
Expression *PropNe(Symbol symbol, int value) {
auto ident = IDENT("inner_element");
symbol_table[*ident] = symbol;
return NEQ(PROPERTY_LOOKUP(ident, prop), LITERAL(value));
}
};
#define EXPECT_EITHER(value, a, b) EXPECT_TRUE(value == a || value == b)
TEST_F(QueryPlanExpandBreadthFirst, Basic) {
auto results = ExpandBF(EdgeAtom::Direction::BOTH, 1000, LITERAL(true));
ASSERT_EQ(results.size(), 3);
// check end nodes
EXPECT_EITHER(GetProp(results[0].second), 1, 2);
EXPECT_EITHER(GetProp(results[1].second), 1, 2);
EXPECT_NE(GetProp(results[0].second), GetProp(results[1].second));
EXPECT_EQ(GetProp(results[2].second), 3);
// check edges
ASSERT_EQ(results[0].first.size(), 1);
EXPECT_EITHER(GetProp(results[0].first[0]), 1, 20);
ASSERT_EQ(results[1].first.size(), 1);
EXPECT_EITHER(GetProp(results[1].first[0]), 1, 20);
ASSERT_EQ(results[2].first.size(), 2);
EXPECT_EITHER(GetProp(results[2].first[1]), 13, 32);
}
TEST_F(QueryPlanExpandBreadthFirst, EdgeDirection) {
{
auto results = ExpandBF(EdgeAtom::Direction::IN, 1000, LITERAL(true));
ASSERT_EQ(results.size(), 3);
EXPECT_EQ(GetProp(results[0].second), 2);
EXPECT_EQ(GetProp(results[1].second), 3);
EXPECT_EQ(GetProp(results[2].second), 1);
for (int i = 0; i < 3; i++) EXPECT_EQ(results[i].first.size(), i + 1);
// assume edges are OK because vertices are (tested before)
}
{
auto results = ExpandBF(EdgeAtom::Direction::OUT, 1000, LITERAL(true));
ASSERT_EQ(results.size(), 3);
EXPECT_EQ(GetProp(results[0].second), 1);
EXPECT_EQ(GetProp(results[1].second), 3);
EXPECT_EQ(GetProp(results[2].second), 2);
for (int i = 0; i < 3; i++) EXPECT_EQ(results[i].first.size(), i + 1);
// assume edges are OK because vertices are (tested before)
}
}
TEST_F(QueryPlanExpandBreadthFirst, Where) {
{
auto results =
ExpandBF(EdgeAtom::Direction::BOTH, 1000, PropNe(inner_node, 2));
ASSERT_EQ(results.size(), 2);
EXPECT_EQ(GetProp(results[0].second), 1);
EXPECT_EQ(GetProp(results[1].second), 3);
}
{
auto results =
ExpandBF(EdgeAtom::Direction::BOTH, 1000, PropNe(inner_edge, 20));
ASSERT_EQ(results.size(), 3);
EXPECT_EQ(GetProp(results[0].second), 1);
EXPECT_EITHER(GetProp(results[1].second), 3, 2);
EXPECT_EITHER(GetProp(results[2].second), 3, 2);
EXPECT_NE(GetProp(results[1].second), GetProp(results[2].second));
}
}
TEST_F(QueryPlanExpandBreadthFirst, GraphState) {
auto ExpandSize = [this](GraphView graph_view) {
return ExpandBF(EdgeAtom::Direction::BOTH, 1000, LITERAL(true), graph_view)
.size();
};
EXPECT_EQ(ExpandSize(GraphView::OLD), 3);
EXPECT_EQ(ExpandSize(GraphView::NEW), 3);
auto new_vertex = dba_->insert_vertex();
new_vertex.PropsSet(prop, 4);
dba_->insert_edge(v[3], new_vertex, edge_type);
EXPECT_EQ(CountIterable(dba_->vertices(false)), 4);
EXPECT_EQ(CountIterable(dba_->vertices(true)), 5);
EXPECT_EQ(ExpandSize(GraphView::OLD), 3);
EXPECT_EQ(ExpandSize(GraphView::NEW), 4);
dba_->advance_command();
EXPECT_EQ(ExpandSize(GraphView::OLD), 4);
EXPECT_EQ(ExpandSize(GraphView::NEW), 4);
}
TEST_F(QueryPlanExpandBreadthFirst, MultipleInputs) {
auto results = ExpandBF(EdgeAtom::Direction::OUT, 1000, LITERAL(true),
GraphView::AS_IS, std::experimental::nullopt);
// expect that each vertex has been returned 3 times
EXPECT_EQ(results.size(), 12);
std::vector<int> found(4, 0);
for (const auto &row : results) found[GetProp(row.second)]++;
EXPECT_EQ(found, std::vector<int>(4, 3));
}
TEST_F(QueryPlanExpandBreadthFirst, ExistingNode) {
auto ExpandPreceeding =
[this](std::experimental::optional<int> preceeding_node_id) {
// scan the nodes optionally filtering on property value
auto n0 = MakeScanAll(storage, symbol_table, "n0");
if (preceeding_node_id) {
auto filter = std::make_shared<Filter>(
n0.op_, EQ(PROPERTY_LOOKUP(n0.node_->identifier_, prop),
LITERAL(*preceeding_node_id)));
// inject the filter op into the ScanAllTuple. that way the filter op
// can be passed into the ExpandBF function without too much refactor
n0.op_ = filter;
}
return ExpandBF(EdgeAtom::Direction::OUT, 1000, LITERAL(true),
GraphView::AS_IS, std::experimental::nullopt, &n0);
};
EXPECT_EQ(ExpandPreceeding(std::experimental::nullopt).size(), 12);
{
auto results = ExpandPreceeding(0);
ASSERT_EQ(results.size(), 3);
for (int i = 0; i < 3; i++) EXPECT_EQ(GetProp(results[i].second), 0);
}
}
TEST(QueryPlan, ExpandOptional) {
Dbms dbms;
auto dba = dbms.active();