Remove GraphDbAccessor and storage types from Ast

Summary:
This diff removes the need for a database when parsing a query and
creating an Ast. Instead of storing storage::{Label,Property,EdgeType}
in Ast nodes, we store the name and an index into all of the names. This
allows for easy creation of a map from {Label,Property,EdgeType} index
into the concrete storage type. Obviously, this comes with a performance
penalty during execution, but it should be minor. The upside is that the
query/frontend minimally depends on storage (PropertyValue), which makes
writing tests easier as well as running them a lot faster (there is no
database setup). This is most noticeable in the ast_serialization test
which took a long time due to start up of a distributed database.

Reviewers: mtomic, llugovic

Reviewed By: mtomic

Subscribers: mferencevic, pullbot

Differential Revision: https://phabricator.memgraph.io/D1774
This commit is contained in:
Teon Banek 2019-01-14 14:41:37 +01:00
parent 0436bf77ce
commit b90375c3ae
54 changed files with 2339 additions and 1688 deletions

View File

@ -25,7 +25,8 @@ std::unordered_map<int16_t, int64_t> BfsRpcClients::CreateBfsSubcursors(
db_->WorkerId(), [&](int worker_id, auto &client) {
auto res = client.template Call<CreateBfsSubcursorRpc>(
dba->transaction_id(), direction, edge_types, filter_lambda,
symbol_table, evaluation_context);
symbol_table, evaluation_context.timestamp,
evaluation_context.parameters);
return std::make_pair(worker_id, res.member);
});
std::unordered_map<int16_t, int64_t> subcursor_ids;

View File

@ -67,16 +67,9 @@ cpp<#
Load(&${member}, ${reader}, ast_storage, &loaded_ast_uids);
cpp<#))
(symbol-table "query::SymbolTable" :capnp-type "Query.SymbolTable")
(evaluation-context "query::EvaluationContext"
:capnp-type "Query.EvaluationContext"
:capnp-save (lambda (builder member capnp-name)
#>cpp
query::SaveEvaluationContext(${member}, &${builder});
cpp<#)
:capnp-load (lambda (reader member capnp-name)
#>cpp
query::LoadEvaluationContext(${reader}, &${member});
cpp<#)))
(timestamp :int64_t)
(parameters "query::Parameters"
:capnp-type "Utils.Map(Utils.BoxInt64, Storage.PropertyValue)"))
(:serialize (:slk :load-args '((ast-storage "query::AstStorage *")))
(:capnp :load-args '((ast-storage "query::AstStorage *")))))
(:response ((member :int64_t))))

View File

@ -35,10 +35,16 @@ class BfsRpcServer {
}
dba = it->second.get();
}
query::EvaluationContext evaluation_context;
evaluation_context.timestamp = req.timestamp;
evaluation_context.parameters = req.parameters;
evaluation_context.properties =
query::NamesToProperties(ast_storage->properties_, dba);
evaluation_context.labels =
query::NamesToLabels(ast_storage->labels_, dba);
auto id = subcursor_storage_->Create(
dba, req.direction, req.edge_types, std::move(req.symbol_table),
std::move(ast_storage), req.filter_lambda,
std::move(req.evaluation_context));
std::move(ast_storage), req.filter_lambda, evaluation_context);
CreateBfsSubcursorRes res(id);
Save(res, res_builder);
});

View File

@ -11,17 +11,21 @@ namespace distributed {
ProduceRpcServer::OngoingProduce::OngoingProduce(
database::Worker *db, tx::TransactionId tx_id,
std::shared_ptr<query::plan::LogicalOperator> op,
query::SymbolTable symbol_table,
query::EvaluationContext evaluation_context,
const PlanConsumer::PlanPack &plan_pack, int64_t timestamp,
const query::Parameters &parameters,
std::vector<query::Symbol> pull_symbols)
: dba_(db->Access(tx_id)),
context_(*dba_),
pull_symbols_(std::move(pull_symbols)),
frame_(symbol_table.max_position()),
cursor_(op->MakeCursor(*dba_)) {
context_.symbol_table_ = std::move(symbol_table);
context_.evaluation_context_ = std::move(evaluation_context);
frame_(plan_pack.symbol_table.max_position()),
cursor_(plan_pack.plan->MakeCursor(*dba_)) {
context_.symbol_table_ = plan_pack.symbol_table;
context_.evaluation_context_.timestamp = timestamp;
context_.evaluation_context_.parameters = parameters;
context_.evaluation_context_.properties =
query::NamesToProperties(plan_pack.storage.properties_, dba_.get());
context_.evaluation_context_.labels =
query::NamesToLabels(plan_pack.storage.labels_, dba_.get());
}
std::pair<std::vector<query::TypedValue>, PullState>
@ -161,9 +165,8 @@ ProduceRpcServer::OngoingProduce &ProduceRpcServer::GetOngoingProduce(
auto &plan_pack = plan_consumer_.PlanForId(req.plan_id);
return ongoing_produces_
.emplace(std::piecewise_construct, std::forward_as_tuple(key_tuple),
std::forward_as_tuple(db_, req.tx_id, plan_pack.plan,
plan_pack.symbol_table,
req.evaluation_context, req.symbols))
std::forward_as_tuple(db_, req.tx_id, plan_pack, req.timestamp,
req.parameters, req.symbols))
.first->second;
}

View File

@ -43,9 +43,8 @@ class ProduceRpcServer {
class OngoingProduce {
public:
OngoingProduce(database::Worker *db, tx::TransactionId tx_id,
std::shared_ptr<query::plan::LogicalOperator> op,
query::SymbolTable symbol_table,
query::EvaluationContext evaluation_context,
const PlanConsumer::PlanPack &plan_pack, int64_t timestamp,
const query::Parameters &parameters,
std::vector<query::Symbol> pull_symbols);
/// Returns a vector of typed values (one for each `pull_symbol`), and an

View File

@ -222,16 +222,9 @@ to the appropriate value. Not used on side that generates the response.")
:capnp-load #'load-snapshot)
(plan-id :int64_t)
(command-id "tx::CommandId")
(evaluation-context "query::EvaluationContext"
:capnp-type "Query.EvaluationContext"
:capnp-save (lambda (builder member capnp-name)
#>cpp
query::SaveEvaluationContext(${member}, &${builder});
cpp<#)
:capnp-load (lambda (reader member capnp-name)
#>cpp
query::LoadEvaluationContext(${reader}, &${member});
cpp<#))
(timestamp :int64_t)
(parameters "query::Parameters"
:capnp-type "Utils.Map(Utils.BoxInt64, Storage.PropertyValue)")
(symbols "std::vector<query::Symbol>" :capnp-type "List(Sem.Symbol)")
(accumulate :bool)
(batch-size :int64_t)

View File

@ -13,21 +13,22 @@ utils::Future<PullData> PullRpcClients::Pull(
const query::EvaluationContext &evaluation_context,
const std::vector<query::Symbol> &symbols, bool accumulate,
int batch_size) {
return coordination_->ExecuteOnWorker<PullData>(worker_id, [
data_manager = data_manager_, dba, plan_id, command_id, evaluation_context,
symbols, accumulate, batch_size
](int worker_id, ClientPool &client_pool) {
auto load_pull_res = [data_manager, dba](const auto &res_reader) {
PullRes res;
Load(&res, res_reader, dba, data_manager);
return res;
};
auto result = client_pool.CallWithLoad<PullRpc>(
load_pull_res, dba->transaction_id(), dba->transaction().snapshot(),
plan_id, command_id, evaluation_context, symbols, accumulate,
batch_size, storage::SendVersions::BOTH);
return PullData{result.data.pull_state, std::move(result.data.frames)};
});
return coordination_->ExecuteOnWorker<PullData>(
worker_id, [data_manager = data_manager_, dba, plan_id, command_id,
evaluation_context, symbols, accumulate,
batch_size](int worker_id, ClientPool &client_pool) {
auto load_pull_res = [data_manager, dba](const auto &res_reader) {
PullRes res;
Load(&res, res_reader, dba, data_manager);
return res;
};
auto result = client_pool.CallWithLoad<PullRpc>(
load_pull_res, dba->transaction_id(), dba->transaction().snapshot(),
plan_id, command_id, evaluation_context.timestamp,
evaluation_context.parameters, symbols, accumulate, batch_size,
storage::SendVersions::BOTH);
return PullData{result.data.pull_state, std::move(result.data.frames)};
});
}
utils::Future<void> PullRpcClients::ResetCursor(database::GraphDbAccessor *dba,

View File

@ -1,6 +1,5 @@
#pragma once
#include "antlr4-runtime.h"
#include "database/graph_db_accessor.hpp"
#include "query/frontend/semantic/symbol_table.hpp"
#include "query/parameters.hpp"
@ -11,8 +10,34 @@ namespace query {
struct EvaluationContext {
int64_t timestamp{-1};
Parameters parameters;
/// All properties indexable via PropertyIx
std::vector<storage::Property> properties;
/// All labels indexable via LabelIx
std::vector<storage::Label> labels;
};
inline std::vector<storage::Property> NamesToProperties(
const std::vector<std::string> &property_names,
database::GraphDbAccessor *dba) {
std::vector<storage::Property> properties;
properties.reserve(property_names.size());
for (const auto &name : property_names) {
properties.push_back(dba->Property(name));
}
return properties;
}
inline std::vector<storage::Label> NamesToLabels(
const std::vector<std::string> &label_names,
database::GraphDbAccessor *dba) {
std::vector<storage::Label> labels;
labels.reserve(label_names.size());
for (const auto &name : label_names) {
labels.push_back(dba->Label(name));
}
return labels;
}
class Context {
public:
Context(const Context &) = delete;
@ -31,6 +56,9 @@ class Context {
plan::ProfilingStats *stats_root_{nullptr};
};
// TODO: Move this to somewhere in query/frontend. Currently, frontend includes
// this and therefore implicitly includes the whole database because of the
// includes at the top of this file.
struct ParsingContext {
bool is_query_cached = false;
};

View File

@ -46,6 +46,9 @@ class DistributedLogicalPlan final : public LogicalPlan {
const SymbolTable &GetSymbolTable() const override {
return plan_.symbol_table;
}
const AstStorage &GetAstStorage() const override {
return plan_.ast_storage;
}
private:
plan::DistributedPlan plan_;
@ -61,19 +64,21 @@ DistributedInterpreter::DistributedInterpreter(database::Master *db)
std::unique_ptr<LogicalPlan> DistributedInterpreter::MakeLogicalPlan(
CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
auto symbol_table = MakeSymbolTable(query);
auto planning_context = plan::MakePlanningContext(ast_storage, symbol_table,
query, vertex_counts);
auto planning_context = plan::MakePlanningContext(&ast_storage, &symbol_table,
query, &vertex_counts);
std::unique_ptr<plan::LogicalOperator> tmp_logical_plan;
double cost;
std::tie(tmp_logical_plan, cost) = plan::MakeLogicalPlan(
planning_context, parameters, FLAGS_query_cost_planner);
auto plan =
MakeDistributedPlan(*tmp_logical_plan, symbol_table, next_plan_id_);
&planning_context, parameters, FLAGS_query_cost_planner);
std::vector<storage::Property> properties_by_ix =
NamesToProperties(ast_storage.properties_, db_accessor);
auto plan = MakeDistributedPlan(ast_storage, *tmp_logical_plan, symbol_table,
next_plan_id_, properties_by_ix);
VLOG(10) << "[Interpreter] Created plan for distributed execution "
<< next_plan_id_ - 1;
return std::make_unique<DistributedLogicalPlan>(std::move(plan), cost,
@ -85,10 +90,10 @@ Interpreter::Results DistributedInterpreter::operator()(
const std::map<std::string, PropertyValue> &params,
bool in_explicit_transaction) {
AstStorage ast_storage;
Context execution_context(db_accessor);
Parameters parameters;
auto queries = StripAndParseQuery(query_string, &execution_context,
&ast_storage, &db_accessor, params);
auto queries = StripAndParseQuery(query_string, &parameters, &ast_storage,
&db_accessor, params);
ParsedQuery &parsed_query = queries.second;
if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {

View File

@ -10,24 +10,8 @@
#include "query/interpret/awesome_memgraph_functions.hpp"
#include "query/typed_value.hpp"
#include "storage/common/types/property_value.hpp"
#include "storage/common/types/types.hpp"
#include "utils/typeinfo.hpp"
// Hash function for the key in pattern atom property maps.
namespace std {
template <>
struct hash<std::pair<std::string, storage::Property>> {
size_t operator()(
const std::pair<std::string, storage::Property> &pair) const {
return string_hash(pair.first) ^ property_hash(pair.second);
};
private:
std::hash<std::string> string_hash{};
std::hash<storage::Property> property_hash{};
};
} // namespace std
namespace database {
class GraphDbAccessor;
}
@ -42,9 +26,6 @@ cpp<#
(lcp:capnp-import 'utils "/utils/serialization.capnp")
(lcp:capnp-type-conversion "PropertyValue" "Storage.PropertyValue")
(lcp:capnp-type-conversion "storage::Label" "Storage.Label")
(lcp:capnp-type-conversion "storage::EdgeType" "Storage.EdgeType")
(lcp:capnp-type-conversion "storage::Property" "Storage.Property")
(lcp:capnp-type-conversion "Symbol" "Symbol.Symbol")
(defun slk-save-ast-pointer (member)
@ -131,8 +112,8 @@ cpp<#
for (size_t i = 0;
i < size;
++i) {
std::pair<std::string, storage::Property> key;
slk::Load(&key, reader);
query::PropertyIx key;
slk::Load(&key, reader, storage);
auto *value = query::LoadAstPointer<query::Expression>(storage, reader, loaded_uids);
self->${member}.emplace(key, value);
}
@ -144,9 +125,7 @@ cpp<#
size_t i = 0;
for (const auto &entry : ${member}) {
auto key_builder = entries_builder[i].initKey();
key_builder.setFirst(entry.first.first);
auto prop_id_builder = key_builder.initSecond();
storage::Save(entry.first.second, &prop_id_builder);
Save(entry.first, &key_builder);
auto value_builder = entries_builder[i].initValue();
Save(*entry.second, &value_builder, saved_uids);
++i;
@ -156,14 +135,119 @@ cpp<#
(defun load-property-map (reader member capnp-name)
#>cpp
for (const auto &entry : ${reader}.getEntries()) {
std::string prop_name = entry.getKey().getFirst();
storage::Property prop_id;
storage::Load(&prop_id, entry.getKey().getSecond());
${member}.emplace(std::make_pair(prop_name, prop_id),
static_cast<Expression *>(Load(storage, entry.getValue(), loaded_uids)));
PropertyIx prop;
Load(&prop, entry.getKey(), storage);
${member}.emplace(prop, static_cast<Expression *>(
Load(storage, entry.getValue(), loaded_uids)));
}
cpp<#)
(defun capnp-load-name-ix (name-type)
(lambda (reader member capnp-name)
(declare (ignore capnp-name))
#>cpp
${member} = storage->Get${name-type}Ix(self->name).ix;
cpp<#))
(defun slk-load-name-ix (name-type)
(lambda (member)
#>cpp
self->${member} = storage->Get${name-type}Ix(self->name).ix;
cpp<#))
;; The following index structs serve as a decoupling point of AST from
;; concrete database types. All the names are collected in AstStorage, and can
;; be indexed through these instances. This means that we can create a vector
;; of concrete database types in the same order as all of the names and use the
;; same index to get the correct behaviour. Additionally, each index is
;; accompanied with the duplicated name found at the same index. The primary
;; reason for this duplication is simplifying the Clone and serialization API.
;; When an old index is being cloned or deserialized into a new AstStorage, we
;; request the new `ix` from the new AstStorage for the same `name`. If we
;; didn't do this, we would have to duplicate the old storage, which would
;; require having access to that storage. This in turn would complicate the
;; client code.
(lcp:define-struct label-ix ()
((name "std::string")
(ix :int64_t
:dont-save t
:slk-load (slk-load-name-ix "Label")
:capnp-init nil
:capnp-load (capnp-load-name-ix "Label")))
(:serialize (:slk :load-args '((storage "query::AstStorage *")))
(:capnp :load-args '((storage "AstStorage *")))))
(lcp:define-struct property-ix ()
((name "std::string")
(ix :int64_t
:dont-save t
:slk-load (slk-load-name-ix "Property")
:capnp-init nil
:capnp-load (capnp-load-name-ix "Property")))
(:serialize (:slk :load-args '((storage "query::AstStorage *")))
(:capnp :load-args '((storage "AstStorage *")))))
(lcp:define-struct edge-type-ix ()
((name "std::string")
(ix :int64_t
:dont-save t
:slk-load (slk-load-name-ix "EdgeType")
:capnp-init nil
:capnp-load (capnp-load-name-ix "EdgeType")))
(:serialize (:slk :load-args '((storage "query::AstStorage *")))
(:capnp :load-args '((storage "AstStorage *")))))
#>cpp
inline bool operator==(const LabelIx &a, const LabelIx &b) {
return a.ix == b.ix && a.name == b.name;
}
inline bool operator!=(const LabelIx &a, const LabelIx &b) { return !(a == b); }
inline bool operator==(const PropertyIx &a, const PropertyIx &b) {
return a.ix == b.ix && a.name == b.name;
}
inline bool operator!=(const PropertyIx &a, const PropertyIx &b) {
return !(a == b);
}
inline bool operator==(const EdgeTypeIx &a, const EdgeTypeIx &b) {
return a.ix == b.ix && a.name == b.name;
}
inline bool operator!=(const EdgeTypeIx &a, const EdgeTypeIx &b) {
return !(a == b);
}
cpp<#
(lcp:pop-namespace) ;; namespace query
#>cpp
namespace std {
template <>
struct hash<query::LabelIx> {
size_t operator()(const query::LabelIx &label) const { return label.ix; }
};
template <>
struct hash<query::PropertyIx> {
size_t operator()(const query::PropertyIx &prop) const { return prop.ix; }
};
template <>
struct hash<query::EdgeTypeIx> {
size_t operator()(const query::EdgeTypeIx &edge_type) const {
return edge_type.ix;
}
};
} // namespace std
cpp<#
(lcp:namespace query)
#>cpp
#define CLONE_BINARY_EXPRESSION \
auto Clone(AstStorage &storage) const->std::remove_const< \
@ -202,9 +286,37 @@ class AstStorage {
return ptr;
}
LabelIx GetLabelIx(const std::string &name) {
return LabelIx{name, FindOrAddName(name, &labels_)};
}
PropertyIx GetPropertyIx(const std::string &name) {
return PropertyIx{name, FindOrAddName(name, &properties_)};
}
EdgeTypeIx GetEdgeTypeIx(const std::string &name) {
return EdgeTypeIx{name, FindOrAddName(name, &edge_types_)};
}
std::vector<std::string> labels_;
std::vector<std::string> edge_types_;
std::vector<std::string> properties_;
// Public only for serialization access
std::vector<std::unique_ptr<Tree>> storage_;
int max_existing_uid_ = -1;
private:
int64_t FindOrAddName(const std::string &name,
std::vector<std::string> *names) {
for (int64_t i = 0; i < names->size(); ++i) {
if ((*names)[i] == name) {
return i;
}
}
names->push_back(name);
return names->size() - 1;
}
};
cpp<#
@ -713,10 +825,10 @@ cpp<#
(:serialize (:slk) (:capnp)))
(lcp:define-class map-literal (base-literal)
((elements "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
((elements "std::unordered_map<PropertyIx, Expression *>"
:slk-save #'slk-save-property-map
:slk-load #'slk-load-property-map
:capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
:capnp-type "Utils.Map(PropertyIx, Tree)"
:capnp-save #'save-property-map
:capnp-load #'load-property-map
:scope :public))
@ -736,8 +848,10 @@ cpp<#
MapLiteral *Clone(AstStorage &storage) const override {
auto *map = storage.Create<MapLiteral>();
for (auto pair : elements_)
map->elements_.emplace(pair.first, pair.second->Clone(storage));
for (auto pair : elements_) {
auto prop = storage.GetPropertyIx(pair.first.name);
map->elements_.emplace(prop, pair.second->Clone(storage));
}
return map;
}
cpp<#)
@ -745,8 +859,7 @@ cpp<#
#>cpp
explicit MapLiteral(int uid) : BaseLiteral(uid) {}
MapLiteral(int uid,
const std::unordered_map<std::pair<std::string, storage::Property>,
Expression *> &elements)
const std::unordered_map<PropertyIx, Expression *> &elements)
: BaseLiteral(uid), elements_(elements) {}
cpp<#)
(:private
@ -789,8 +902,11 @@ cpp<#
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(property-name "std::string" :scope :public)
(property "storage::Property" :scope :public))
(property "PropertyIx" :scope :public
:slk-load (lambda (member)
#>cpp
slk::Load(&self->${member}, reader, storage);
cpp<#)))
(:public
#>cpp
PropertyLookup() = default;
@ -806,23 +922,15 @@ cpp<#
PropertyLookup *Clone(AstStorage &storage) const override {
return storage.Create<PropertyLookup>(expression_->Clone(storage),
property_name_, property_);
storage.GetPropertyIx(property_.name));
}
cpp<#)
(:protected
#>cpp
PropertyLookup(int uid, Expression *expression,
const std::string &property_name, storage::Property property)
PropertyLookup(int uid, Expression *expression, PropertyIx property)
: Expression(uid),
expression_(expression),
property_name_(property_name),
property_(property) {}
PropertyLookup(int uid, Expression *expression,
const std::pair<std::string, storage::Property> &property)
: Expression(uid),
expression_(expression),
property_name_(property.first),
property_(property.second) {}
cpp<#)
(:private
#>cpp
@ -837,13 +945,23 @@ cpp<#
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(labels "std::vector<storage::Label>" :scope :public
:capnp-save (lcp:capnp-save-vector
"storage::capnp::Label"
"storage::Label")
(labels "std::vector<LabelIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"storage::capnp::Label"
"storage::Label")))
"capnp::LabelIx" "LabelIx"
"[storage](const auto &reader) {
LabelIx ix;
Load(&ix, reader, storage);
return ix;
}")))
(:public
#>cpp
LabelsTest() = default;
@ -858,13 +976,18 @@ cpp<#
}
LabelsTest *Clone(AstStorage &storage) const override {
return storage.Create<LabelsTest>(expression_->Clone(storage), labels_);
std::vector<LabelIx> new_labels;
new_labels.reserve(labels_.size());
for (const auto &label : labels_) {
new_labels.push_back(storage.GetLabelIx(label.name));
}
return storage.Create<LabelsTest>(expression_->Clone(storage), new_labels);
}
cpp<#)
(:protected
#>cpp
LabelsTest(int uid, Expression *expression,
const std::vector<storage::Label> &labels)
const std::vector<LabelIx> &labels)
: Expression(uid), expression_(expression), labels_(labels) {}
cpp<#)
(:private
@ -1337,17 +1460,27 @@ cpp<#
(:capnp :ignore-other-base-classes t)))
(lcp:define-class node-atom (pattern-atom)
((labels "std::vector<storage::Label>" :scope :public
:capnp-save (lcp:capnp-save-vector
"storage::capnp::Label"
"storage::Label")
((labels "std::vector<LabelIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"storage::capnp::Label"
"storage::Label"))
(properties "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
"capnp::LabelIx" "LabelIx"
"[storage](const auto &reader) {
LabelIx ix;
Load(&ix, reader, storage);
return ix;
}"))
(properties "std::unordered_map<PropertyIx, Expression *>"
:slk-save #'slk-save-property-map
:slk-load #'slk-load-property-map
:capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
:capnp-type "Utils.Map(PropertyIx, Tree)"
:capnp-save #'save-property-map
:capnp-load #'load-property-map
:scope :public))
@ -1367,9 +1500,13 @@ cpp<#
NodeAtom *Clone(AstStorage &storage) const override {
auto *node_atom = storage.Create<NodeAtom>(identifier_->Clone(storage));
node_atom->labels_ = labels_;
node_atom->labels_.reserve(labels_.size());
for (const auto &label : labels_) {
node_atom->labels_.push_back(storage.GetLabelIx(label.name));
}
for (auto property : properties_) {
node_atom->properties_[property.first] = property.second->Clone(storage);
auto prop = storage.GetPropertyIx(property.first.name);
node_atom->properties_[prop] = property.second->Clone(storage);
}
return node_atom;
}
@ -1387,18 +1524,28 @@ cpp<#
(lcp:define-class edge-atom (pattern-atom)
((type "Type" :initval "Type::SINGLE" :scope :public)
(direction "Direction" :initval "Direction::BOTH" :scope :public)
(edge-types "std::vector<storage::EdgeType>" :scope :public
:capnp-save (lcp:capnp-save-vector
"storage::capnp::EdgeType"
"storage::EdgeType")
(edge-types "std::vector<EdgeTypeIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"storage::capnp::EdgeType"
"storage::EdgeType"))
(properties "std::unordered_map<std::pair<std::string, storage::Property>, Expression *>"
"capnp::EdgeTypeIx" "EdgeTypeIx"
"[storage](const auto &reader) {
EdgeTypeIx ix;
Load(&ix, reader, storage);
return ix;
}"))
(properties "std::unordered_map<PropertyIx, Expression *>"
:scope :public
:slk-save #'slk-save-property-map
:slk-load #'slk-load-property-map
:capnp-type "Utils.Map(Utils.Pair(Text, Storage.Property), Tree)"
:capnp-type "Utils.Map(PropertyIx, Tree)"
:capnp-save #'save-property-map
:capnp-load #'load-property-map)
(lower-bound "Expression *" :initval "nullptr" :scope :public
@ -1505,9 +1652,13 @@ cpp<#
auto *edge_atom = storage.Create<EdgeAtom>(identifier_->Clone(storage));
edge_atom->direction_ = direction_;
edge_atom->type_ = type_;
edge_atom->edge_types_ = edge_types_;
edge_atom->edge_types_.reserve(edge_types_.size());
for (const auto &edge_type : edge_types_) {
edge_atom->edge_types_.push_back(storage.GetEdgeTypeIx(edge_type.name));
}
for (auto property : properties_) {
edge_atom->properties_[property.first] = property.second->Clone(storage);
auto prop = storage.GetPropertyIx(property.first.name);
edge_atom->properties_[prop] = property.second->Clone(storage);
}
edge_atom->lower_bound_ = CloneOpt(lower_bound_, storage);
edge_atom->upper_bound_ = CloneOpt(upper_bound_, storage);
@ -1541,7 +1692,7 @@ cpp<#
// Creates an edge atom for a SINGLE expansion with the given .
EdgeAtom(int uid, Identifier *identifier, Type type, Direction direction,
const std::vector<storage::EdgeType> &edge_types)
const std::vector<EdgeTypeIx> &edge_types)
: PatternAtom(uid, identifier),
type_(type),
direction_(direction),
@ -1847,10 +1998,28 @@ cpp<#
(lcp:define-class index-query (query)
((action "Action" :scope :public)
(label "storage::Label" :scope :public)
(properties "std::vector<storage::Property>" :scope :public
:capnp-save (lcp:capnp-save-vector "storage::capnp::Property" "storage::Property")
:capnp-load (lcp:capnp-load-vector "storage::capnp::Property" "storage::Property")))
(label "LabelIx" :scope :public
:slk-load (lambda (member)
#>cpp
slk::Load(&self->${member}, reader, storage);
cpp<#))
(properties "std::vector<PropertyIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"capnp::PropertyIx" "PropertyIx"
"[storage](const auto &reader) {
PropertyIx ix;
Load(&ix, reader, storage);
return ix;
}")))
(:public
(lcp:define-enum action
(create create-unique drop)
@ -1862,14 +2031,20 @@ cpp<#
DEFVISITABLE(QueryVisitor<void>);
IndexQuery *Clone(AstStorage &storage) const override {
return storage.Create<IndexQuery>(action_, label_, properties_);
std::vector<PropertyIx> new_properties;
new_properties.reserve(properties_.size());
for (const auto &prop : properties_) {
new_properties.push_back(storage.GetPropertyIx(prop.name));
}
return storage.Create<IndexQuery>(action_, storage.GetLabelIx(label_.name),
new_properties);
}
cpp<#)
(:protected
#>cpp
explicit IndexQuery(int uid) : Query(uid) {}
IndexQuery(int uid, Action action, storage::Label label,
std::vector<storage::Property> properties)
IndexQuery(int uid, Action action, LabelIx label,
std::vector<PropertyIx> properties)
: Query(uid), action_(action), label_(label), properties_(properties) {}
cpp<#)
(:private
@ -2330,13 +2505,23 @@ cpp<#
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Identifier *"))
(labels "std::vector<storage::Label>" :scope :public
:capnp-save (lcp:capnp-save-vector
"storage::capnp::Label"
"storage::Label")
(labels "std::vector<LabelIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"storage::capnp::Label"
"storage::Label")))
"capnp::LabelIx" "LabelIx"
"[storage](const auto &reader) {
LabelIx ix;
Load(&ix, reader, storage);
return ix;
}")))
(:public
#>cpp
SetLabels() = default;
@ -2349,14 +2534,19 @@ cpp<#
}
SetLabels *Clone(AstStorage &storage) const override {
return storage.Create<SetLabels>(identifier_->Clone(storage), labels_);
std::vector<LabelIx> new_labels;
new_labels.reserve(labels_.size());
for (const auto &label : labels_) {
new_labels.push_back(storage.GetLabelIx(label.name));
}
return storage.Create<SetLabels>(identifier_->Clone(storage), new_labels);
}
cpp<#)
(:protected
#>cpp
explicit SetLabels(int uid) : Clause(uid) {}
SetLabels(int uid, Identifier *identifier,
const std::vector<storage::Label> &labels)
const std::vector<LabelIx> &labels)
: Clause(uid), identifier_(identifier), labels_(labels) {}
cpp<#)
(:private
@ -2406,13 +2596,23 @@ cpp<#
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Identifier *"))
(labels "std::vector<storage::Label>" :scope :public
:capnp-save (lcp:capnp-save-vector
"storage::capnp::Label"
"storage::Label")
(labels "std::vector<LabelIx>" :scope :public
:slk-load (lambda (member)
#>cpp
size_t size = 0;
slk::Load(&size, reader);
self->${member}.resize(size);
for (size_t i = 0; i < size; ++i) {
slk::Load(&self->${member}[i], reader, storage);
}
cpp<#)
:capnp-load (lcp:capnp-load-vector
"storage::capnp::Label"
"storage::Label")))
"capnp::LabelIx" "LabelIx"
"[storage](const auto &reader) {
LabelIx ix;
Load(&ix, reader, storage);
return ix;
}")))
(:public
#>cpp
RemoveLabels() = default;
@ -2425,14 +2625,19 @@ cpp<#
}
RemoveLabels *Clone(AstStorage &storage) const override {
return storage.Create<RemoveLabels>(identifier_->Clone(storage), labels_);
std::vector<LabelIx> new_labels;
new_labels.reserve(labels_.size());
for (const auto &label : labels_) {
new_labels.push_back(storage.GetLabelIx(label.name));
}
return storage.Create<RemoveLabels>(identifier_->Clone(storage), new_labels);
}
cpp<#)
(:protected
#>cpp
explicit RemoveLabels(int uid) : Clause(uid) {}
RemoveLabels(int uid, Identifier *identifier,
const std::vector<storage::Label> &labels)
const std::vector<LabelIx> &labels)
: Clause(uid), identifier_(identifier), labels_(labels) {}
cpp<#)
(:private

View File

@ -12,9 +12,8 @@
#include <utility>
#include <vector>
#include "glog/logging.h"
#include <glog/logging.h>
#include "database/graph_db.hpp"
#include "query/common.hpp"
#include "query/exceptions.hpp"
#include "query/interpret/awesome_memgraph_functions.hpp"
@ -87,10 +86,9 @@ antlrcpp::Any CypherMainVisitor::visitCreateIndex(
MemgraphCypher::CreateIndexContext *ctx) {
auto *index_query = storage_->Create<IndexQuery>();
index_query->action_ = IndexQuery::Action::CREATE;
index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
std::pair<std::string, storage::Property> name_key =
ctx->propertyKeyName()->accept(this);
index_query->properties_ = {name_key.second};
index_query->label_ = AddLabel(ctx->labelName()->accept(this));
PropertyIx name_key = ctx->propertyKeyName()->accept(this);
index_query->properties_ = {name_key};
return index_query;
}
@ -98,12 +96,11 @@ antlrcpp::Any CypherMainVisitor::visitCreateUniqueIndex(
MemgraphCypher::CreateUniqueIndexContext *ctx) {
auto *index_query = storage_->Create<IndexQuery>();
index_query->action_ = IndexQuery::Action::CREATE_UNIQUE;
index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
index_query->label_ = AddLabel(ctx->labelName()->accept(this));
index_query->properties_.reserve(ctx->propertyKeyName().size());
for (const auto &prop_name : ctx->propertyKeyName()) {
std::pair<std::string, storage::Property> name_key =
prop_name->accept(this);
index_query->properties_.push_back(name_key.second);
PropertyIx name_key = prop_name->accept(this);
index_query->properties_.push_back(name_key);
}
return index_query;
}
@ -112,10 +109,9 @@ antlrcpp::Any CypherMainVisitor::visitDropIndex(
MemgraphCypher::DropIndexContext *ctx) {
auto *index_query = storage_->Create<IndexQuery>();
index_query->action_ = IndexQuery::Action::DROP;
std::pair<std::string, storage::Property> key =
ctx->propertyKeyName()->accept(this);
index_query->properties_ = {key.second};
index_query->label_ = dba_->Label(ctx->labelName()->accept(this));
PropertyIx key = ctx->propertyKeyName()->accept(this);
index_query->properties_ = {key};
index_query->label_ = AddLabel(ctx->labelName()->accept(this));
return index_query;
}
@ -746,24 +742,21 @@ antlrcpp::Any CypherMainVisitor::visitNodePattern(
anonymous_identifiers.push_back(&node->identifier_);
}
if (ctx->nodeLabels()) {
node->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
node->labels_ = ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
}
if (ctx->properties()) {
node->properties_ =
ctx->properties()
->accept(this)
.as<std::unordered_map<std::pair<std::string, storage::Property>,
Expression *>>();
node->properties_ = ctx->properties()
->accept(this)
.as<std::unordered_map<PropertyIx, Expression *>>();
}
return node;
}
antlrcpp::Any CypherMainVisitor::visitNodeLabels(
MemgraphCypher::NodeLabelsContext *ctx) {
std::vector<storage::Label> labels;
std::vector<LabelIx> labels;
for (auto *node_label : ctx->nodeLabel()) {
labels.push_back(dba_->Label(node_label->accept(this)));
labels.push_back(AddLabel(node_label->accept(this)));
}
return labels;
}
@ -783,11 +776,9 @@ antlrcpp::Any CypherMainVisitor::visitProperties(
antlrcpp::Any CypherMainVisitor::visitMapLiteral(
MemgraphCypher::MapLiteralContext *ctx) {
std::unordered_map<std::pair<std::string, storage::Property>, Expression *>
map;
std::unordered_map<PropertyIx, Expression *> map;
for (int i = 0; i < static_cast<int>(ctx->propertyKeyName().size()); ++i) {
std::pair<std::string, storage::Property> key =
ctx->propertyKeyName()[i]->accept(this);
PropertyIx key = ctx->propertyKeyName()[i]->accept(this);
Expression *value = ctx->expression()[i]->accept(this);
if (!map.insert({key, value}).second) {
throw SemanticException("Same key can't appear twice in a map literal.");
@ -806,8 +797,7 @@ antlrcpp::Any CypherMainVisitor::visitListLiteral(
antlrcpp::Any CypherMainVisitor::visitPropertyKeyName(
MemgraphCypher::PropertyKeyNameContext *ctx) {
const std::string key_name = visitChildren(ctx);
return std::make_pair(key_name, dba_->Property(key_name));
return AddProperty(visitChildren(ctx));
}
antlrcpp::Any CypherMainVisitor::visitSymbolicName(
@ -930,7 +920,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
edge->edge_types_ = ctx->relationshipDetail()
->relationshipTypes()
->accept(this)
.as<std::vector<storage::EdgeType>>();
.as<std::vector<EdgeTypeIx>>();
}
auto relationshipLambdas = relationshipDetail->relationshipLambda();
@ -1009,8 +999,7 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipPattern(
edge->properties_ =
properties[0]
->accept(this)
.as<std::unordered_map<std::pair<std::string, storage::Property>,
Expression *>>();
.as<std::unordered_map<PropertyIx, Expression *>>();
break;
}
default:
@ -1035,9 +1024,9 @@ antlrcpp::Any CypherMainVisitor::visitRelationshipLambda(
antlrcpp::Any CypherMainVisitor::visitRelationshipTypes(
MemgraphCypher::RelationshipTypesContext *ctx) {
std::vector<storage::EdgeType> types;
std::vector<EdgeTypeIx> types;
for (auto *edge_type : ctx->relTypeName()) {
types.push_back(dba_->EdgeType(edge_type->accept(this)));
types.push_back(AddEdgeType(edge_type->accept(this)));
}
return types;
}
@ -1296,7 +1285,7 @@ antlrcpp::Any CypherMainVisitor::visitExpression2a(
Expression *expression = ctx->expression2b()->accept(this);
if (ctx->nodeLabels()) {
auto labels =
ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
expression = storage_->Create<LabelsTest>(expression, labels);
}
return expression;
@ -1306,9 +1295,8 @@ antlrcpp::Any CypherMainVisitor::visitExpression2b(
MemgraphCypher::Expression2bContext *ctx) {
Expression *expression = ctx->atom()->accept(this);
for (auto *lookup : ctx->propertyLookup()) {
std::pair<std::string, storage::Property> key = lookup->accept(this);
auto property_lookup =
storage_->Create<PropertyLookup>(expression, key.first, key.second);
PropertyIx key = lookup->accept(this);
auto property_lookup = storage_->Create<PropertyLookup>(expression, key);
expression = property_lookup;
}
return expression;
@ -1444,8 +1432,7 @@ antlrcpp::Any CypherMainVisitor::visitLiteral(
return static_cast<Expression *>(storage_->Create<MapLiteral>(
ctx->mapLiteral()
->accept(this)
.as<std::unordered_map<std::pair<std::string, storage::Property>,
Expression *>>()));
.as<std::unordered_map<PropertyIx, Expression *>>()));
}
return visitChildren(ctx);
}
@ -1604,7 +1591,7 @@ antlrcpp::Any CypherMainVisitor::visitSetItem(
set_labels->identifier_ = storage_->Create<Identifier>(
ctx->variable()->accept(this).as<std::string>());
set_labels->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
return static_cast<Clause *>(set_labels);
}
@ -1631,7 +1618,7 @@ antlrcpp::Any CypherMainVisitor::visitRemoveItem(
remove_labels->identifier_ = storage_->Create<Identifier>(
ctx->variable()->accept(this).as<std::string>());
remove_labels->labels_ =
ctx->nodeLabels()->accept(this).as<std::vector<storage::Label>>();
ctx->nodeLabels()->accept(this).as<std::vector<LabelIx>>();
return static_cast<Clause *>(remove_labels);
}
@ -1639,9 +1626,9 @@ antlrcpp::Any CypherMainVisitor::visitPropertyExpression(
MemgraphCypher::PropertyExpressionContext *ctx) {
Expression *expression = ctx->atom()->accept(this);
for (auto *lookup : ctx->propertyLookup()) {
std::pair<std::string, storage::Property> key = lookup->accept(this);
PropertyIx key = lookup->accept(this);
auto property_lookup =
storage_->Create<PropertyLookup>(expression, key.first, key.second);
storage_->Create<PropertyLookup>(expression, key);
expression = property_lookup;
}
// It is guaranteed by grammar that there is at least one propertyLookup.
@ -1722,4 +1709,16 @@ antlrcpp::Any CypherMainVisitor::visitFilterExpression(
return 0;
}
LabelIx CypherMainVisitor::AddLabel(const std::string &name) {
return storage_->GetLabelIx(name);
}
PropertyIx CypherMainVisitor::AddProperty(const std::string &name) {
return storage_->GetPropertyIx(name);
}
EdgeTypeIx CypherMainVisitor::AddEdgeType(const std::string &name) {
return storage_->GetEdgeTypeIx(name);
}
} // namespace query::frontend

View File

@ -16,13 +16,11 @@ namespace query {
namespace frontend {
using antlropencypher::MemgraphCypher;
using query::Context;
class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
public:
explicit CypherMainVisitor(ParsingContext context, AstStorage *storage,
database::GraphDbAccessor *dba)
: context_(context), storage_(storage), dba_(dba) {}
explicit CypherMainVisitor(ParsingContext context, AstStorage *storage)
: context_(context), storage_(storage) {}
private:
Expression *CreateBinaryOperatorByToken(size_t token, Expression *e1,
@ -409,13 +407,13 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
MemgraphCypher::NodePatternContext *ctx) override;
/**
* @return vector<storage::Label>
* @return vector<LabelIx>
*/
antlrcpp::Any visitNodeLabels(
MemgraphCypher::NodeLabelsContext *ctx) override;
/**
* @return unordered_map<storage::Property, Expression*>
* @return unordered_map<PropertyIx, Expression*>
*/
antlrcpp::Any visitProperties(
MemgraphCypher::PropertiesContext *ctx) override;
@ -433,7 +431,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
MemgraphCypher::ListLiteralContext *ctx) override;
/**
* @return storage::Property
* @return PropertyIx
*/
antlrcpp::Any visitPropertyKeyName(
MemgraphCypher::PropertyKeyNameContext *ctx) override;
@ -488,7 +486,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
MemgraphCypher::RelationshipLambdaContext *ctx) override;
/**
* @return vector<storage::EdgeType>
* @return vector<EdgeTypeIx>
*/
antlrcpp::Any visitRelationshipTypes(
MemgraphCypher::RelationshipTypesContext *ctx) override;
@ -776,9 +774,12 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
const static std::string kAnonPrefix;
private:
LabelIx AddLabel(const std::string &name);
PropertyIx AddProperty(const std::string &name);
EdgeTypeIx AddEdgeType(const std::string &name);
ParsingContext context_;
AstStorage *storage_;
database::GraphDbAccessor *dba_;
// Set of identifiers from queries.
std::unordered_set<std::string> users_identifiers;

View File

@ -10,7 +10,8 @@ namespace {
class ExpressionPrettyPrinter : public ExpressionVisitor<void> {
public:
explicit ExpressionPrettyPrinter(std::ostream *out);
ExpressionPrettyPrinter(const AstStorage *storage, std::ostream *out)
: storage_(storage), out_(out) {}
// Unary operators
void Visit(NotOperator &op) override;
@ -56,58 +57,78 @@ class ExpressionPrettyPrinter : public ExpressionVisitor<void> {
void Visit(NamedExpression &op) override;
private:
const AstStorage *storage_;
std::ostream *out_;
// Declare all of the different `PrintObject` overloads upfront since they're
// mutually recursive. Without this, overload resolution depends on the
// ordering of the overloads within the source, which is quite fragile.
template <typename T>
void PrintObject(std::ostream *out, const T &arg);
void PrintObject(std::ostream *out, const std::string &str);
void PrintObject(std::ostream *out, Aggregation::Op op);
void PrintObject(std::ostream *out, Expression *expr);
void PrintObject(std::ostream *out, const PropertyValue &value);
template <typename T>
void PrintObject(std::ostream *out, const std::vector<T> &vec);
template <typename K, typename V>
void PrintObject(std::ostream *out, const std::map<K, V> &map);
template <typename T>
void PrintOperatorArgs(std::ostream *out, const T &arg) {
*out << " ";
PrintObject(out, arg);
*out << ")";
}
template <typename T, typename... Ts>
void PrintOperatorArgs(std::ostream *out, const T &arg, const Ts &... args) {
*out << " ";
PrintObject(out, arg);
PrintOperatorArgs(out, args...);
}
template <typename... Ts>
void PrintOperator(std::ostream *out, const std::string &name,
const Ts &... args) {
*out << "(" << name;
PrintOperatorArgs(out, args...);
}
};
// Declare all of the different `PrintObject` overloads upfront since they're
// mutually recursive. Without this, overload resolution depends on the ordering
// of the overloads within the source, which is quite fragile.
template <typename T>
void PrintObject(std::ostream *out, const T &arg);
void PrintObject(std::ostream *out, const std::string &str);
void PrintObject(std::ostream *out, Aggregation::Op op);
void PrintObject(std::ostream *out, Expression *expr);
void PrintObject(std::ostream *out, const PropertyValue &value);
template <typename T>
void PrintObject(std::ostream *out, const std::vector<T> &vec);
template <typename K, typename V>
void PrintObject(std::ostream *out, const std::map<K, V> &map);
void PrintObject(
std::ostream *out,
const std::unordered_map<std::pair<std::string, storage::Property>,
Expression *> &map);
template <typename T>
void PrintObject(std::ostream *out, const T &arg) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out, const T &arg) {
*out << arg;
}
void PrintObject(std::ostream *out, const std::string &str) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
const std::string &str) {
*out << utils::Escape(str);
}
void PrintObject(std::ostream *out, Aggregation::Op op) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
Aggregation::Op op) {
*out << Aggregation::OpToString(op);
}
void PrintObject(std::ostream *out, Expression *expr) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out, Expression *expr) {
if (expr) {
ExpressionPrettyPrinter printer{out};
ExpressionPrettyPrinter printer{storage_, out};
expr->Accept(printer);
} else {
*out << "<null>";
}
}
void PrintObject(std::ostream *out, const PropertyValue &value) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
const PropertyValue &value) {
switch (value.type()) {
case PropertyValue::Type::Null:
*out << "null";
@ -140,62 +161,27 @@ void PrintObject(std::ostream *out, const PropertyValue &value) {
}
template <typename T>
void PrintObject(std::ostream *out, const std::vector<T> &vec) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
const std::vector<T> &vec) {
*out << "[";
utils::PrintIterable(*out, vec, ", ", [](auto &stream, const auto &item) {
PrintObject(&stream, item);
utils::PrintIterable(*out, vec, ", ", [this](auto &stream, const auto &item) {
this->PrintObject(&stream, item);
});
*out << "]";
}
template <typename K, typename V>
void PrintObject(std::ostream *out, const std::map<K, V> &map) {
void ExpressionPrettyPrinter::PrintObject(std::ostream *out,
const std::map<K, V> &map) {
*out << "{";
utils::PrintIterable(*out, map, ", ", [](auto &stream, const auto &item) {
PrintObject(&stream, item.first);
utils::PrintIterable(*out, map, ", ", [this](auto &stream, const auto &item) {
this->PrintObject(&stream, item.first);
stream << ": ";
PrintObject(&stream, item.second);
this->PrintObject(&stream, item.second);
});
*out << "}";
}
void PrintObject(
std::ostream *out,
const std::unordered_map<std::pair<std::string, storage::Property>,
Expression *> &map) {
*out << "{";
utils::PrintIterable(*out, map, ", ", [](auto &stream, const auto &item) {
PrintObject(&stream, item.first.first);
stream << ": ";
PrintObject(&stream, item.second);
});
*out << "}";
}
template <typename T>
void PrintOperatorArgs(std::ostream *out, const T &arg) {
*out << " ";
PrintObject(out, arg);
*out << ")";
}
template <typename T, typename... Ts>
void PrintOperatorArgs(std::ostream *out, const T &arg, const Ts &... args) {
*out << " ";
PrintObject(out, arg);
PrintOperatorArgs(out, args...);
}
template <typename... Ts>
void PrintOperator(std::ostream *out, const std::string &name,
const Ts &... args) {
*out << "(" << name;
PrintOperatorArgs(out, args...);
}
ExpressionPrettyPrinter::ExpressionPrettyPrinter(std::ostream *out)
: out_(out) {}
#define UNARY_OPERATOR_VISIT(OP_NODE, OP_STR) \
void ExpressionPrettyPrinter::Visit(OP_NODE &op) { \
PrintOperator(out_, OP_STR, op.expression_); \
@ -247,7 +233,11 @@ void ExpressionPrettyPrinter::Visit(ListLiteral &op) {
}
void ExpressionPrettyPrinter::Visit(MapLiteral &op) {
PrintObject(out_, op.elements_);
std::map<std::string, Expression *> map;
for (const auto &kv : op.elements_) {
map[storage_->properties_[kv.first.ix]] = kv.second;
}
PrintObject(out_, map);
}
void ExpressionPrettyPrinter::Visit(LabelsTest &op) {
@ -294,7 +284,8 @@ void ExpressionPrettyPrinter::Visit(PrimitiveLiteral &op) {
}
void ExpressionPrettyPrinter::Visit(PropertyLookup &op) {
PrintOperator(out_, "PropertyLookup", op.expression_, op.property_name_);
const auto &prop_name = storage_->properties_[op.property_.ix];
PrintOperator(out_, "PropertyLookup", op.expression_, prop_name);
}
void ExpressionPrettyPrinter::Visit(ParameterLookup &op) {
@ -307,13 +298,15 @@ void ExpressionPrettyPrinter::Visit(NamedExpression &op) {
} // namespace
void PrintExpression(Expression *expr, std::ostream *out) {
ExpressionPrettyPrinter printer{out};
void PrintExpression(const AstStorage &storage, Expression *expr,
std::ostream *out) {
ExpressionPrettyPrinter printer{&storage, out};
expr->Accept(printer);
}
void PrintExpression(NamedExpression *expr, std::ostream *out) {
ExpressionPrettyPrinter printer{out};
void PrintExpression(const AstStorage &storage, NamedExpression *expr,
std::ostream *out) {
ExpressionPrettyPrinter printer{&storage, out};
expr->Accept(printer);
}

View File

@ -6,7 +6,9 @@
namespace query {
void PrintExpression(Expression *expr, std::ostream *out);
void PrintExpression(NamedExpression *expr, std::ostream *out);
void PrintExpression(const AstStorage &storage, Expression *expr,
std::ostream *out);
void PrintExpression(const AstStorage &storage, NamedExpression *expr,
std::ostream *out);
} // namespace query

View File

@ -270,14 +270,14 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
return TypedValue::Null;
case TypedValue::Type::Vertex:
return expression_result.Value<VertexAccessor>().PropsAt(
property_lookup.property_);
GetProperty(property_lookup.property_));
case TypedValue::Type::Edge:
return expression_result.Value<EdgeAccessor>().PropsAt(
property_lookup.property_);
GetProperty(property_lookup.property_));
case TypedValue::Type::Map: {
auto &map =
expression_result.Value<std::map<std::string, TypedValue>>();
auto found = map.find(property_lookup.property_name_);
auto found = map.find(property_lookup.property_.name);
if (found == map.end()) return TypedValue::Null;
return found->second;
}
@ -295,7 +295,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
case TypedValue::Type::Vertex: {
auto vertex = expression_result.Value<VertexAccessor>();
for (const auto label : labels_test.labels_) {
if (!vertex.has_label(label)) {
if (!vertex.has_label(GetLabel(label))) {
return false;
}
}
@ -323,7 +323,7 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
TypedValue Visit(MapLiteral &literal) override {
std::map<std::string, TypedValue> result;
for (const auto &pair : literal.elements_)
result.emplace(pair.first.first, pair.second->Accept(*this));
result.emplace(pair.first.name, pair.second->Accept(*this));
return result;
}
@ -481,6 +481,14 @@ class ExpressionEvaluator : public ExpressionVisitor<TypedValue> {
}
private:
storage::Property GetProperty(PropertyIx prop) {
return ctx_->properties[prop.ix];
}
storage::Label GetLabel(LabelIx label) {
return ctx_->labels[label.ix];
}
// If the given TypedValue contains accessors, switch them to New or Old,
// depending on use_new_ flag.
void SwitchAccessors(TypedValue &value) {

View File

@ -42,6 +42,7 @@ class SingleNodeLogicalPlan final : public LogicalPlan {
const plan::LogicalOperator &GetRoot() const override { return *root_; }
double GetCost() const override { return cost_; }
const SymbolTable &GetSymbolTable() const override { return symbol_table_; }
const AstStorage &GetAstStorage() const override { return storage_; }
private:
std::unique_ptr<plan::LogicalOperator> root_;
@ -70,13 +71,19 @@ TypedValue EvaluateOptionalExpression(Expression *expression,
}
Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
const EvaluationContext &evaluation_context,
const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
// Empty frame for evaluation of password expression. This is OK since
// password should be either null or string literal and it's evaluation
// should not depend on frame.
Frame frame(0);
SymbolTable symbol_table;
EvaluationContext evaluation_context;
evaluation_context.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
evaluation_context.parameters = parameters;
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
@ -355,12 +362,18 @@ Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
Callback HandleStreamQuery(StreamQuery *stream_query,
integrations::kafka::Streams *streams,
const EvaluationContext &evaluation_context,
const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
// Empty frame and symbol table for evaluation of expressions. This is OK
// since all expressions should be literals or parameter lookups.
Frame frame(0);
SymbolTable symbol_table;
EvaluationContext evaluation_context;
evaluation_context.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
evaluation_context.parameters = parameters;
ExpressionEvaluator eval(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
@ -513,8 +526,12 @@ Callback HandleIndexQuery(IndexQuery *index_query,
std::function<void()> invalidate_plan_cache,
database::GraphDbAccessor *db_accessor) {
auto action = index_query->action_;
auto label = index_query->label_;
auto properties = index_query->properties_;
auto label = db_accessor->Label(index_query->label_.name);
std::vector<storage::Property> properties;
properties.reserve(index_query->properties_.size());
for (const auto &prop : index_query->properties_) {
properties.push_back(db_accessor->Property(prop.name));
}
if (properties.size() > 1) {
throw utils::NotYetImplemented("index on multiple properties");
@ -574,12 +591,12 @@ Interpreter::Results Interpreter::operator()(
AstStorage ast_storage;
Context execution_context(db_accessor);
auto &evaluation_context = execution_context.evaluation_context_;
Parameters parameters;
std::map<std::string, TypedValue> summary;
utils::Timer parsing_timer;
auto queries = StripAndParseQuery(query_string, &execution_context,
&ast_storage, &db_accessor, params);
auto queries = StripAndParseQuery(query_string, &parameters, &ast_storage,
&db_accessor, params);
StrippedQuery &stripped_query = queries.first;
ParsedQuery &parsed_query = queries.second;
auto parsing_time = parsing_timer.Elapsed();
@ -600,15 +617,12 @@ Interpreter::Results Interpreter::operator()(
if (auto *cypher_query = dynamic_cast<CypherQuery *>(parsed_query.query)) {
plan = CypherQueryToPlan(stripped_query.hash(), cypher_query,
std::move(ast_storage),
evaluation_context.parameters, &db_accessor);
std::move(ast_storage), parameters, &db_accessor);
auto planning_time = planning_timer.Elapsed();
summary["planning_time"] = planning_time.count();
summary["cost_estimate"] = plan->cost();
execution_context.symbol_table_ = plan->symbol_table();
auto output_symbols =
plan->plan().OutputSymbols(execution_context.symbol_table_);
auto output_symbols = plan->plan().OutputSymbols(plan->symbol_table());
std::vector<std::string> header;
for (const auto &symbol : output_symbols) {
@ -620,11 +634,8 @@ Interpreter::Results Interpreter::operator()(
.first);
}
auto cursor = plan->plan().MakeCursor(db_accessor);
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, header, summary,
parsed_query.required_privileges);
return Results(&db_accessor, parameters, plan, output_symbols, header,
summary, parsed_query.required_privileges);
}
if (auto *explain_query = dynamic_cast<ExplainQuery *>(parsed_query.query)) {
@ -636,9 +647,9 @@ Interpreter::Results Interpreter::operator()(
auto cypher_query_hash =
fnv(stripped_query.query().substr(kExplainQueryStart.size()));
std::shared_ptr<CachedPlan> cypher_query_plan = CypherQueryToPlan(
cypher_query_hash, explain_query->cypher_query_, std::move(ast_storage),
evaluation_context.parameters, &db_accessor);
std::shared_ptr<CachedPlan> cypher_query_plan =
CypherQueryToPlan(cypher_query_hash, explain_query->cypher_query_,
std::move(ast_storage), parameters, &db_accessor);
std::stringstream printed_plan;
PrettyPrintPlan(db_accessor, &cypher_query_plan->plan(), &printed_plan);
@ -649,27 +660,23 @@ Interpreter::Results Interpreter::operator()(
printed_plan_rows.push_back(std::vector<TypedValue>{row});
}
auto query_plan_symbol =
execution_context.symbol_table_.CreateSymbol("QUERY PLAN", false);
SymbolTable symbol_table;
auto query_plan_symbol = symbol_table.CreateSymbol("QUERY PLAN", false);
std::vector<Symbol> output_symbols{query_plan_symbol};
std::vector<std::string> header{query_plan_symbol.name()};
auto output_plan =
std::make_unique<plan::OutputTable>(output_symbols, printed_plan_rows);
plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
std::move(output_plan), 0.0, AstStorage{},
execution_context.symbol_table_));
std::move(output_plan), 0.0, AstStorage{}, symbol_table));
auto planning_time = planning_timer.Elapsed();
summary["planning_time"] = planning_time.count();
execution_context.symbol_table_ = plan->symbol_table();
auto cursor = plan->plan().MakeCursor(db_accessor);
std::vector<std::string> header{query_plan_symbol.name()};
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, header, summary,
parsed_query.required_privileges);
return Results(&db_accessor, parameters, plan, output_symbols, header,
summary, parsed_query.required_privileges);
}
if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {
@ -689,22 +696,20 @@ Interpreter::Results Interpreter::operator()(
auto cypher_query_hash =
fnv(stripped_query.query().substr(kProfileQueryStart.size()));
auto cypher_query_plan = CypherQueryToPlan(
cypher_query_hash, profile_query->cypher_query_, std::move(ast_storage),
evaluation_context.parameters, &db_accessor);
auto cypher_query_plan =
CypherQueryToPlan(cypher_query_hash, profile_query->cypher_query_,
std::move(ast_storage), parameters, &db_accessor);
// Copy the symbol table and add our own symbols (used by the `OutputTable`
// operator below)
execution_context.symbol_table_ = cypher_query_plan->symbol_table();
SymbolTable symbol_table(cypher_query_plan->symbol_table());
auto operator_symbol =
execution_context.symbol_table_.CreateSymbol("OPERATOR", false);
auto actual_hits_symbol =
execution_context.symbol_table_.CreateSymbol("ACTUAL HITS", false);
auto operator_symbol = symbol_table.CreateSymbol("OPERATOR", false);
auto actual_hits_symbol = symbol_table.CreateSymbol("ACTUAL HITS", false);
auto relative_time_symbol =
execution_context.symbol_table_.CreateSymbol("RELATIVE TIME", false);
symbol_table.CreateSymbol("RELATIVE TIME", false);
auto absolute_time_symbol =
execution_context.symbol_table_.CreateSymbol("ABSOLUTE TIME", false);
symbol_table.CreateSymbol("ABSOLUTE TIME", false);
std::vector<Symbol> output_symbols = {operator_symbol, actual_hits_symbol,
relative_time_symbol,
@ -714,9 +719,9 @@ Interpreter::Results Interpreter::operator()(
relative_time_symbol.name(), absolute_time_symbol.name()};
auto output_plan = std::make_unique<plan::OutputTable>(
output_symbols,
[cypher_query_plan, &db_accessor](Frame *frame, Context *context) {
auto cursor = cypher_query_plan->plan().MakeCursor(db_accessor);
output_symbols, [cypher_query_plan](Frame *frame, Context *context) {
auto cursor =
cypher_query_plan->plan().MakeCursor(context->db_accessor_);
// Pull everything to profile the execution
utils::Timer timer;
@ -726,18 +731,14 @@ Interpreter::Results Interpreter::operator()(
});
plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
std::move(output_plan), 0.0, AstStorage{},
execution_context.symbol_table_));
std::move(output_plan), 0.0, AstStorage{}, symbol_table));
auto planning_time = planning_timer.Elapsed();
summary["planning_time"] = planning_time.count();
execution_context.is_profile_query_ = true;
auto cursor = plan->plan().MakeCursor(db_accessor);
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, header, summary,
parsed_query.required_privileges);
return Results(&db_accessor, parameters, plan, output_symbols, header,
summary, parsed_query.required_privileges,
/* is_profile_query */ true);
}
Callback callback;
@ -758,40 +759,36 @@ Interpreter::Results Interpreter::operator()(
if (in_explicit_transaction) {
throw UserModificationInMulticommandTxException();
}
callback =
HandleAuthQuery(auth_query, auth_, evaluation_context, &db_accessor);
callback = HandleAuthQuery(auth_query, auth_, parameters, &db_accessor);
} else if (auto *stream_query =
dynamic_cast<StreamQuery *>(parsed_query.query)) {
if (in_explicit_transaction) {
throw StreamClauseInMulticommandTxException();
}
callback = HandleStreamQuery(stream_query, kafka_streams_,
evaluation_context, &db_accessor);
callback = HandleStreamQuery(stream_query, kafka_streams_, parameters,
&db_accessor);
} else {
LOG(FATAL) << "Should not get here -- unknown query type!";
}
SymbolTable symbol_table;
std::vector<Symbol> output_symbols;
for (const auto &column : callback.header) {
output_symbols.emplace_back(
execution_context.symbol_table_.CreateSymbol(column, "false"));
output_symbols.emplace_back(symbol_table.CreateSymbol(column, "false"));
}
plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
std::make_unique<plan::OutputTable>(
output_symbols,
[fn = callback.fn](Frame *, Context *) { return fn(); }),
0.0, AstStorage{}, execution_context.symbol_table_));
0.0, AstStorage{}, symbol_table));
auto planning_time = planning_timer.Elapsed();
summary["planning_time"] = planning_time.count();
summary["cost_estimate"] = 0.0;
auto cursor = plan->plan().MakeCursor(db_accessor);
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, callback.header, summary,
parsed_query.required_privileges);
return Results(&db_accessor, parameters, plan, output_symbols,
callback.header, summary, parsed_query.required_privileges);
}
std::shared_ptr<Interpreter::CachedPlan> Interpreter::CypherQueryToPlan(
@ -825,7 +822,7 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
return std::make_unique<frontend::opencypher::Parser>(original_query);
}();
// Convert antlr4 AST into Memgraph AST.
frontend::CypherMainVisitor visitor(context, ast_storage, db_accessor);
frontend::CypherMainVisitor visitor(context, ast_storage);
visitor.visit(parser->tree());
return ParsedQuery{visitor.query(),
query::GetRequiredPrivileges(visitor.query())};
@ -855,8 +852,7 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
}();
// Convert antlr4 AST into Memgraph AST.
AstStorage cached_ast_storage;
frontend::CypherMainVisitor visitor(context, &cached_ast_storage,
db_accessor);
frontend::CypherMainVisitor visitor(context, &cached_ast_storage);
visitor.visit(parser->tree());
CachedQuery cached_query{std::move(cached_ast_storage), visitor.query(),
query::GetRequiredPrivileges(visitor.query())};
@ -865,31 +861,28 @@ Interpreter::ParsedQuery Interpreter::ParseQuery(
ast_cache_accessor.insert(stripped_query_hash, std::move(cached_query))
.first;
}
ast_storage->properties_ = ast_it->second.ast_storage.properties_;
ast_storage->labels_ = ast_it->second.ast_storage.labels_;
ast_storage->edge_types_ = ast_it->second.ast_storage.edge_types_;
return ParsedQuery{ast_it->second.query->Clone(*ast_storage),
ast_it->second.required_privileges};
}
std::pair<StrippedQuery, Interpreter::ParsedQuery>
Interpreter::StripAndParseQuery(
const std::string &query_string, Context *execution_context,
const std::string &query_string, Parameters *parameters,
AstStorage *ast_storage, database::GraphDbAccessor *db_accessor,
const std::map<std::string, PropertyValue> &params) {
StrippedQuery stripped_query(query_string);
auto &evaluation_context = execution_context->evaluation_context_;
evaluation_context.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
evaluation_context.parameters = stripped_query.literals();
*parameters = stripped_query.literals();
for (const auto &param_pair : stripped_query.parameters()) {
auto param_it = params.find(param_pair.second);
if (param_it == params.end()) {
throw query::UnprovidedParameterError("Parameter ${} not provided.",
param_pair.second);
}
evaluation_context.parameters.Add(param_pair.first, param_it->second);
parameters->Add(param_pair.first, param_it->second);
}
ParsingContext parsing_context;
@ -904,15 +897,15 @@ Interpreter::StripAndParseQuery(
std::unique_ptr<LogicalPlan> Interpreter::MakeLogicalPlan(
CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
auto vertex_counts = plan::MakeVertexCountCache(db_accessor);
auto symbol_table = MakeSymbolTable(query);
auto planning_context = plan::MakePlanningContext(ast_storage, symbol_table,
query, vertex_counts);
auto planning_context = plan::MakePlanningContext(&ast_storage, &symbol_table,
query, &vertex_counts);
std::unique_ptr<plan::LogicalOperator> root;
double cost;
std::tie(root, cost) = plan::MakeLogicalPlan(planning_context, parameters,
std::tie(root, cost) = plan::MakeLogicalPlan(&planning_context, parameters,
FLAGS_query_cost_planner);
return std::make_unique<SingleNodeLogicalPlan>(
std::move(root), cost, std::move(ast_storage), std::move(symbol_table));

View File

@ -35,6 +35,7 @@ class LogicalPlan {
virtual const plan::LogicalOperator &GetRoot() const = 0;
virtual double GetCost() const = 0;
virtual const SymbolTable &GetSymbolTable() const = 0;
virtual const AstStorage &GetAstStorage() const = 0;
};
class Interpreter {
@ -46,6 +47,7 @@ class Interpreter {
const auto &plan() const { return plan_->GetRoot(); }
double cost() const { return plan_->GetCost(); }
const auto &symbol_table() const { return plan_->GetSymbolTable(); }
const auto &ast_storage() const { return plan_->GetAstStorage(); }
bool IsExpired() const {
return cache_timer_.Elapsed() >
@ -81,19 +83,33 @@ class Interpreter {
*/
class Results {
friend Interpreter;
Results(Context ctx, std::shared_ptr<CachedPlan> plan,
std::unique_ptr<query::plan::Cursor> cursor,
Results(database::GraphDbAccessor *db_accessor,
const query::Parameters &parameters,
std::shared_ptr<CachedPlan> plan,
std::vector<Symbol> output_symbols, std::vector<std::string> header,
std::map<std::string, TypedValue> summary,
std::vector<AuthQuery::Privilege> privileges)
: ctx_(std::move(ctx)),
std::vector<AuthQuery::Privilege> privileges,
bool is_profile_query = false)
: ctx_(*db_accessor),
plan_(plan),
cursor_(std::move(cursor)),
frame_(ctx_.symbol_table_.max_position()),
cursor_(plan_->plan().MakeCursor(*db_accessor)),
frame_(plan_->symbol_table().max_position()),
output_symbols_(output_symbols),
header_(header),
summary_(summary),
privileges_(std::move(privileges)) {}
privileges_(std::move(privileges)) {
ctx_.is_profile_query_ = is_profile_query;
ctx_.symbol_table_ = plan_->symbol_table();
ctx_.evaluation_context_.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
ctx_.evaluation_context_.parameters = parameters;
ctx_.evaluation_context_.properties =
NamesToProperties(plan_->ast_storage().properties_, db_accessor);
ctx_.evaluation_context_.labels =
NamesToLabels(plan_->ast_storage().labels_, db_accessor);
}
public:
Results(const Results &) = delete;
@ -185,7 +201,7 @@ class Interpreter {
protected:
std::pair<StrippedQuery, ParsedQuery> StripAndParseQuery(
const std::string &, Context *, AstStorage *ast_storage,
const std::string &, Parameters *, AstStorage *ast_storage,
database::GraphDbAccessor *,
const std::map<std::string, PropertyValue> &);

View File

@ -65,18 +65,18 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
using HierarchicalLogicalOperatorVisitor::PostVisit;
using HierarchicalLogicalOperatorVisitor::PreVisit;
CostEstimator(const TDbAccessor &db_accessor, const Parameters &parameters)
CostEstimator(TDbAccessor *db_accessor, const Parameters &parameters)
: db_accessor_(db_accessor), parameters(parameters) {}
bool PostVisit(ScanAll &) override {
cardinality_ *= db_accessor_.VerticesCount();
cardinality_ *= db_accessor_->VerticesCount();
// ScanAll performs some work for every element that is produced
IncrementCost(CostParam::kScanAll);
return true;
}
bool PostVisit(ScanAllByLabel &scan_all_by_label) override {
cardinality_ *= db_accessor_.VerticesCount(scan_all_by_label.label_);
cardinality_ *= db_accessor_->VerticesCount(scan_all_by_label.label_);
// ScanAll performs some work for every element that is produced
IncrementCost(CostParam::kScanAllByLabel);
return true;
@ -90,13 +90,13 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
double factor = 1.0;
if (property_value)
// get the exact influence based on ScanAll(label, property, value)
factor = db_accessor_.VerticesCount(
factor = db_accessor_->VerticesCount(
logical_op.label_, logical_op.property_, property_value.value());
else
// estimate the influence as ScanAll(label, property) * filtering
factor = db_accessor_.VerticesCount(logical_op.label_,
logical_op.property_) *
CardParam::kFilter;
factor =
db_accessor_->VerticesCount(logical_op.label_, logical_op.property_) *
CardParam::kFilter;
cardinality_ *= factor;
@ -114,12 +114,12 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
int64_t factor = 1;
if (upper || lower)
// if we have either Bound<PropertyValue>, use the value index
factor = db_accessor_.VerticesCount(logical_op.label_,
logical_op.property_, lower, upper);
factor = db_accessor_->VerticesCount(logical_op.label_,
logical_op.property_, lower, upper);
else
// no values, but we still have the label
factor =
db_accessor_.VerticesCount(logical_op.label_, logical_op.property_);
db_accessor_->VerticesCount(logical_op.label_, logical_op.property_);
// if we failed to take either bound from the op into account, then apply
// the filtering constant to the factor
@ -197,7 +197,7 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
double cardinality_{1};
// accessor used for cardinality estimates in ScanAll and ScanAllByLabel
const TDbAccessor &db_accessor_;
TDbAccessor *db_accessor_;
const Parameters &parameters;
void IncrementCost(double param) { cost_ += param * cardinality_; }
@ -231,7 +231,7 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
/** Returns the estimated cost of the given plan. */
template <class TDbAccessor>
double EstimatePlanCost(const TDbAccessor &db, const Parameters &parameters,
double EstimatePlanCost(TDbAccessor *db, const Parameters &parameters,
LogicalOperator &plan) {
CostEstimator<TDbAccessor> estimator(db, parameters);
plan.Accept(estimator);

View File

@ -139,7 +139,10 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
auto ident = storage_->Create<Identifier>(
scan.output_symbol_.name(), scan.output_symbol_.user_declared());
(*symbol_table_)[*ident] = scan.output_symbol_;
return storage_->Create<PropertyLookup>(ident, "", scan.property_);
// TODO: When this extraction of a filter is removed, also remove
// property_name from ScanAll operators.
return storage_->Create<PropertyLookup>(
ident, storage_->GetPropertyIx(scan.property_name_));
};
Expression *extracted_filter = nullptr;
std::shared_ptr<ScanAll> new_scan;
@ -163,7 +166,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
// Case 1.a)
new_scan = std::make_shared<ScanAllByLabelPropertyRange>(
scan.input(), scan.output_symbol_, scan.label_, scan.property_,
std::experimental::nullopt, scan.upper_bound_, scan.graph_view_);
scan.property_name_, std::experimental::nullopt, scan.upper_bound_,
scan.graph_view_);
}
}
if (upper_depends) {
@ -192,7 +196,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
// Case 1.a)
new_scan = std::make_shared<ScanAllByLabelPropertyRange>(
scan.input(), scan.output_symbol_, scan.label_, scan.property_,
scan.lower_bound_, std::experimental::nullopt, scan.graph_view_);
scan.property_name_, scan.lower_bound_,
std::experimental::nullopt, scan.graph_view_);
} else {
// Case 1.b)
new_scan = std::make_shared<ScanAllByLabel>(
@ -231,8 +236,8 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
auto ident = storage_->Create<Identifier>(
scan.output_symbol_.name(), scan.output_symbol_.user_declared());
(*symbol_table_)[*ident] = scan.output_symbol_;
auto prop_lookup =
storage_->Create<PropertyLookup>(ident, "", scan.property_);
auto prop_lookup = storage_->Create<PropertyLookup>(
ident, storage_->GetPropertyIx(scan.property_name_));
auto prop_equal =
storage_->Create<EqualOperator>(prop_lookup, scan.expression_);
auto parent = std::make_shared<Filter>(subtree, prop_equal);
@ -1630,9 +1635,12 @@ class DistributedPlanner : public HierarchicalLogicalOperatorVisitor {
} // namespace
DistributedPlan MakeDistributedPlan(const LogicalOperator &original_plan,
const SymbolTable &symbol_table,
std::atomic<int64_t> &next_plan_id) {
DistributedPlan MakeDistributedPlan(
const AstStorage &ast_storage, const LogicalOperator &original_plan,
const SymbolTable &symbol_table, std::atomic<int64_t> &next_plan_id,
// TODO: Remove this variable when we remove creating a `PropertyLookup`
// expression during distributed planning.
const std::vector<storage::Property> &properties_by_ix) {
DistributedPlan distributed_plan;
// If we will generate multiple worker plans, we will need to increment the
// next_plan_id for each one.

View File

@ -23,8 +23,9 @@ struct DistributedPlan {
};
/// Creates a `DistributedPlan` from a regular plan.
DistributedPlan MakeDistributedPlan(const LogicalOperator &plan,
const SymbolTable &symbol_table,
std::atomic<int64_t> &next_plan_id);
DistributedPlan MakeDistributedPlan(
const AstStorage &ast_storage, const LogicalOperator &plan,
const SymbolTable &symbol_table, std::atomic<int64_t> &next_plan_id,
const std::vector<storage::Property> &properties_by_ix);
} // namespace query::plan

View File

@ -346,11 +346,13 @@ std::unique_ptr<Cursor> ScanAllByLabel::MakeCursor(
ScanAllByLabelPropertyRange::ScanAllByLabelPropertyRange(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
storage::Label label, storage::Property property,
const std::string &property_name,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound, GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
label_(label),
property_(property),
property_name_(property_name),
lower_bound_(lower_bound),
upper_bound_(upper_bound) {
DCHECK(lower_bound_ || upper_bound_) << "Only one bound can be left out";
@ -397,11 +399,13 @@ std::unique_ptr<Cursor> ScanAllByLabelPropertyRange::MakeCursor(
ScanAllByLabelPropertyValue::ScanAllByLabelPropertyValue(
const std::shared_ptr<LogicalOperator> &input, Symbol output_symbol,
storage::Label label, storage::Property property, Expression *expression,
storage::Label label, storage::Property property,
const std::string &property_name, Expression *expression,
GraphView graph_view)
: ScanAll(input, output_symbol, graph_view),
label_(label),
property_(property),
property_name_(property_name),
expression_(expression) {
DCHECK(expression) << "Expression is not optional.";
}
@ -1833,8 +1837,9 @@ void Delete::DeleteCursor::Shutdown() { input_cursor_->Shutdown(); }
void Delete::DeleteCursor::Reset() { input_cursor_->Reset(); }
SetProperty::SetProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs, Expression *rhs)
: input_(input), lhs_(lhs), rhs_(rhs) {}
storage::Property property, PropertyLookup *lhs,
Expression *rhs)
: input_(input), property_(property), lhs_(lhs), rhs_(rhs) {}
ACCEPT_WITH_INPUT(SetProperty)
@ -1866,18 +1871,17 @@ bool SetProperty::SetPropertyCursor::Pull(Frame &frame, Context &context) {
switch (lhs.type()) {
case TypedValue::Type::Vertex:
PropsSetChecked(&lhs.Value<VertexAccessor>(), self_.lhs_->property_, rhs);
PropsSetChecked(&lhs.Value<VertexAccessor>(), self_.property_, rhs);
break;
case TypedValue::Type::Edge:
PropsSetChecked(&lhs.Value<EdgeAccessor>(), self_.lhs_->property_, rhs);
PropsSetChecked(&lhs.Value<EdgeAccessor>(), self_.property_, rhs);
break;
case TypedValue::Type::Null:
// Skip setting properties on Null (can occur in optional match).
break;
case TypedValue::Type::Map:
// Semantically modifying a map makes sense, but it's not supported due
// to
// all the copying we do (when PropertyValue -> TypedValue and in
// to all the copying we do (when PropertyValue -> TypedValue and in
// ExpressionEvaluator). So even though we set a map property here, that
// is never visible to the user and it's not stored.
// TODO: fix above described bug
@ -2041,8 +2045,8 @@ void SetLabels::SetLabelsCursor::Shutdown() { input_cursor_->Shutdown(); }
void SetLabels::SetLabelsCursor::Reset() { input_cursor_->Reset(); }
RemoveProperty::RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs)
: input_(input), lhs_(lhs) {}
storage::Property property, PropertyLookup *lhs)
: input_(input), property_(property), lhs_(lhs) {}
ACCEPT_WITH_INPUT(RemoveProperty)
@ -2075,7 +2079,7 @@ bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
switch (lhs.type()) {
case TypedValue::Type::Vertex:
try {
lhs.Value<VertexAccessor>().PropsErase(self_.lhs_->property_);
lhs.Value<VertexAccessor>().PropsErase(self_.property_);
} catch (const RecordDeletedError &) {
throw QueryRuntimeException(
"Trying to remove properties from a deleted node.");
@ -2083,7 +2087,7 @@ bool RemoveProperty::RemovePropertyCursor::Pull(Frame &frame,
break;
case TypedValue::Type::Edge:
try {
lhs.Value<EdgeAccessor>().PropsErase(self_.lhs_->property_);
lhs.Value<EdgeAccessor>().PropsErase(self_.property_);
} catch (const RecordDeletedError &) {
throw QueryRuntimeException(
"Trying to remove properties from a deleted edge.");

View File

@ -777,6 +777,7 @@ given label.
(lcp:define-class scan-all-by-label-property-range (scan-all)
((label "storage::Label" :scope :public)
(property "storage::Property" :scope :public)
(property-name "std::string" :scope :public)
(lower-bound "std::experimental::optional<Bound>" :scope :public
:capnp-type "Utils.Optional(Utils.Bound(Ast.Tree))"
:slk-save #'slk-save-optional-bound
@ -816,6 +817,7 @@ property value which is inside a range (inclusive or exlusive).
ScanAllByLabelPropertyRange(const std::shared_ptr<LogicalOperator> &input,
Symbol output_symbol, storage::Label label,
storage::Property property,
const std::string &property_name,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound,
GraphView graph_view = GraphView::OLD);
@ -829,6 +831,7 @@ property value which is inside a range (inclusive or exlusive).
(lcp:define-class scan-all-by-label-property-value (scan-all)
((label "storage::Label" :scope :public)
(property "storage::Property" :scope :public)
(property-name "std::string" :scope :public)
(expression "Expression *" :scope :public
:slk-save #'slk-save-ast-pointer
:slk-load (slk-load-ast-pointer "Expression")
@ -857,6 +860,7 @@ property value.
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);
@ -1386,6 +1390,7 @@ Has a flag for using DETACH DELETE when deleting vertices.")
:slk-load #'slk-load-operator-pointer
:capnp-save #'save-operator-pointer
:capnp-load #'load-operator-pointer)
(property "storage::Property" :scope :public)
(lhs "PropertyLookup *" :scope :public
:slk-save #'slk-save-ast-pointer
:slk-load (slk-load-ast-pointer "PropertyLookup")
@ -1406,7 +1411,8 @@ can be stored (a TypedValue that can be converted to PropertyValue).")
SetProperty() {}
SetProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs, Expression *rhs);
storage::Property property, PropertyLookup *lhs,
Expression *rhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
@ -1560,6 +1566,7 @@ It does NOT remove labels that are already set on that Vertex.")
:slk-load #'slk-load-operator-pointer
:capnp-save #'save-operator-pointer
:capnp-load #'load-operator-pointer)
(property "storage::Property" :scope :public)
(lhs "PropertyLookup *" :scope :public
:slk-save #'slk-save-ast-pointer
:slk-load (slk-load-ast-pointer "PropertyLookup")
@ -1572,7 +1579,7 @@ It does NOT remove labels that are already set on that Vertex.")
RemoveProperty() {}
RemoveProperty(const std::shared_ptr<LogicalOperator> &input,
PropertyLookup *lhs);
storage::Property property, PropertyLookup *lhs);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;

View File

@ -36,9 +36,10 @@ namespace plan {
template <template <class> class TPlanner, class TDbAccessor>
auto MakeLogicalPlanForSingleQuery(
std::vector<SingleQueryPart> single_query_parts,
PlanningContext<TDbAccessor> &context) {
context.bound_symbols.clear();
return TPlanner<decltype(context)>(context).Plan(single_query_parts);
PlanningContext<TDbAccessor> *context) {
context->bound_symbols.clear();
return TPlanner<PlanningContext<TDbAccessor>>(context).Plan(
single_query_parts);
}
/// Generates the LogicalOperator tree and returns the resulting plan.
@ -50,11 +51,11 @@ auto MakeLogicalPlanForSingleQuery(
/// @return pair consisting of the plan's first logical operator @c
/// LogicalOperator and the estimated cost of that plan
template <class TPlanningContext>
auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
auto MakeLogicalPlan(TPlanningContext *context, const Parameters &parameters,
bool use_variable_planner) {
auto query_parts = CollectQueryParts(context.symbol_table,
context.ast_storage, context.query);
auto &vertex_counts = context.db;
auto query_parts = CollectQueryParts(*context->symbol_table,
*context->ast_storage, context->query);
auto &vertex_counts = *context->db;
double total_cost = 0;
std::unique_ptr<LogicalOperator> last_op;
@ -66,7 +67,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
auto plans = MakeLogicalPlanForSingleQuery<VariableStartPlanner>(
query_part.single_query_parts, context);
for (auto plan : plans) {
auto cost = EstimatePlanCost(vertex_counts, parameters, *plan);
auto cost = EstimatePlanCost(&vertex_counts, parameters, *plan);
if (!op || cost < min_cost) {
// Plans are generated lazily and the current plan will disappear, so
// it's ok to move it.
@ -77,7 +78,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
} else {
op = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(
query_part.single_query_parts, context);
min_cost = EstimatePlanCost(vertex_counts, parameters, *op);
min_cost = EstimatePlanCost(&vertex_counts, parameters, *op);
}
total_cost += min_cost;
@ -86,7 +87,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
std::shared_ptr<LogicalOperator> curr_op(std::move(op));
std::shared_ptr<LogicalOperator> prev_op(std::move(last_op));
last_op = std::unique_ptr<LogicalOperator>(
impl::GenUnion(*union_, prev_op, curr_op, context.symbol_table));
impl::GenUnion(*union_, prev_op, curr_op, *context->symbol_table));
} else if (query_part.query_combinator) {
throw utils::NotYetImplemented("query combinator");
} else {
@ -97,7 +98,7 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
if (query_parts.distinct) {
std::shared_ptr<LogicalOperator> prev_op(std::move(last_op));
last_op = std::make_unique<Distinct>(
prev_op, prev_op->OutputSymbols(context.symbol_table));
prev_op, prev_op->OutputSymbols(*context->symbol_table));
}
return std::make_pair(std::move(last_op), total_cost);

View File

@ -152,7 +152,7 @@ auto SplitExpressionOnAnd(Expression *expression) {
} // namespace
PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
const Symbol &symbol, storage::Property property,
const Symbol &symbol, PropertyIx property,
Expression *value)
: symbol_(symbol), property_(property), value_(value) {
UsedSymbolsCollector collector(symbol_table);
@ -162,7 +162,7 @@ PropertyFilter::PropertyFilter(const SymbolTable &symbol_table,
PropertyFilter::PropertyFilter(
const SymbolTable &symbol_table, const Symbol &symbol,
storage::Property property,
PropertyIx property,
const std::experimental::optional<PropertyFilter::Bound> &lower_bound,
const std::experimental::optional<PropertyFilter::Bound> &upper_bound)
: symbol_(symbol),
@ -189,7 +189,7 @@ void Filters::EraseFilter(const FilterInfo &filter) {
all_filters_.end());
}
void Filters::EraseLabelFilter(const Symbol &symbol, storage::Label label) {
void Filters::EraseLabelFilter(const Symbol &symbol, LabelIx label) {
for (auto filter_it = all_filters_.begin();
filter_it != all_filters_.end();) {
if (filter_it->type != FilterInfo::Type::Label) {
@ -282,7 +282,7 @@ void Filters::CollectPatternFilters(Pattern &pattern, SymbolTable &symbol_table,
collector.symbols_};
// Store a PropertyFilter on the value of the property.
filter_info.property_filter.emplace(
symbol_table, symbol, prop_pair.first.second, prop_pair.second);
symbol_table, symbol, prop_pair.first, prop_pair.second);
all_filters_.emplace_back(filter_info);
}
};

View File

@ -80,15 +80,14 @@ class PropertyFilter {
public:
using Bound = ScanAllByLabelPropertyRange::Bound;
PropertyFilter(const SymbolTable &, const Symbol &, storage::Property,
Expression *);
PropertyFilter(const SymbolTable &, const Symbol &, storage::Property,
PropertyFilter(const SymbolTable &, const Symbol &, PropertyIx, Expression *);
PropertyFilter(const SymbolTable &, const Symbol &, PropertyIx,
const std::experimental::optional<Bound> &,
const std::experimental::optional<Bound> &);
/// Symbol whose property is looked up.
Symbol symbol_;
storage::Property property_;
PropertyIx property_;
/// True if the same symbol is used in expressions for value or bounds.
bool is_symbol_in_value_ = false;
/// Expression which when evaluated produces the value a property must
@ -113,7 +112,7 @@ struct FilterInfo {
/// Set of used symbols by the filter @c expression.
std::unordered_set<Symbol> used_symbols;
/// Labels for Type::Label filtering.
std::vector<storage::Label> labels;
std::vector<LabelIx> labels;
/// Property information for Type::Property filtering.
std::experimental::optional<PropertyFilter> property_filter;
};
@ -144,7 +143,7 @@ class Filters {
}
auto FilteredLabels(const Symbol &symbol) const {
std::unordered_set<storage::Label> labels;
std::unordered_set<LabelIx> labels;
for (const auto &filter : all_filters_) {
if (filter.type == FilterInfo::Type::Label &&
utils::Contains(filter.used_symbols, symbol)) {
@ -162,7 +161,7 @@ class Filters {
void EraseFilter(const FilterInfo &);
// Remove a label filter for symbol; may invalidate iterators.
void EraseLabelFilter(const Symbol &, storage::Label);
void EraseLabelFilter(const Symbol &, LabelIx);
// Returns a vector of FilterInfo for properties.
auto PropertyFilters(const Symbol &symbol) const {

View File

@ -21,58 +21,6 @@ namespace query::plan {
namespace {
/// Utility function for iterating pattern atoms and accumulating a result.
///
/// Each pattern is of the form `NodeAtom (, EdgeAtom, NodeAtom)*`. Therefore,
/// the `base` function is called on the first `NodeAtom`, while the `collect`
/// is called for the whole triplet. Result of the function is passed to the
/// next call. Final result is returned.
///
/// Example usage of counting edge atoms in the pattern.
///
/// auto base = [](NodeAtom *first_node) { return 0; };
/// auto collect = [](int accum, NodeAtom *prev_node, EdgeAtom *edge,
/// NodeAtom *node) {
/// return accum + 1;
/// };
/// int edge_count = ReducePattern<int>(pattern, base, collect);
///
// TODO: It might be a good idea to move this somewhere else, for easier usage
// in other files.
template <typename T>
auto ReducePattern(
Pattern &pattern, std::function<T(NodeAtom *)> base,
std::function<T(T, NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
auto atoms_it = pattern.atoms_.begin();
auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
DCHECK(current_node) << "First pattern atom is not a node";
auto last_res = base(current_node);
// Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
while (atoms_it != pattern.atoms_.end()) {
auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
DCHECK(edge) << "Expected an edge atom in pattern.";
DCHECK(atoms_it != pattern.atoms_.end())
<< "Edge atom should not end the pattern.";
auto prev_node = current_node;
current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
DCHECK(current_node) << "Expected a node atom in pattern.";
last_res = collect(std::move(last_res), prev_node, edge, current_node);
}
return last_res;
}
auto GenCreate(Create &create, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto last_op = std::move(input_op);
for (auto pattern : create.patterns_) {
last_op = impl::GenCreateForPattern(*pattern, std::move(last_op),
symbol_table, bound_symbols);
}
return last_op;
}
bool HasBoundFilterSymbols(const std::unordered_set<Symbol> &bound_symbols,
const FilterInfo &filter) {
for (const auto &symbol : filter.used_symbols) {
@ -612,110 +560,6 @@ std::unique_ptr<LogicalOperator> GenReturn(
return GenReturnBody(std::move(input_op), advance_command, body, accumulate);
}
std::unique_ptr<LogicalOperator> GenCreateForPattern(
Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto node_to_creation_info = [&](const NodeAtom &node) {
const auto &node_symbol = symbol_table.at(*node.identifier_);
std::vector<storage::Label> labels(node.labels_);
std::vector<std::pair<storage::Property, Expression *>> properties;
properties.reserve(node.properties_.size());
for (const auto &kv : node.properties_) {
properties.push_back({kv.first.second, kv.second});
}
return NodeCreationInfo{node_symbol, labels, properties};
};
auto base = [&](NodeAtom *node) -> std::unique_ptr<LogicalOperator> {
if (bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
auto node_info = node_to_creation_info(*node);
return std::make_unique<CreateNode>(std::move(input_op), node_info);
} else {
return std::move(input_op);
}
};
auto collect = [&](std::unique_ptr<LogicalOperator> last_op,
NodeAtom *prev_node, EdgeAtom *edge, NodeAtom *node) {
// Store the symbol from the first node as the input to CreateExpand.
const auto &input_symbol = symbol_table.at(*prev_node->identifier_);
// If the expand node was already bound, then we need to indicate this,
// so that CreateExpand only creates an edge.
bool node_existing = false;
if (!bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
node_existing = true;
}
const auto &edge_symbol = symbol_table.at(*edge->identifier_);
if (!bound_symbols.insert(edge_symbol).second) {
LOG(FATAL) << "Symbols used for created edges cannot be redeclared.";
}
auto node_info = node_to_creation_info(*node);
std::vector<std::pair<storage::Property, Expression *>> properties;
properties.reserve(edge->properties_.size());
for (const auto &kv : edge->properties_) {
properties.push_back({kv.first.second, kv.second});
}
CHECK(edge->edge_types_.size() == 1)
<< "Creating an edge with a single type should be required by syntax";
EdgeCreationInfo edge_info{edge_symbol, properties, edge->edge_types_[0],
edge->direction_};
return std::make_unique<CreateExpand>(
node_info, edge_info, std::move(last_op), input_symbol, node_existing);
};
auto last_op =
ReducePattern<std::unique_ptr<LogicalOperator>>(pattern, base, collect);
// If the pattern is named, append the path constructing logical operator.
if (pattern.identifier_->user_declared_) {
std::vector<Symbol> path_elements;
for (const PatternAtom *atom : pattern.atoms_)
path_elements.emplace_back(symbol_table.at(*atom->identifier_));
last_op = std::make_unique<ConstructNamedPath>(
std::move(last_op), symbol_table.at(*pattern.identifier_),
path_elements);
}
return last_op;
}
// Generate an operator for a clause which writes to the database. Ownership of
// input_op is transferred to the newly created operator. If the clause isn't
// handled, returns nullptr and input_op is left as is.
std::unique_ptr<LogicalOperator> HandleWriteClause(
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
if (auto *create = dynamic_cast<Create *>(clause)) {
return GenCreate(*create, std::move(input_op), symbol_table, bound_symbols);
} else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
return std::make_unique<plan::Delete>(std::move(input_op),
del->expressions_, del->detach_);
} else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
return std::make_unique<plan::SetProperty>(
std::move(input_op), set->property_lookup_, set->expression_);
} else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
auto op = set->update_ ? plan::SetProperties::Op::UPDATE
: plan::SetProperties::Op::REPLACE;
const auto &input_symbol = symbol_table.at(*set->identifier_);
return std::make_unique<plan::SetProperties>(
std::move(input_op), input_symbol, set->expression_, op);
} else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*set->identifier_);
return std::make_unique<plan::SetLabels>(std::move(input_op), input_symbol,
set->labels_);
} else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
return std::make_unique<plan::RemoveProperty>(std::move(input_op),
rem->property_lookup_);
} else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*rem->identifier_);
return std::make_unique<plan::RemoveLabels>(std::move(input_op),
input_symbol, rem->labels_);
}
return nullptr;
}
std::unique_ptr<LogicalOperator> GenWith(
With &with, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,

View File

@ -20,15 +20,15 @@ struct PlanningContext {
/// operators.
///
/// Newly created AST nodes may be added to reference existing symbols.
SymbolTable &symbol_table;
SymbolTable *symbol_table{nullptr};
/// @brief The storage is used to create new AST nodes for use in operators.
AstStorage &ast_storage;
AstStorage *ast_storage{nullptr};
/// @brief Cypher query to be planned
CypherQuery *query;
CypherQuery *query{nullptr};
/// @brief TDbAccessor, which may be used to get some information from the
/// database to generate better plans. The accessor is required only to live
/// long enough for the plan generation to finish.
const TDbAccessor &db;
TDbAccessor *db{nullptr};
/// @brief Symbol set is used to differentiate cycles in pattern matching.
/// During planning, symbols will be added as each operator produces values
/// for them. This way, the operator can be correctly initialized whether to
@ -39,8 +39,8 @@ struct PlanningContext {
};
template <class TDbAccessor>
auto MakePlanningContext(AstStorage &ast_storage, SymbolTable &symbol_table,
CypherQuery *query, const TDbAccessor &db) {
auto MakePlanningContext(AstStorage *ast_storage, SymbolTable *symbol_table,
CypherQuery *query, TDbAccessor *db) {
return PlanningContext<TDbAccessor>{symbol_table, ast_storage, query, db};
}
@ -74,6 +74,47 @@ std::unique_ptr<LogicalOperator> GenFilters(std::unique_ptr<LogicalOperator>,
const std::unordered_set<Symbol> &,
Filters &, AstStorage &);
/// Utility function for iterating pattern atoms and accumulating a result.
///
/// Each pattern is of the form `NodeAtom (, EdgeAtom, NodeAtom)*`. Therefore,
/// the `base` function is called on the first `NodeAtom`, while the `collect`
/// is called for the whole triplet. Result of the function is passed to the
/// next call. Final result is returned.
///
/// Example usage of counting edge atoms in the pattern.
///
/// auto base = [](NodeAtom *first_node) { return 0; };
/// auto collect = [](int accum, NodeAtom *prev_node, EdgeAtom *edge,
/// NodeAtom *node) {
/// return accum + 1;
/// };
/// int edge_count = ReducePattern<int>(pattern, base, collect);
///
// TODO: It might be a good idea to move this somewhere else, for easier usage
// in other files.
template <typename T>
auto ReducePattern(
Pattern &pattern, std::function<T(NodeAtom *)> base,
std::function<T(T, NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
auto atoms_it = pattern.atoms_.begin();
auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
DCHECK(current_node) << "First pattern atom is not a node";
auto last_res = base(current_node);
// Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
while (atoms_it != pattern.atoms_.end()) {
auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
DCHECK(edge) << "Expected an edge atom in pattern.";
DCHECK(atoms_it != pattern.atoms_.end())
<< "Edge atom should not end the pattern.";
auto prev_node = current_node;
current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
DCHECK(current_node) << "Expected a node atom in pattern.";
last_res = collect(std::move(last_res), prev_node, edge, current_node);
}
return last_res;
}
// For all given `named_paths` checks if all its symbols have been bound.
// If so, it creates a logical operator for named path generation, binds its
// symbol, removes that path from the collection of unhandled ones and returns
@ -88,14 +129,6 @@ std::unique_ptr<LogicalOperator> GenReturn(
SymbolTable &symbol_table, bool is_write,
const std::unordered_set<Symbol> &bound_symbols, AstStorage &storage);
std::unique_ptr<LogicalOperator> GenCreateForPattern(
Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
std::unique_ptr<LogicalOperator> HandleWriteClause(
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table, std::unordered_set<Symbol> &bound_symbols);
std::unique_ptr<LogicalOperator> GenWith(
With &with, std::unique_ptr<LogicalOperator> input_op,
SymbolTable &symbol_table, bool is_write,
@ -122,23 +155,23 @@ Expression *BoolJoin(AstStorage &storage, Expression *expr1,
template <class TPlanningContext>
class RuleBasedPlanner {
public:
explicit RuleBasedPlanner(TPlanningContext &context) : context_(context) {}
explicit RuleBasedPlanner(TPlanningContext *context) : context_(context) {}
/// @brief The result of plan generation is the root of the generated operator
/// tree.
using PlanResult = std::unique_ptr<LogicalOperator>;
/// @brief Generates the operator tree based on explicitly set rules.
PlanResult Plan(const std::vector<SingleQueryPart> &query_parts) {
auto &context = context_;
auto &context = *context_;
std::unique_ptr<LogicalOperator> input_op;
// Set to true if a query command writes to the database.
bool is_write = false;
for (const auto &query_part : query_parts) {
MatchContext match_ctx{query_part.matching, context.symbol_table,
MatchContext match_ctx{query_part.matching, *context.symbol_table,
context.bound_symbols};
input_op = PlanMatching(match_ctx, std::move(input_op));
for (const auto &matching : query_part.optional_matching) {
MatchContext opt_ctx{matching, context.symbol_table,
MatchContext opt_ctx{matching, *context.symbol_table,
context.bound_symbols};
auto match_op = PlanMatching(opt_ctx, nullptr);
if (match_op) {
@ -152,8 +185,8 @@ class RuleBasedPlanner {
<< "Unexpected Match in remaining clauses";
if (auto *ret = dynamic_cast<Return *>(clause)) {
input_op = impl::GenReturn(
*ret, std::move(input_op), context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
*ret, std::move(input_op), *context.symbol_table, is_write,
context.bound_symbols, *context.ast_storage);
} else if (auto *merge = dynamic_cast<query::Merge *>(clause)) {
input_op = GenMerge(*merge, std::move(input_op),
query_part.merge_matching[merge_id++]);
@ -162,18 +195,18 @@ class RuleBasedPlanner {
is_write = true;
} else if (auto *with = dynamic_cast<query::With *>(clause)) {
input_op = impl::GenWith(*with, std::move(input_op),
context.symbol_table, is_write,
context.bound_symbols, context.ast_storage);
*context.symbol_table, is_write,
context.bound_symbols, *context.ast_storage);
// WITH clause advances the command, so reset the flag.
is_write = false;
} else if (auto op = impl::HandleWriteClause(clause, input_op,
context.symbol_table,
context.bound_symbols)) {
} else if (auto op = HandleWriteClause(clause, input_op,
*context.symbol_table,
context.bound_symbols)) {
is_write = true;
input_op = std::move(op);
} else if (auto *unwind = dynamic_cast<query::Unwind *>(clause)) {
const auto &symbol =
context.symbol_table.at(*unwind->named_expression_);
context.symbol_table->at(*unwind->named_expression_);
context.bound_symbols.insert(symbol);
input_op = std::make_unique<plan::Unwind>(
std::move(input_op), unwind->named_expression_->expression_,
@ -187,15 +220,164 @@ class RuleBasedPlanner {
}
private:
TPlanningContext &context_;
TPlanningContext *context_;
struct LabelPropertyIndex {
storage::Label label;
LabelIx label;
// FilterInfo with PropertyFilter.
FilterInfo filter;
int64_t vertex_count;
};
storage::Label GetLabel(LabelIx label) {
return context_->db->Label(context_->ast_storage->labels_[label.ix]);
}
storage::Property GetProperty(PropertyIx prop) {
return context_->db->Property(context_->ast_storage->properties_[prop.ix]);
}
storage::EdgeType GetEdgeType(EdgeTypeIx edge_type) {
return context_->db->EdgeType(
context_->ast_storage->edge_types_[edge_type.ix]);
}
std::unique_ptr<LogicalOperator> GenCreate(
Create &create, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto last_op = std::move(input_op);
for (auto pattern : create.patterns_) {
last_op = GenCreateForPattern(*pattern, std::move(last_op), symbol_table,
bound_symbols);
}
return last_op;
}
std::unique_ptr<LogicalOperator> GenCreateForPattern(
Pattern &pattern, std::unique_ptr<LogicalOperator> input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
auto node_to_creation_info = [&](const NodeAtom &node) {
const auto &node_symbol = symbol_table.at(*node.identifier_);
std::vector<storage::Label> labels;
labels.reserve(node.labels_.size());
for (const auto &label : node.labels_) {
labels.push_back(GetLabel(label));
}
std::vector<std::pair<storage::Property, Expression *>> properties;
properties.reserve(node.properties_.size());
for (const auto &kv : node.properties_) {
properties.push_back({GetProperty(kv.first), kv.second});
}
return NodeCreationInfo{node_symbol, labels, properties};
};
auto base = [&](NodeAtom *node) -> std::unique_ptr<LogicalOperator> {
const auto &node_symbol = symbol_table.at(*node->identifier_);
if (bound_symbols.insert(node_symbol).second) {
auto node_info = node_to_creation_info(*node);
return std::make_unique<CreateNode>(std::move(input_op), node_info);
} else {
return std::move(input_op);
}
};
auto collect = [&](std::unique_ptr<LogicalOperator> last_op,
NodeAtom *prev_node, EdgeAtom *edge, NodeAtom *node) {
// Store the symbol from the first node as the input to CreateExpand.
const auto &input_symbol = symbol_table.at(*prev_node->identifier_);
// If the expand node was already bound, then we need to indicate this,
// so that CreateExpand only creates an edge.
bool node_existing = false;
if (!bound_symbols.insert(symbol_table.at(*node->identifier_)).second) {
node_existing = true;
}
const auto &edge_symbol = symbol_table.at(*edge->identifier_);
if (!bound_symbols.insert(edge_symbol).second) {
LOG(FATAL) << "Symbols used for created edges cannot be redeclared.";
}
auto node_info = node_to_creation_info(*node);
std::vector<std::pair<storage::Property, Expression *>> properties;
properties.reserve(edge->properties_.size());
for (const auto &kv : edge->properties_) {
properties.push_back({GetProperty(kv.first), kv.second});
}
CHECK(edge->edge_types_.size() == 1)
<< "Creating an edge with a single type should be required by syntax";
EdgeCreationInfo edge_info{edge_symbol, properties,
GetEdgeType(edge->edge_types_[0]),
edge->direction_};
return std::make_unique<CreateExpand>(node_info, edge_info,
std::move(last_op), input_symbol,
node_existing);
};
auto last_op = impl::ReducePattern<std::unique_ptr<LogicalOperator>>(
pattern, base, collect);
// If the pattern is named, append the path constructing logical operator.
if (pattern.identifier_->user_declared_) {
std::vector<Symbol> path_elements;
for (const PatternAtom *atom : pattern.atoms_)
path_elements.emplace_back(symbol_table.at(*atom->identifier_));
last_op = std::make_unique<ConstructNamedPath>(
std::move(last_op), symbol_table.at(*pattern.identifier_),
path_elements);
}
return last_op;
}
// Generate an operator for a clause which writes to the database. Ownership
// of input_op is transferred to the newly created operator. If the clause
// isn't handled, returns nullptr and input_op is left as is.
std::unique_ptr<LogicalOperator> HandleWriteClause(
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
if (auto *create = dynamic_cast<Create *>(clause)) {
return GenCreate(*create, std::move(input_op), symbol_table,
bound_symbols);
} else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
return std::make_unique<plan::Delete>(std::move(input_op),
del->expressions_, del->detach_);
} else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
return std::make_unique<plan::SetProperty>(
std::move(input_op), GetProperty(set->property_lookup_->property_),
set->property_lookup_, set->expression_);
} else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
auto op = set->update_ ? plan::SetProperties::Op::UPDATE
: plan::SetProperties::Op::REPLACE;
const auto &input_symbol = symbol_table.at(*set->identifier_);
return std::make_unique<plan::SetProperties>(
std::move(input_op), input_symbol, set->expression_, op);
} else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*set->identifier_);
std::vector<storage::Label> labels;
labels.reserve(set->labels_.size());
for (const auto &label : set->labels_) {
labels.push_back(GetLabel(label));
}
return std::make_unique<plan::SetLabels>(std::move(input_op),
input_symbol, labels);
} else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
return std::make_unique<plan::RemoveProperty>(
std::move(input_op), GetProperty(rem->property_lookup_->property_),
rem->property_lookup_);
} else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
const auto &input_symbol = symbol_table.at(*rem->identifier_);
std::vector<storage::Label> labels;
labels.reserve(rem->labels_.size());
for (const auto &label : rem->labels_) {
labels.push_back(GetLabel(label));
}
return std::make_unique<plan::RemoveLabels>(std::move(input_op),
input_symbol, labels);
}
return nullptr;
}
// Finds the label-property combination which has indexed the lowest amount of
// vertices. If the index cannot be found, nullopt is returned.
std::experimental::optional<LabelPropertyIndex> FindBestLabelPropertyIndex(
@ -213,8 +395,10 @@ class RuleBasedPlanner {
for (const auto &label : filters.FilteredLabels(symbol)) {
for (const auto &filter : filters.PropertyFilters(symbol)) {
const auto &property = filter.property_filter->property_;
if (context_.db.LabelPropertyIndexExists(label, property)) {
int64_t vertex_count = context_.db.VerticesCount(label, property);
if (context_->db->LabelPropertyIndexExists(GetLabel(label),
GetProperty(property))) {
int64_t vertex_count = context_->db->VerticesCount(
GetLabel(label), GetProperty(property));
if (!found || vertex_count < found->vertex_count) {
if (filter.property_filter->is_symbol_in_value_) {
// Skip filter expressions which use the symbol whose property
@ -234,15 +418,15 @@ class RuleBasedPlanner {
return found;
}
storage::Label FindBestLabelIndex(
const std::unordered_set<storage::Label> &labels) {
DCHECK(!labels.empty())
LabelIx FindBestLabelIndex(const std::unordered_set<LabelIx> &labels) {
CHECK(!labels.empty())
<< "Trying to find the best label without any labels.";
return *std::min_element(labels.begin(), labels.end(),
[this](const auto &label1, const auto &label2) {
return context_.db.VerticesCount(label1) <
context_.db.VerticesCount(label2);
});
return *std::min_element(
labels.begin(), labels.end(),
[this](const auto &label1, const auto &label2) {
return context_->db->VerticesCount(GetLabel(label1)) <
context_->db->VerticesCount(GetLabel(label2));
});
}
// Creates a ScanAll by the best possible index for the `node_symbol`. Best
@ -275,33 +459,35 @@ class RuleBasedPlanner {
filters.EraseLabelFilter(node_symbol, found_index->label);
if (prop_filter.lower_bound_ || prop_filter.upper_bound_) {
return std::make_unique<ScanAllByLabelPropertyRange>(
std::move(last_op), node_symbol, found_index->label,
prop_filter.property_, prop_filter.lower_bound_,
prop_filter.upper_bound_, match_ctx.graph_view);
std::move(last_op), node_symbol, GetLabel(found_index->label),
GetProperty(prop_filter.property_), prop_filter.property_.name,
prop_filter.lower_bound_, prop_filter.upper_bound_,
match_ctx.graph_view);
} else {
DCHECK(prop_filter.value_) << "Property filter should either have "
"bounds or a value expression.";
return std::make_unique<ScanAllByLabelPropertyValue>(
std::move(last_op), node_symbol, found_index->label,
prop_filter.property_, prop_filter.value_, match_ctx.graph_view);
std::move(last_op), node_symbol, GetLabel(found_index->label),
GetProperty(prop_filter.property_), prop_filter.property_.name,
prop_filter.value_, match_ctx.graph_view);
}
}
auto label = FindBestLabelIndex(labels);
if (max_vertex_count &&
context_.db.VerticesCount(label) > *max_vertex_count) {
context_->db->VerticesCount(GetLabel(label)) > *max_vertex_count) {
// Don't create an indexed lookup, since we have more labeled vertices
// than the allowed count.
return nullptr;
}
filters.EraseLabelFilter(node_symbol, label);
return std::make_unique<ScanAllByLabel>(std::move(last_op), node_symbol,
label, match_ctx.graph_view);
return std::make_unique<ScanAllByLabel>(
std::move(last_op), node_symbol, GetLabel(label), match_ctx.graph_view);
}
std::unique_ptr<LogicalOperator> PlanMatching(
MatchContext &match_context, std::unique_ptr<LogicalOperator> input_op) {
auto &bound_symbols = match_context.bound_symbols;
auto &storage = context_.ast_storage;
auto &storage = *context_->ast_storage;
const auto &symbol_table = match_context.symbol_table;
const auto &matching = match_context.matching;
// Copy filters, because we will modify them as we generate Filters.
@ -347,6 +533,11 @@ class RuleBasedPlanner {
const auto &edge_symbol = symbol_table.at(*edge->identifier_);
DCHECK(!utils::Contains(bound_symbols, edge_symbol))
<< "Existing edges are not supported";
std::vector<storage::EdgeType> edge_types;
edge_types.reserve(edge->edge_types_.size());
for (const auto &type : edge->edge_types_) {
edge_types.push_back(GetEdgeType(type));
}
if (edge->IsVariable()) {
std::experimental::optional<ExpansionLambda> weight_lambda;
std::experimental::optional<Symbol> total_weight;
@ -405,7 +596,7 @@ class RuleBasedPlanner {
// TODO: Pass weight lambda.
last_op = std::make_unique<ExpandVariable>(
std::move(last_op), node1_symbol, node_symbol, edge_symbol,
edge->type_, expansion.direction, edge->edge_types_,
edge->type_, expansion.direction, edge_types,
expansion.is_flipped, edge->lower_bound_, edge->upper_bound_,
existing_node, filter_lambda, weight_lambda, total_weight,
match_context.graph_view);
@ -428,7 +619,7 @@ class RuleBasedPlanner {
}
last_op = std::make_unique<Expand>(
std::move(last_op), node1_symbol, node_symbol, edge_symbol,
expansion.direction, edge->edge_types_, existing_node,
expansion.direction, edge_types, existing_node,
match_context.graph_view);
}
@ -481,22 +672,22 @@ class RuleBasedPlanner {
const Matching &matching) {
// Copy the bound symbol set, because we don't want to use the updated
// 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};
std::unordered_set<Symbol> bound_symbols_copy(context_->bound_symbols);
MatchContext match_ctx{matching, *context_->symbol_table,
bound_symbols_copy, GraphView::NEW};
auto on_match = PlanMatching(match_ctx, nullptr);
// Use the original bound_symbols, so we fill it with new symbols.
auto on_create = impl::GenCreateForPattern(*merge.pattern_, nullptr,
context_.symbol_table,
context_.bound_symbols);
auto on_create =
GenCreateForPattern(*merge.pattern_, nullptr, *context_->symbol_table,
context_->bound_symbols);
for (auto &set : merge.on_create_) {
on_create = impl::HandleWriteClause(set, on_create, context_.symbol_table,
context_.bound_symbols);
on_create = HandleWriteClause(set, on_create, *context_->symbol_table,
context_->bound_symbols);
DCHECK(on_create) << "Expected SET in MERGE ... ON CREATE";
}
for (auto &set : merge.on_match_) {
on_match = impl::HandleWriteClause(set, on_match, context_.symbol_table,
context_.bound_symbols);
on_match = HandleWriteClause(set, on_match, *context_->symbol_table,
context_->bound_symbols);
DCHECK(on_match) << "Expected SET in MERGE ... ON MATCH";
}
return std::make_unique<plan::Merge>(

View File

@ -308,7 +308,7 @@ class VaryQueryPartMatching {
template <class TPlanningContext>
class VariableStartPlanner {
private:
TPlanningContext &context_;
TPlanningContext *context_;
// Generates different, equivalent query parts by taking different graph
// matching routes for each query part.
@ -325,18 +325,18 @@ class VariableStartPlanner {
}
public:
explicit VariableStartPlanner(TPlanningContext &context)
explicit VariableStartPlanner(TPlanningContext *context)
: context_(context) {}
/// @brief Generate multiple plans by varying the order of graph traversal.
auto Plan(const std::vector<SingleQueryPart> &query_parts) {
return iter::imap(
[context = &context_](const auto &alternative_query_parts) {
RuleBasedPlanner<TPlanningContext> rule_planner(*context);
[context = context_](const auto &alternative_query_parts) {
RuleBasedPlanner<TPlanningContext> rule_planner(context);
context->bound_symbols.clear();
return rule_planner.Plan(alternative_query_parts);
},
VaryQueryMatching(query_parts, context_.symbol_table));
VaryQueryMatching(query_parts, *context_->symbol_table));
}
/// @brief The result of plan generation is an iterable of roots to multiple

View File

@ -15,54 +15,56 @@ namespace query::plan {
template <class TDbAccessor>
class VertexCountCache {
public:
VertexCountCache(const TDbAccessor &db) : db_(db) {}
VertexCountCache(TDbAccessor *db) : db_(db) {}
int64_t VerticesCount() const {
if (!vertices_count_) vertices_count_ = db_.VerticesCount();
auto Label(const std::string &name) { return db_->Label(name); }
auto Property(const std::string &name) { return db_->Property(name); }
auto EdgeType(const std::string &name) { return db_->EdgeType(name); }
int64_t VerticesCount() {
if (!vertices_count_) vertices_count_ = db_->VerticesCount();
return *vertices_count_;
}
int64_t VerticesCount(storage::Label label) const {
int64_t VerticesCount(storage::Label label) {
if (label_vertex_count_.find(label) == label_vertex_count_.end())
label_vertex_count_[label] = db_.VerticesCount(label);
label_vertex_count_[label] = db_->VerticesCount(label);
return label_vertex_count_.at(label);
}
int64_t VerticesCount(storage::Label label,
storage::Property property) const {
int64_t VerticesCount(storage::Label label, storage::Property property) {
auto key = std::make_pair(label, property);
if (label_property_vertex_count_.find(key) ==
label_property_vertex_count_.end())
label_property_vertex_count_[key] = db_.VerticesCount(label, property);
label_property_vertex_count_[key] = db_->VerticesCount(label, property);
return label_property_vertex_count_.at(key);
}
int64_t VerticesCount(storage::Label label, storage::Property property,
const PropertyValue &value) const {
const PropertyValue &value) {
auto label_prop = std::make_pair(label, property);
auto &value_vertex_count = property_value_vertex_count_[label_prop];
if (value_vertex_count.find(value) == value_vertex_count.end())
value_vertex_count[value] = db_.VerticesCount(label, property, value);
value_vertex_count[value] = db_->VerticesCount(label, property, value);
return value_vertex_count.at(value);
}
int64_t VerticesCount(
storage::Label label, storage::Property property,
const std::experimental::optional<utils::Bound<PropertyValue>> &lower,
const std::experimental::optional<utils::Bound<PropertyValue>> &upper)
const {
const std::experimental::optional<utils::Bound<PropertyValue>> &upper) {
auto label_prop = std::make_pair(label, property);
auto &bounds_vertex_count = property_bounds_vertex_count_[label_prop];
BoundsKey bounds = std::make_pair(lower, upper);
if (bounds_vertex_count.find(bounds) == bounds_vertex_count.end())
bounds_vertex_count[bounds] =
db_.VerticesCount(label, property, lower, upper);
db_->VerticesCount(label, property, lower, upper);
return bounds_vertex_count.at(bounds);
}
bool LabelPropertyIndexExists(storage::Label label,
storage::Property property) const {
return db_.LabelPropertyIndexExists(label, property);
storage::Property property) {
return db_->LabelPropertyIndexExists(label, property);
}
private:
@ -109,18 +111,18 @@ class VertexCountCache {
}
};
const TDbAccessor &db_;
mutable std::experimental::optional<int64_t> vertices_count_;
mutable std::unordered_map<storage::Label, int64_t> label_vertex_count_;
mutable std::unordered_map<LabelPropertyKey, int64_t, LabelPropertyHash>
TDbAccessor *db_;
std::experimental::optional<int64_t> vertices_count_;
std::unordered_map<storage::Label, int64_t> label_vertex_count_;
std::unordered_map<LabelPropertyKey, int64_t, LabelPropertyHash>
label_property_vertex_count_;
mutable std::unordered_map<
std::unordered_map<
LabelPropertyKey,
std::unordered_map<query::TypedValue, int64_t, query::TypedValue::Hash,
query::TypedValue::BoolEqual>,
LabelPropertyHash>
property_value_vertex_count_;
mutable std::unordered_map<
std::unordered_map<
LabelPropertyKey,
std::unordered_map<BoundsKey, int64_t, BoundsHash, BoundsEqual>,
LabelPropertyHash>
@ -128,7 +130,7 @@ class VertexCountCache {
};
template <class TDbAccessor>
auto MakeVertexCountCache(const TDbAccessor &db) {
auto MakeVertexCountCache(TDbAccessor *db) {
return VertexCountCache<TDbAccessor>(db);
}

View File

@ -17,11 +17,6 @@ struct TypedValueVectorCompare {
ordering @0 :List(Ast.Ordering);
}
struct EvaluationContext {
timestamp @0 : Int64;
params @1 : Utils.Map(Utils.BoxInt64, Storage.PropertyValue);
}
struct TypedValue {
union {
nullType @0 :Void;

View File

@ -2,6 +2,7 @@
#include "distributed/data_manager.hpp"
#include "query/frontend/ast/ast_serialization.hpp"
#include "utils/serialization.hpp"
namespace query {
@ -140,13 +141,12 @@ void LoadCapnpTypedValue(const capnp::TypedValue::Reader &reader,
}
}
void SaveEvaluationContext(const EvaluationContext &ctx,
capnp::EvaluationContext::Builder *builder) {
builder->setTimestamp(ctx.timestamp);
auto params_builder =
builder->initParams().initEntries(ctx.parameters.size());
void Save(const Parameters &parameters,
utils::capnp::Map<utils::capnp::BoxInt64,
storage::capnp::PropertyValue>::Builder *builder) {
auto params_builder = builder->initEntries(parameters.size());
size_t i = 0;
for (auto &entry : ctx.parameters) {
for (auto &entry : parameters) {
auto builder = params_builder[i];
auto key_builder = builder.initKey();
key_builder.setValue(entry.first);
@ -156,13 +156,14 @@ void SaveEvaluationContext(const EvaluationContext &ctx,
}
}
void LoadEvaluationContext(const capnp::EvaluationContext::Reader &reader,
EvaluationContext *ctx) {
ctx->timestamp = reader.getTimestamp();
for (const auto &entry_reader : reader.getParams().getEntries()) {
void Load(
Parameters *parameters,
const utils::capnp::Map<utils::capnp::BoxInt64,
storage::capnp::PropertyValue>::Reader &reader) {
for (const auto &entry_reader : reader.getEntries()) {
PropertyValue value;
storage::LoadCapnpPropertyValue(entry_reader.getValue(), &value);
ctx->parameters.Add(entry_reader.getKey().getValue(), value);
parameters->Add(entry_reader.getKey().getValue(), value);
}
}
@ -351,22 +352,20 @@ void Load(query::TypedValue *value, slk::Reader *reader,
}
}
void Save(const query::EvaluationContext &ctx, slk::Builder *builder) {
slk::Save(ctx.timestamp, builder);
slk::Save(ctx.parameters.size(), builder);
for (auto &entry : ctx.parameters) {
void Save(const query::Parameters &parameters, slk::Builder *builder) {
slk::Save(parameters.size(), builder);
for (auto &entry : parameters) {
slk::Save(entry, builder);
}
}
void Load(query::EvaluationContext *ctx, slk::Reader *reader) {
slk::Load(&ctx->timestamp, reader);
void Load(query::Parameters *parameters, slk::Reader *reader) {
size_t size = 0;
slk::Load(&size, reader);
for (size_t i = 0; i < size; ++i) {
std::pair<int, PropertyValue> entry;
slk::Load(&entry, reader);
ctx->parameters.Add(entry.first, entry.second);
parameters->Add(entry.first, entry.second);
}
}

View File

@ -23,12 +23,6 @@ void LoadCapnpTypedValue(const capnp::TypedValue::Reader &reader,
database::GraphDbAccessor *dba,
distributed::DataManager *data_manager);
void SaveEvaluationContext(const EvaluationContext &ctx,
capnp::EvaluationContext::Builder *builder);
void LoadEvaluationContext(const capnp::EvaluationContext::Reader &reader,
EvaluationContext *ctx);
void Save(const TypedValueVectorCompare &comparator,
capnp::TypedValueVectorCompare::Builder *builder);
@ -60,6 +54,15 @@ inline void Load(SymbolTable *symbol_table,
}
}
void Save(const Parameters &parameters,
utils::capnp::Map<utils::capnp::BoxInt64,
storage::capnp::PropertyValue>::Builder *builder);
void Load(
Parameters *parameters,
const utils::capnp::Map<utils::capnp::BoxInt64,
storage::capnp::PropertyValue>::Reader &reader);
} // namespace query
namespace slk {
@ -75,10 +78,6 @@ inline void Load(query::SymbolTable *symbol_table, slk::Reader *reader) {
slk::Load(&symbol_table->table_, reader);
}
void Save(const query::EvaluationContext &ctx, slk::Builder *builder);
void Load(query::EvaluationContext *ctx, slk::Reader *reader);
void Save(const query::TypedValue &value, slk::Builder *builder,
storage::SendVersions versions, int16_t worker_id);
@ -95,4 +94,8 @@ void Save(const query::TypedValueVectorCompare &comparator,
void Load(query::TypedValueVectorCompare *comparator, slk::Reader *reader);
void Save(const query::Parameters &parameters, slk::Builder *builder);
void Load(query::Parameters *parameters, slk::Reader *reader);
} // namespace slk

View File

@ -42,8 +42,8 @@ static void BM_PlanChainedMatches(benchmark::State &state) {
int num_matches = state.range(0);
auto *query = AddChainedMatches(num_matches, storage);
auto symbol_table = query::MakeSymbolTable(query);
auto ctx =
query::plan::MakePlanningContext(storage, symbol_table, query, *dba);
auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
dba.get());
state.ResumeTiming();
auto query_parts =
query::plan::CollectQueryParts(symbol_table, storage, query);
@ -52,7 +52,7 @@ static void BM_PlanChainedMatches(benchmark::State &state) {
}
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
auto plans = query::plan::MakeLogicalPlanForSingleQuery<
query::plan::VariableStartPlanner>(single_query_parts, ctx);
query::plan::VariableStartPlanner>(single_query_parts, &ctx);
for (const auto &plan : plans) {
// Exhaust through all generated plans, since they are lazily generated.
benchmark::DoNotOptimize(plan.get());
@ -65,10 +65,10 @@ BENCHMARK(BM_PlanChainedMatches)
->Range(50, 400)
->Unit(benchmark::kMillisecond);
static query::CypherQuery *AddIndexedMatches(
int num_matches, storage::Label label,
const std::pair<std::string, storage::Property> &property,
query::AstStorage &storage) {
static query::CypherQuery *AddIndexedMatches(int num_matches,
const std::string &label,
const std::string &property,
query::AstStorage &storage) {
auto *query = storage.Create<query::CypherQuery>();
for (int i = 0; i < num_matches; ++i) {
auto *match = storage.Create<query::Match>();
@ -79,8 +79,9 @@ static query::CypherQuery *AddIndexedMatches(
std::string node1_name = "node" + std::to_string(i - 1);
auto *node = storage.Create<query::NodeAtom>(
storage.Create<query::Identifier>(node1_name));
node->labels_.emplace_back(label);
node->properties_[property] = storage.Create<query::PrimitiveLiteral>(i);
node->labels_.emplace_back(storage.GetLabelIx(label));
node->properties_[storage.GetPropertyIx(property)] =
storage.Create<query::PrimitiveLiteral>(i);
pattern->atoms_.emplace_back(node);
single_query->clauses_.emplace_back(match);
query->single_query_ = single_query;
@ -102,13 +103,13 @@ static auto CreateIndexedVertices(int index_count, int vertex_count,
}
}
dba->Commit();
return std::make_pair(label, prop);
return std::make_pair("label", "prop");
}
static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
database::GraphDb db;
storage::Label label;
storage::Property prop;
std::string label;
std::string prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
@ -117,12 +118,11 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
while (state.KeepRunning()) {
state.PauseTiming();
query::AstStorage storage;
auto *query = AddIndexedMatches(index_count, label,
std::make_pair("prop", prop), storage);
auto *query = AddIndexedMatches(index_count, label, prop, storage);
auto symbol_table = query::MakeSymbolTable(query);
state.ResumeTiming();
auto ctx =
query::plan::MakePlanningContext(storage, symbol_table, query, *dba);
auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
dba.get());
auto query_parts =
query::plan::CollectQueryParts(symbol_table, storage, query);
if (query_parts.query_parts.size() == 0) {
@ -130,9 +130,9 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
}
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
auto plans = query::plan::MakeLogicalPlanForSingleQuery<
query::plan::VariableStartPlanner>(single_query_parts, ctx);
query::plan::VariableStartPlanner>(single_query_parts, &ctx);
for (auto plan : plans) {
query::plan::EstimatePlanCost(*dba, parameters, *plan);
query::plan::EstimatePlanCost(dba.get(), parameters, *plan);
}
}
}
@ -140,23 +140,22 @@ static void BM_PlanAndEstimateIndexedMatching(benchmark::State &state) {
static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
benchmark::State &state) {
database::GraphDb db;
storage::Label label;
storage::Property prop;
std::string label;
std::string prop;
int index_count = state.range(0);
int vertex_count = state.range(1);
std::tie(label, prop) = CreateIndexedVertices(index_count, vertex_count, db);
auto dba = db.Access();
auto vertex_counts = query::plan::MakeVertexCountCache(*dba);
auto vertex_counts = query::plan::MakeVertexCountCache(dba.get());
query::Parameters parameters;
while (state.KeepRunning()) {
state.PauseTiming();
query::AstStorage storage;
auto *query = AddIndexedMatches(index_count, label,
std::make_pair("prop", prop), storage);
auto *query = AddIndexedMatches(index_count, label, prop, storage);
auto symbol_table = query::MakeSymbolTable(query);
state.ResumeTiming();
auto ctx = query::plan::MakePlanningContext(storage, symbol_table, query,
vertex_counts);
auto ctx = query::plan::MakePlanningContext(&storage, &symbol_table, query,
&vertex_counts);
auto query_parts =
query::plan::CollectQueryParts(symbol_table, storage, query);
if (query_parts.query_parts.size() == 0) {
@ -164,9 +163,9 @@ static void BM_PlanAndEstimateIndexedMatchingWithCachedCounts(
}
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
auto plans = query::plan::MakeLogicalPlanForSingleQuery<
query::plan::VariableStartPlanner>(single_query_parts, ctx);
query::plan::VariableStartPlanner>(single_query_parts, &ctx);
for (auto plan : plans) {
query::plan::EstimatePlanCost(vertex_counts, parameters, *plan);
query::plan::EstimatePlanCost(&vertex_counts, parameters, *plan);
}
}
}

View File

@ -17,7 +17,10 @@ DEFCOMMAND(ShowDistributed) {
if (ss.fail() || !ss.eof() || plan_ix >= plans.size()) return;
const auto &plan = plans[plan_ix].first;
std::atomic<int64_t> plan_id{0};
auto distributed_plan = MakeDistributedPlan(*plan, symbol_table, plan_id);
std::vector<storage::Property> properties_by_ix =
query::NamesToProperties(ast_storage.properties_, &dba);
auto distributed_plan = MakeDistributedPlan(ast_storage, *plan, symbol_table,
plan_id, properties_by_ix);
{
std::cout << "---- Master Plan ---- " << std::endl;
query::plan::DistributedPrettyPrint(dba,

View File

@ -129,14 +129,18 @@ class Timer {
// Dummy DbAccessor which forwards user input for various vertex counts.
class InteractiveDbAccessor {
public:
InteractiveDbAccessor(database::GraphDbAccessor &dba, int64_t vertices_count,
InteractiveDbAccessor(database::GraphDbAccessor *dba, int64_t vertices_count,
Timer &timer)
: dba_(dba), vertices_count_(vertices_count), timer_(timer) {}
int64_t VerticesCount() const { return vertices_count_; }
auto Label(const std::string &name) { return dba_->Label(name); }
auto Property(const std::string &name) { return dba_->Property(name); }
auto EdgeType(const std::string &name) { return dba_->EdgeType(name); }
int64_t VerticesCount(storage::Label label_id) const {
auto label = dba_.LabelName(label_id);
int64_t VerticesCount() { return vertices_count_; }
int64_t VerticesCount(storage::Label label_id) {
auto label = dba_->LabelName(label_id);
if (label_vertex_count_.find(label) == label_vertex_count_.end()) {
label_vertex_count_[label] = ReadVertexCount("label '" + label + "'");
}
@ -144,9 +148,9 @@ class InteractiveDbAccessor {
}
int64_t VerticesCount(storage::Label label_id,
storage::Property property_id) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
storage::Property property_id) {
auto label = dba_->LabelName(label_id);
auto property = dba_->PropertyName(property_id);
auto key = std::make_pair(label, property);
if (label_property_vertex_count_.find(key) ==
label_property_vertex_count_.end()) {
@ -157,9 +161,9 @@ class InteractiveDbAccessor {
}
int64_t VerticesCount(storage::Label label_id, storage::Property property_id,
const PropertyValue &value) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
const PropertyValue &value) {
auto label = dba_->LabelName(label_id);
auto property = dba_->PropertyName(property_id);
auto label_prop = std::make_pair(label, property);
if (label_property_index_.find(label_prop) == label_property_index_.end()) {
return 0;
@ -178,10 +182,9 @@ class InteractiveDbAccessor {
int64_t VerticesCount(
storage::Label label_id, storage::Property property_id,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
const std::experimental::optional<utils::Bound<PropertyValue>> upper) {
auto label = dba_->LabelName(label_id);
auto property = dba_->PropertyName(property_id);
std::stringstream range_string;
if (lower) {
range_string << (lower->IsInclusive() ? "[" : "(") << lower->value()
@ -197,9 +200,9 @@ class InteractiveDbAccessor {
}
bool LabelPropertyIndexExists(storage::Label label_id,
storage::Property property_id) const {
auto label = dba_.LabelName(label_id);
auto property = dba_.PropertyName(property_id);
storage::Property property_id) {
auto label = dba_->LabelName(label_id);
auto property = dba_->PropertyName(property_id);
auto key = std::make_pair(label, property);
if (label_property_index_.find(key) == label_property_index_.end()) {
bool resp = timer_.WithPause([&label, &property]() {
@ -312,15 +315,14 @@ class InteractiveDbAccessor {
private:
typedef std::pair<std::string, std::string> LabelPropertyKey;
database::GraphDbAccessor &dba_;
database::GraphDbAccessor *dba_;
int64_t vertices_count_;
Timer &timer_;
mutable std::map<std::string, int64_t> label_vertex_count_;
mutable std::map<std::pair<std::string, std::string>, int64_t>
std::map<std::string, int64_t> label_vertex_count_;
std::map<std::pair<std::string, std::string>, int64_t>
label_property_vertex_count_;
mutable std::map<std::pair<std::string, std::string>, bool>
label_property_index_;
mutable std::map<
std::map<std::pair<std::string, std::string>, bool> label_property_index_;
std::map<
std::pair<std::string, std::string>,
std::unordered_map<query::TypedValue, int64_t, query::TypedValue::Hash,
query::TypedValue::BoolEqual>>
@ -408,7 +410,8 @@ DEFCOMMAND(Help) {
void ExaminePlans(
database::GraphDbAccessor &dba, const query::SymbolTable &symbol_table,
std::vector<std::pair<std::unique_ptr<query::plan::LogicalOperator>,
double>> &plans) {
double>> &plans,
const query::AstStorage &ast) {
while (true) {
auto line = ReadLine("plan? ");
if (!line || *line == "quit") break;
@ -428,18 +431,17 @@ void ExaminePlans(
<< " arguments" << std::endl;
continue;
}
command.function(dba, symbol_table, plans, args);
command.function(dba, symbol_table, plans, args, ast);
}
}
query::Query *MakeAst(const std::string &query, query::AstStorage *storage,
database::GraphDbAccessor &dba) {
query::Query *MakeAst(const std::string &query, query::AstStorage *storage) {
query::ParsingContext parsing_context;
parsing_context.is_query_cached = false;
// query -> AST
auto parser = std::make_unique<query::frontend::opencypher::Parser>(query);
// AST -> high level tree
query::frontend::CypherMainVisitor visitor(parsing_context, storage, &dba);
query::frontend::CypherMainVisitor visitor(parsing_context, storage);
visitor.visit(parser->tree());
return visitor.query();
}
@ -448,24 +450,22 @@ query::Query *MakeAst(const std::string &query, query::AstStorage *storage,
// order by cost.
auto MakeLogicalPlans(query::CypherQuery *query, query::AstStorage &ast,
query::SymbolTable &symbol_table,
InteractiveDbAccessor &dba) {
InteractiveDbAccessor *dba) {
auto query_parts = query::plan::CollectQueryParts(symbol_table, ast, query);
std::vector<std::pair<std::unique_ptr<query::plan::LogicalOperator>, double>>
plans_with_cost;
auto ctx = query::plan::MakePlanningContext(ast, symbol_table, query, dba);
auto ctx = query::plan::MakePlanningContext(&ast, &symbol_table, query, dba);
if (query_parts.query_parts.size() <= 0) {
std::cerr << "Failed to extract query parts" << std::endl;
std::exit(EXIT_FAILURE);
}
auto plans = query::plan::MakeLogicalPlanForSingleQuery<
query::plan::VariableStartPlanner>(
query_parts.query_parts.at(0).single_query_parts, ctx);
query_parts.query_parts.at(0).single_query_parts, &ctx);
query::Parameters parameters;
for (auto plan : plans) {
query::plan::CostEstimator<InteractiveDbAccessor> estimator(dba,
parameters);
plan->Accept(estimator);
plans_with_cost.emplace_back(std::move(plan), estimator.cost());
auto cost = query::plan::EstimatePlanCost(dba, parameters, *plan);
plans_with_cost.emplace_back(std::move(plan), cost);
}
std::stable_sort(
plans_with_cost.begin(), plans_with_cost.end(),
@ -482,7 +482,7 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
}
Timer planning_timer;
InteractiveDbAccessor interactive_db(
*dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
dba, in_db_filename.empty() ? ReadInt("Vertices in DB: ") : 0,
planning_timer);
if (!in_db_filename.empty()) {
std::ifstream db_file(in_db_filename);
@ -495,7 +495,7 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
try {
query::AstStorage ast;
auto *query =
dynamic_cast<query::CypherQuery *>(MakeAst(*line, &ast, *dba));
dynamic_cast<query::CypherQuery *>(MakeAst(*line, &ast));
if (!query) {
throw utils::BasicException(
"Interactive planning is only avaialable for regular openCypher "
@ -503,14 +503,14 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
}
auto symbol_table = query::MakeSymbolTable(query);
planning_timer.Start();
auto plans = MakeLogicalPlans(query, ast, symbol_table, interactive_db);
auto plans = MakeLogicalPlans(query, ast, symbol_table, &interactive_db);
auto planning_time = planning_timer.Elapsed();
std::cout
<< "Planning took "
<< std::chrono::duration<double, std::milli>(planning_time).count()
<< "ms" << std::endl;
std::cout << "Generated " << plans.size() << " plans" << std::endl;
ExaminePlans(*dba, symbol_table, plans);
ExaminePlans(*dba, symbol_table, plans, ast);
} catch (const utils::BasicException &e) {
std::cout << "Error: " << e.what() << std::endl;
}

View File

@ -23,7 +23,7 @@ struct Command {
typedef std::vector<std::string> Args;
// Function of this command
std::function<void(database::GraphDbAccessor &, const query::SymbolTable &,
PlansWithCost &, const Args &)>
PlansWithCost &, const Args &, const query::AstStorage &)>
function;
// Number of arguments the function works with.
int arg_count;
@ -31,10 +31,11 @@ struct Command {
std::string documentation;
};
#define DEFCOMMAND(Name) \
void Name##Command(database::GraphDbAccessor &dba, \
const query::SymbolTable &symbol_table, \
PlansWithCost &plans, const Command::Args &args)
#define DEFCOMMAND(Name) \
void Name##Command(database::GraphDbAccessor &dba, \
const query::SymbolTable &symbol_table, \
PlansWithCost &plans, const Command::Args &args, \
const query::AstStorage &ast_storage)
void AddCommand(const std::string &name, const Command &command);

View File

@ -10,7 +10,6 @@
#include <gtest/gtest.h>
#include "communication/rpc/serialization.hpp"
#include "database/distributed/distributed_graph_db.hpp"
#include "query/context.hpp"
#include "query/frontend/ast/ast.hpp"
#include "query/frontend/ast/ast_serialization.hpp"
@ -30,26 +29,20 @@ using testing::ElementsAre;
using testing::Pair;
using testing::UnorderedElementsAre;
static std::unique_ptr<database::Master> db_;
// Base class for all test types
class Base {
public:
explicit Base(const std::string &query) : query_string_(query) {
db_accessor_ = db_->Access();
}
std::unique_ptr<database::GraphDbAccessor> db_accessor_;
explicit Base(const std::string &query) : query_string_(query) {}
ParsingContext context_;
Parameters parameters_;
std::string query_string_;
auto Prop(const std::string &prop_name) {
return db_accessor_->Property(prop_name);
}
virtual PropertyIx Prop(const std::string &prop_name) = 0;
auto PropPair(const std::string &prop_name) {
return std::make_pair(prop_name, Prop(prop_name));
}
virtual LabelIx Label(const std::string &label_name) = 0;
virtual EdgeTypeIx EdgeType(const std::string &edge_type_name) = 0;
TypedValue LiteralValue(Expression *expression) {
if (context_.is_query_cached) {
@ -86,7 +79,7 @@ class CapnpAstGenerator : public Base {
explicit CapnpAstGenerator(const std::string &query) : Base(query) {
::frontend::opencypher::Parser parser(query);
AstStorage tmp_storage;
CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
CypherMainVisitor visitor(context_, &tmp_storage);
visitor.visit(parser.tree());
::capnp::MallocMessageBuilder message;
@ -105,6 +98,18 @@ class CapnpAstGenerator : public Base {
}
}
PropertyIx Prop(const std::string &prop_name) override {
return storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return storage_.GetEdgeTypeIx(name);
}
AstStorage storage_;
Query *query_;
};
@ -114,7 +119,7 @@ class SlkAstGenerator : public Base {
explicit SlkAstGenerator(const std::string &query) : Base(query) {
::frontend::opencypher::Parser parser(query);
AstStorage tmp_storage;
CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
CypherMainVisitor visitor(context_, &tmp_storage);
visitor.visit(parser.tree());
slk::Builder builder;
@ -130,24 +135,24 @@ class SlkAstGenerator : public Base {
}
}
PropertyIx Prop(const std::string &prop_name) override {
return storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return storage_.GetEdgeTypeIx(name);
}
AstStorage storage_;
Query *query_;
};
template <typename T>
class CypherMainVisitorTest : public ::testing::Test {
public:
static void SetUpTestCase() {
db_ = std::make_unique<database::Master>();
db_->Start();
}
static void TearDownTestCase() {
db_->Shutdown();
db_->AwaitShutdown();
db_ = nullptr;
}
};
class CypherMainVisitorTest : public ::testing::Test {};
typedef ::testing::Types<CapnpAstGenerator, SlkAstGenerator> AstGeneratorTypes;
@ -175,8 +180,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) {
auto identifier = dynamic_cast<Identifier *>(property_lookup->expression_);
ASSERT_TRUE(identifier);
ASSERT_EQ(identifier->name_, "n");
ASSERT_EQ(property_lookup->property_,
ast_generator.db_accessor_->Property("x"));
ASSERT_EQ(property_lookup->property_, ast_generator.Prop("x"));
}
TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
@ -194,8 +198,7 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
ASSERT_TRUE(identifier);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("x"),
ast_generator.db_accessor_->Label("y")));
ElementsAre(ast_generator.Label("x"), ast_generator.Label("y")));
}
TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
@ -211,7 +214,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l")));
ElementsAre(ast_generator.Label("l-$\"'ab`e``l")));
}
TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
@ -227,8 +230,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
return_clause->body_.named_expressions[0]->expression_);
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label(label)));
ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label(label)));
}
}
@ -244,8 +246,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
return_clause->body_.named_expressions[0]->expression_);
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
EXPECT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("a")));
ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label("a")));
}
TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) {
@ -884,12 +885,12 @@ TYPED_TEST(CypherMainVisitorTest, MapLiteral) {
return_clause->body_.named_expressions[0]->expression_);
ASSERT_TRUE(map_literal);
ASSERT_EQ(3, map_literal->elements_.size());
ast_generator.CheckLiteral(
map_literal->elements_[ast_generator.PropPair("a")], 1);
ast_generator.CheckLiteral(
map_literal->elements_[ast_generator.PropPair("b")], "bla");
ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("a")],
1);
ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("b")],
"bla");
auto *elem_2 = dynamic_cast<ListLiteral *>(
map_literal->elements_[ast_generator.PropPair("c")]);
map_literal->elements_[ast_generator.Prop("c")]);
ASSERT_TRUE(elem_2);
EXPECT_EQ(2, elem_2->elements_.size());
auto *elem_2_1 = dynamic_cast<MapLiteral *>(elem_2->elements_[1]);
@ -918,19 +919,19 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
EXPECT_EQ(node->identifier_->name_,
CypherMainVisitor::kAnonPrefix + std::to_string(1));
EXPECT_FALSE(node->identifier_->user_declared_);
EXPECT_THAT(node->labels_, UnorderedElementsAre(
ast_generator.db_accessor_->Label("label1"),
ast_generator.db_accessor_->Label("label2"),
ast_generator.db_accessor_->Label("label3")));
std::map<std::pair<std::string, storage::Property>, int64_t> properties;
EXPECT_THAT(node->labels_,
UnorderedElementsAre(ast_generator.Label("label1"),
ast_generator.Label("label2"),
ast_generator.Label("label3")));
std::unordered_map<PropertyIx, int64_t> properties;
for (auto x : node->properties_) {
TypedValue value = ast_generator.LiteralValue(x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);
properties[x.first] = value.Value<int64_t>();
}
EXPECT_THAT(properties,
UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
Pair(ast_generator.PropPair("b"), 10)));
UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
Pair(ast_generator.Prop("b"), 10)));
}
TYPED_TEST(CypherMainVisitorTest, PropertyMapSameKeyAppearsTwice) {
@ -1022,19 +1023,18 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
auto *edge = dynamic_cast<EdgeAtom *>(match->patterns_[0]->atoms_[1]);
ASSERT_TRUE(edge);
EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN);
EXPECT_THAT(
edge->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
std::map<std::pair<std::string, storage::Property>, int64_t> properties;
EXPECT_THAT(edge->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
std::unordered_map<PropertyIx, int64_t> properties;
for (auto x : edge->properties_) {
TypedValue value = ast_generator.LiteralValue(x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);
properties[x.first] = value.Value<int64_t>();
}
EXPECT_THAT(properties,
UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
Pair(ast_generator.PropPair("b"), 10)));
UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
Pair(ast_generator.Prop("b"), 10)));
}
TYPED_TEST(CypherMainVisitorTest, RelationshipPatternVariable) {
@ -1169,8 +1169,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUnboundedWithProperty) {
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
EXPECT_EQ(edge->upper_bound_, nullptr);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
}
TYPED_TEST(CypherMainVisitorTest,
@ -1187,10 +1186,9 @@ TYPED_TEST(CypherMainVisitorTest,
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
EXPECT_EQ(edge->upper_bound_, nullptr);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
ASSERT_EQ(edge->edge_types_.size(), 1U);
auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type");
auto edge_type = ast_generator.EdgeType("edge_type");
EXPECT_EQ(edge->edge_types_[0], edge_type);
}
@ -1207,8 +1205,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUpperBoundedWithProperty) {
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
ast_generator.CheckLiteral(edge->upper_bound_, 2);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
}
TYPED_TEST(CypherMainVisitorTest, ReturnUnanemdIdentifier) {
@ -1316,7 +1313,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
ASSERT_TRUE(identifier1);
ASSERT_EQ(identifier1->name_, "a");
ASSERT_EQ(set_property->property_lookup_->property_,
ast_generator.db_accessor_->Property("x"));
ast_generator.Prop("x"));
auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_);
ASSERT_EQ(identifier2->name_, "b");
}
@ -1351,8 +1348,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
ASSERT_TRUE(set_labels->identifier_);
ASSERT_EQ(set_labels->identifier_->name_, "g");
ASSERT_THAT(set_labels->labels_,
UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
ast_generator.db_accessor_->Label("i")));
UnorderedElementsAre(ast_generator.Label("h"),
ast_generator.Label("i")));
}
}
@ -1363,7 +1360,6 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
ASSERT_TRUE(query->single_query_);
auto *single_query = query->single_query_;
ASSERT_EQ(single_query->clauses_.size(), 2U);
{
auto *remove_property =
dynamic_cast<RemoveProperty *>(single_query->clauses_[0]);
@ -1374,7 +1370,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
ASSERT_TRUE(identifier1);
ASSERT_EQ(identifier1->name_, "a");
ASSERT_EQ(remove_property->property_lookup_->property_,
ast_generator.db_accessor_->Property("x"));
ast_generator.Prop("x"));
}
{
auto *remove_labels =
@ -1383,8 +1379,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
ASSERT_TRUE(remove_labels->identifier_);
ASSERT_EQ(remove_labels->identifier_->name_, "g");
ASSERT_THAT(remove_labels->labels_,
UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
ast_generator.db_accessor_->Label("i")));
UnorderedElementsAre(ast_generator.Label("h"),
ast_generator.Label("i")));
}
}
@ -1591,9 +1587,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties = {
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
EXPECT_EQ(index_query->properties_, expected_properties);
}
@ -1602,10 +1597,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko"),
ast_generator.db_accessor_->Property("pero")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko"),
ast_generator.Prop("pero")};
ASSERT_EQ(index_query->properties_, expected_properties);
}
@ -1619,9 +1613,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndexWithSingleProperty) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
ASSERT_EQ(index_query->properties_, expected_properties);
}
@ -1630,9 +1623,8 @@ TYPED_TEST(CypherMainVisitorTest, DropIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::DROP);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
EXPECT_EQ(index_query->properties_, expected_properties);
}
@ -1748,10 +1740,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) {
ASSERT_TRUE(bfs);
EXPECT_TRUE(bfs->IsVariable());
EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
bfs->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(bfs->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
EXPECT_EQ(bfs->identifier_->name_, "r");
EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e");
EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_);
@ -1798,10 +1789,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) {
EXPECT_TRUE(shortest->IsVariable());
EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
shortest->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(shortest->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
ast_generator.CheckLiteral(shortest->upper_bound_, 10);
EXPECT_FALSE(shortest->lower_bound_);
EXPECT_EQ(shortest->identifier_->name_, "r");
@ -1838,10 +1828,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) {
EXPECT_TRUE(shortest->IsVariable());
EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
shortest->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(shortest->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
ast_generator.CheckLiteral(shortest->upper_bound_, 10);
EXPECT_FALSE(shortest->lower_bound_);
EXPECT_EQ(shortest->identifier_->name_, "r");

View File

@ -398,6 +398,10 @@ void BfsTest(Database *db, int lower_bound, int upper_bound,
query::plan::ExpansionLambda{inner_edge_sym, inner_node_sym,
filter_expr});
context.evaluation_context_.properties =
query::NamesToProperties(storage.properties_, &dba);
context.evaluation_context_.labels =
query::NamesToLabels(storage.labels_, &dba);
std::vector<std::vector<query::TypedValue>> results;
// An exception should be thrown on one of the pulls.

View File

@ -29,19 +29,15 @@ using testing::UnorderedElementsAre;
class Base {
public:
explicit Base(const std::string &query) : query_string_(query) {}
database::GraphDb db_;
std::unique_ptr<database::GraphDbAccessor> db_accessor_{db_.Access()};
ParsingContext context_;
Parameters parameters_;
std::string query_string_;
auto Prop(const std::string &prop_name) {
return db_accessor_->Property(prop_name);
}
virtual PropertyIx Prop(const std::string &prop_name) = 0;
auto PropPair(const std::string &prop_name) {
return std::make_pair(prop_name, Prop(prop_name));
}
virtual LabelIx Label(const std::string &label_name) = 0;
virtual EdgeTypeIx EdgeType(const std::string &edge_type_name) = 0;
TypedValue LiteralValue(Expression *expression) {
if (context_.is_query_cached) {
@ -78,11 +74,23 @@ class AstGenerator : public Base {
public:
explicit AstGenerator(const std::string &query) : Base(query) {
::frontend::opencypher::Parser parser(query);
CypherMainVisitor visitor(context_, &ast_storage_, db_accessor_.get());
CypherMainVisitor visitor(context_, &ast_storage_);
visitor.visit(parser.tree());
query_ = visitor.query();
}
PropertyIx Prop(const std::string &prop_name) override {
return ast_storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return ast_storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return ast_storage_.GetEdgeTypeIx(name);
}
AstStorage ast_storage_;
Query *query_;
};
@ -96,6 +104,18 @@ class OriginalAfterCloningAstGenerator : public AstGenerator {
AstStorage storage;
query_->Clone(storage);
}
PropertyIx Prop(const std::string &prop_name) override {
return ast_storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return ast_storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return ast_storage_.GetEdgeTypeIx(name);
}
};
// This generator clones parsed ast and uses that one.
@ -106,11 +126,23 @@ class ClonedAstGenerator : public Base {
explicit ClonedAstGenerator(const std::string &query) : Base(query) {
::frontend::opencypher::Parser parser(query);
AstStorage tmp_storage;
CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
CypherMainVisitor visitor(context_, &tmp_storage);
visitor.visit(parser.tree());
query_ = visitor.query()->Clone(ast_storage_);
}
PropertyIx Prop(const std::string &prop_name) override {
return ast_storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return ast_storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return ast_storage_.GetEdgeTypeIx(name);
}
AstStorage ast_storage_;
Query *query_;
};
@ -125,11 +157,23 @@ class CachedAstGenerator : public Base {
parameters_ = stripped.literals();
::frontend::opencypher::Parser parser(stripped.query());
AstStorage tmp_storage;
CypherMainVisitor visitor(context_, &tmp_storage, db_accessor_.get());
CypherMainVisitor visitor(context_, &tmp_storage);
visitor.visit(parser.tree());
query_ = visitor.query()->Clone(ast_storage_);
}
PropertyIx Prop(const std::string &prop_name) override {
return ast_storage_.GetPropertyIx(prop_name);
}
LabelIx Label(const std::string &name) override {
return ast_storage_.GetLabelIx(name);
}
EdgeTypeIx EdgeType(const std::string &name) override {
return ast_storage_.GetEdgeTypeIx(name);
}
AstStorage ast_storage_;
Query *query_;
};
@ -165,8 +209,7 @@ TYPED_TEST(CypherMainVisitorTest, PropertyLookup) {
auto identifier = dynamic_cast<Identifier *>(property_lookup->expression_);
ASSERT_TRUE(identifier);
ASSERT_EQ(identifier->name_, "n");
ASSERT_EQ(property_lookup->property_,
ast_generator.db_accessor_->Property("x"));
ASSERT_EQ(property_lookup->property_, ast_generator.Prop("x"));
}
TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
@ -184,8 +227,7 @@ TYPED_TEST(CypherMainVisitorTest, LabelsTest) {
ASSERT_TRUE(identifier);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("x"),
ast_generator.db_accessor_->Label("y")));
ElementsAre(ast_generator.Label("x"), ast_generator.Label("y")));
}
TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
@ -201,7 +243,7 @@ TYPED_TEST(CypherMainVisitorTest, EscapedLabel) {
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("l-$\"'ab`e``l")));
ElementsAre(ast_generator.Label("l-$\"'ab`e``l")));
}
TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
@ -217,8 +259,7 @@ TYPED_TEST(CypherMainVisitorTest, KeywordLabel) {
return_clause->body_.named_expressions[0]->expression_);
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
ASSERT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label(label)));
ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label(label)));
}
}
@ -234,8 +275,7 @@ TYPED_TEST(CypherMainVisitorTest, HexLetterLabel) {
return_clause->body_.named_expressions[0]->expression_);
auto identifier = dynamic_cast<Identifier *>(labels_test->expression_);
EXPECT_EQ(identifier->name_, "n");
ASSERT_THAT(labels_test->labels_,
ElementsAre(ast_generator.db_accessor_->Label("a")));
ASSERT_THAT(labels_test->labels_, ElementsAre(ast_generator.Label("a")));
}
TYPED_TEST(CypherMainVisitorTest, ReturnNoDistinctNoBagSemantics) {
@ -874,12 +914,12 @@ TYPED_TEST(CypherMainVisitorTest, MapLiteral) {
return_clause->body_.named_expressions[0]->expression_);
ASSERT_TRUE(map_literal);
ASSERT_EQ(3, map_literal->elements_.size());
ast_generator.CheckLiteral(
map_literal->elements_[ast_generator.PropPair("a")], 1);
ast_generator.CheckLiteral(
map_literal->elements_[ast_generator.PropPair("b")], "bla");
ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("a")],
1);
ast_generator.CheckLiteral(map_literal->elements_[ast_generator.Prop("b")],
"bla");
auto *elem_2 = dynamic_cast<ListLiteral *>(
map_literal->elements_[ast_generator.PropPair("c")]);
map_literal->elements_[ast_generator.Prop("c")]);
ASSERT_TRUE(elem_2);
EXPECT_EQ(2, elem_2->elements_.size());
auto *elem_2_1 = dynamic_cast<MapLiteral *>(elem_2->elements_[1]);
@ -908,19 +948,19 @@ TYPED_TEST(CypherMainVisitorTest, NodePattern) {
EXPECT_EQ(node->identifier_->name_,
CypherMainVisitor::kAnonPrefix + std::to_string(1));
EXPECT_FALSE(node->identifier_->user_declared_);
EXPECT_THAT(node->labels_, UnorderedElementsAre(
ast_generator.db_accessor_->Label("label1"),
ast_generator.db_accessor_->Label("label2"),
ast_generator.db_accessor_->Label("label3")));
std::map<std::pair<std::string, storage::Property>, int64_t> properties;
EXPECT_THAT(node->labels_,
UnorderedElementsAre(ast_generator.Label("label1"),
ast_generator.Label("label2"),
ast_generator.Label("label3")));
std::unordered_map<PropertyIx, int64_t> properties;
for (auto x : node->properties_) {
TypedValue value = ast_generator.LiteralValue(x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);
properties[x.first] = value.Value<int64_t>();
}
EXPECT_THAT(properties,
UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
Pair(ast_generator.PropPair("b"), 10)));
UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
Pair(ast_generator.Prop("b"), 10)));
}
TYPED_TEST(CypherMainVisitorTest, PropertyMapSameKeyAppearsTwice) {
@ -1012,19 +1052,18 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternDetails) {
auto *edge = dynamic_cast<EdgeAtom *>(match->patterns_[0]->atoms_[1]);
ASSERT_TRUE(edge);
EXPECT_EQ(edge->direction_, EdgeAtom::Direction::IN);
EXPECT_THAT(
edge->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
std::map<std::pair<std::string, storage::Property>, int64_t> properties;
EXPECT_THAT(edge->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
std::unordered_map<PropertyIx, int64_t> properties;
for (auto x : edge->properties_) {
TypedValue value = ast_generator.LiteralValue(x.second);
ASSERT_TRUE(value.type() == TypedValue::Type::Int);
properties[x.first] = value.Value<int64_t>();
}
EXPECT_THAT(properties,
UnorderedElementsAre(Pair(ast_generator.PropPair("a"), 5),
Pair(ast_generator.PropPair("b"), 10)));
UnorderedElementsAre(Pair(ast_generator.Prop("a"), 5),
Pair(ast_generator.Prop("b"), 10)));
}
TYPED_TEST(CypherMainVisitorTest, RelationshipPatternVariable) {
@ -1159,8 +1198,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUnboundedWithProperty) {
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
EXPECT_EQ(edge->upper_bound_, nullptr);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
}
TYPED_TEST(CypherMainVisitorTest,
@ -1177,10 +1215,9 @@ TYPED_TEST(CypherMainVisitorTest,
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
EXPECT_EQ(edge->upper_bound_, nullptr);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
ASSERT_EQ(edge->edge_types_.size(), 1U);
auto edge_type = ast_generator.db_accessor_->EdgeType("edge_type");
auto edge_type = ast_generator.EdgeType("edge_type");
EXPECT_EQ(edge->edge_types_[0], edge_type);
}
@ -1197,8 +1234,7 @@ TYPED_TEST(CypherMainVisitorTest, RelationshipPatternUpperBoundedWithProperty) {
EXPECT_EQ(edge->type_, EdgeAtom::Type::DEPTH_FIRST);
EXPECT_EQ(edge->lower_bound_, nullptr);
ast_generator.CheckLiteral(edge->upper_bound_, 2);
ast_generator.CheckLiteral(edge->properties_[ast_generator.PropPair("prop")],
42);
ast_generator.CheckLiteral(edge->properties_[ast_generator.Prop("prop")], 42);
}
// TODO maybe uncomment
@ -1323,7 +1359,7 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
ASSERT_TRUE(identifier1);
ASSERT_EQ(identifier1->name_, "a");
ASSERT_EQ(set_property->property_lookup_->property_,
ast_generator.db_accessor_->Property("x"));
ast_generator.Prop("x"));
auto *identifier2 = dynamic_cast<Identifier *>(set_property->expression_);
ASSERT_EQ(identifier2->name_, "b");
}
@ -1358,8 +1394,8 @@ TYPED_TEST(CypherMainVisitorTest, Set) {
ASSERT_TRUE(set_labels->identifier_);
ASSERT_EQ(set_labels->identifier_->name_, "g");
ASSERT_THAT(set_labels->labels_,
UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
ast_generator.db_accessor_->Label("i")));
UnorderedElementsAre(ast_generator.Label("h"),
ast_generator.Label("i")));
}
}
@ -1381,7 +1417,7 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
ASSERT_TRUE(identifier1);
ASSERT_EQ(identifier1->name_, "a");
ASSERT_EQ(remove_property->property_lookup_->property_,
ast_generator.db_accessor_->Property("x"));
ast_generator.Prop("x"));
}
{
auto *remove_labels =
@ -1390,8 +1426,8 @@ TYPED_TEST(CypherMainVisitorTest, Remove) {
ASSERT_TRUE(remove_labels->identifier_);
ASSERT_EQ(remove_labels->identifier_->name_, "g");
ASSERT_THAT(remove_labels->labels_,
UnorderedElementsAre(ast_generator.db_accessor_->Label("h"),
ast_generator.db_accessor_->Label("i")));
UnorderedElementsAre(ast_generator.Label("h"),
ast_generator.Label("i")));
}
}
@ -1598,9 +1634,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties = {
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties = {ast_generator.Prop("slavko")};
EXPECT_EQ(index_query->properties_, expected_properties);
}
@ -1609,10 +1644,9 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko"),
ast_generator.db_accessor_->Property("pero")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko"),
ast_generator.Prop("pero")};
ASSERT_EQ(index_query->properties_, expected_properties);
}
@ -1626,9 +1660,8 @@ TYPED_TEST(CypherMainVisitorTest, CreateUniqueIndexWithSingleProperty) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::CREATE_UNIQUE);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
ASSERT_EQ(index_query->properties_, expected_properties);
}
@ -1637,9 +1670,8 @@ TYPED_TEST(CypherMainVisitorTest, DropIndex) {
auto *index_query = dynamic_cast<IndexQuery *>(ast_generator.query_);
ASSERT_TRUE(index_query);
EXPECT_EQ(index_query->action_, IndexQuery::Action::DROP);
EXPECT_EQ(index_query->label_, ast_generator.db_accessor_->Label("mirko"));
std::vector<storage::Property> expected_properties{
ast_generator.db_accessor_->Property("slavko")};
EXPECT_EQ(index_query->label_, ast_generator.Label("mirko"));
std::vector<PropertyIx> expected_properties{ast_generator.Prop("slavko")};
EXPECT_EQ(index_query->properties_, expected_properties);
}
@ -1755,10 +1787,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchBfsReturn) {
ASSERT_TRUE(bfs);
EXPECT_TRUE(bfs->IsVariable());
EXPECT_EQ(bfs->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
bfs->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(bfs->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
EXPECT_EQ(bfs->identifier_->name_, "r");
EXPECT_EQ(bfs->filter_lambda_.inner_edge->name_, "e");
EXPECT_TRUE(bfs->filter_lambda_.inner_edge->user_declared_);
@ -1805,10 +1836,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestReturn) {
EXPECT_TRUE(shortest->IsVariable());
EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
shortest->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(shortest->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
ast_generator.CheckLiteral(shortest->upper_bound_, 10);
EXPECT_FALSE(shortest->lower_bound_);
EXPECT_EQ(shortest->identifier_->name_, "r");
@ -1845,10 +1875,9 @@ TYPED_TEST(CypherMainVisitorTest, MatchWShortestNoFilterReturn) {
EXPECT_TRUE(shortest->IsVariable());
EXPECT_EQ(shortest->type_, EdgeAtom::Type::WEIGHTED_SHORTEST_PATH);
EXPECT_EQ(shortest->direction_, EdgeAtom::Direction::OUT);
EXPECT_THAT(
shortest->edge_types_,
UnorderedElementsAre(ast_generator.db_accessor_->EdgeType("type1"),
ast_generator.db_accessor_->EdgeType("type2")));
EXPECT_THAT(shortest->edge_types_,
UnorderedElementsAre(ast_generator.EdgeType("type1"),
ast_generator.EdgeType("type2")));
ast_generator.CheckLiteral(shortest->upper_bound_, 10);
EXPECT_FALSE(shortest->lower_bound_);
EXPECT_EQ(shortest->identifier_->name_, "r");

File diff suppressed because it is too large Load Diff

View File

@ -57,15 +57,15 @@ auto ToMap(const TypedValue &t) {
return map;
};
std::string ToString(Expression *expr) {
std::string ToString(const AstStorage &storage, Expression *expr) {
std::ostringstream ss;
PrintExpression(expr, &ss);
PrintExpression(storage, expr, &ss);
return ss.str();
}
std::string ToString(NamedExpression *expr) {
std::string ToString(const AstStorage &storage, NamedExpression *expr) {
std::ostringstream ss;
PrintExpression(expr, &ss);
PrintExpression(storage, expr, &ss);
return ss.str();
}
@ -121,15 +121,22 @@ auto GetOrderBy(T... exprs) {
template <class TDbAccessor>
auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba,
const std::string &name, storage::Property property) {
return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
dba.PropertyName(property), property);
return storage.Create<PropertyLookup>(
storage.Create<Identifier>(name),
storage.GetPropertyIx(dba.PropertyName(property)));
}
template <class TDbAccessor>
auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba, Expression *expr,
storage::Property property) {
return storage.Create<PropertyLookup>(expr, dba.PropertyName(property),
property);
return storage.Create<PropertyLookup>(
expr, storage.GetPropertyIx(dba.PropertyName(property)));
}
template <class TDbAccessor>
auto GetPropertyLookup(AstStorage &storage, TDbAccessor &dba, Expression *expr,
const std::string &property) {
return storage.Create<PropertyLookup>(expr, storage.GetPropertyIx(property));
}
template <class TDbAccessor>
@ -137,15 +144,15 @@ auto GetPropertyLookup(
AstStorage &storage, TDbAccessor &, const std::string &name,
const std::pair<std::string, storage::Property> &prop_pair) {
return storage.Create<PropertyLookup>(storage.Create<Identifier>(name),
prop_pair.first, prop_pair.second);
storage.GetPropertyIx(prop_pair.first));
}
template <class TDbAccessor>
auto GetPropertyLookup(
AstStorage &storage, TDbAccessor &, Expression *expr,
const std::pair<std::string, storage::Property> &prop_pair) {
return storage.Create<PropertyLookup>(expr, prop_pair.first,
prop_pair.second);
return storage.Create<PropertyLookup>(expr,
storage.GetPropertyIx(prop_pair.first));
}
/// Create an EdgeAtom with given name, direction and edge_type.
@ -153,9 +160,14 @@ auto GetPropertyLookup(
/// Name is used to create the Identifier which is assigned to the edge.
auto GetEdge(AstStorage &storage, const std::string &name,
EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
const std::vector<storage::EdgeType> &edge_types = {}) {
const std::vector<std::string> &edge_types = {}) {
std::vector<EdgeTypeIx> types;
types.reserve(edge_types.size());
for (const auto &type : edge_types) {
types.push_back(storage.GetEdgeTypeIx(type));
}
return storage.Create<EdgeAtom>(storage.Create<Identifier>(name),
EdgeAtom::Type::SINGLE, dir, edge_types);
EdgeAtom::Type::SINGLE, dir, types);
}
/// Create a variable length expansion EdgeAtom with given name, direction and
@ -165,15 +177,20 @@ auto GetEdge(AstStorage &storage, const std::string &name,
auto GetEdgeVariable(AstStorage &storage, const std::string &name,
EdgeAtom::Type type = EdgeAtom::Type::DEPTH_FIRST,
EdgeAtom::Direction dir = EdgeAtom::Direction::BOTH,
const std::vector<storage::EdgeType> &edge_types = {},
const std::vector<std::string> &edge_types = {},
Identifier *flambda_inner_edge = nullptr,
Identifier *flambda_inner_node = nullptr,
Identifier *wlambda_inner_edge = nullptr,
Identifier *wlambda_inner_node = nullptr,
Expression *wlambda_expression = nullptr,
Identifier *total_weight = nullptr) {
std::vector<EdgeTypeIx> types;
types.reserve(edge_types.size());
for (const auto &type : edge_types) {
types.push_back(storage.GetEdgeTypeIx(type));
}
auto r_val = storage.Create<EdgeAtom>(storage.Create<Identifier>(name), type,
dir, edge_types);
dir, types);
r_val->filter_lambda_.inner_edge =
flambda_inner_edge ? flambda_inner_edge
@ -205,10 +222,10 @@ auto GetEdgeVariable(AstStorage &storage, const std::string &name,
///
/// Name is used to create the Identifier which is assigned to the node.
auto GetNode(AstStorage &storage, const std::string &name,
std::experimental::optional<storage::Label> label =
std::experimental::optional<std::string> label =
std::experimental::nullopt) {
auto node = storage.Create<NodeAtom>(storage.Create<Identifier>(name));
if (label) node->labels_.emplace_back(*label);
if (label) node->labels_.emplace_back(storage.GetLabelIx(*label));
return node;
}
@ -444,7 +461,12 @@ auto GetSet(AstStorage &storage, const std::string &name, Expression *expr,
/// Create a set labels clause for given identifier name and labels.
auto GetSet(AstStorage &storage, const std::string &name,
std::vector<storage::Label> labels) {
std::vector<std::string> label_names) {
std::vector<LabelIx> labels;
labels.reserve(label_names.size());
for (const auto &label : label_names) {
labels.push_back(storage.GetLabelIx(label));
}
return storage.Create<SetLabels>(storage.Create<Identifier>(name), labels);
}
@ -455,7 +477,12 @@ auto GetRemove(AstStorage &storage, PropertyLookup *prop_lookup) {
/// Create a remove labels clause for given identifier name and labels.
auto GetRemove(AstStorage &storage, const std::string &name,
std::vector<storage::Label> labels) {
std::vector<std::string> label_names) {
std::vector<LabelIx> labels;
labels.reserve(label_names.size());
for (const auto &label : label_names) {
labels.push_back(storage.GetLabelIx(label));
}
return storage.Create<RemoveLabels>(storage.Create<Identifier>(name), labels);
}
@ -513,10 +540,9 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
#define LIST(...) \
storage.Create<query::ListLiteral>( \
std::vector<query::Expression *>{__VA_ARGS__})
#define MAP(...) \
storage.Create<query::MapLiteral>( \
std::unordered_map<std::pair<std::string, storage::Property>, \
query::Expression *>{__VA_ARGS__})
#define MAP(...) \
storage.Create<query::MapLiteral>( \
std::unordered_map<query::PropertyIx, query::Expression *>{__VA_ARGS__})
#define PROPERTY_PAIR(property_name) \
std::make_pair(property_name, dba.Property(property_name))
#define PROPERTY_LOOKUP(...) \
@ -557,7 +583,7 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
#define CREATE_INDEX_ON(label, property) \
storage.Create<query::IndexQuery>( \
query::IndexQuery::Action::CREATE, (label), \
std::vector<storage::Property>{(property)})
std::vector<query::PropertyIx>{(property)})
#define QUERY(...) query::test_common::GetQuery(storage, __VA_ARGS__)
#define SINGLE_QUERY(...) \
query::test_common::GetSingleQuery(storage.Create<SingleQuery>(), __VA_ARGS__)

View File

@ -62,7 +62,7 @@ class QueryCostEstimator : public ::testing::Test {
}
auto Cost() {
CostEstimator<database::GraphDbAccessor> cost_estimator(*dba, parameters_);
CostEstimator<database::GraphDbAccessor> cost_estimator(dba.get(), parameters_);
last_op_->Accept(cost_estimator);
return cost_estimator.cost();
}
@ -112,7 +112,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyValueConstant) {
AddVertices(100, 30, 20);
for (auto const_val : {Literal(12), Parameter(12)}) {
MakeOp<ScanAllByLabelPropertyValue>(nullptr, NextSymbol(), label, property,
const_val);
"property", const_val);
EXPECT_COST(1 * CostParam::MakeScanAllByLabelPropertyValue);
}
}
@ -121,7 +121,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyValueConstExpr) {
AddVertices(100, 30, 20);
for (auto const_val : {Literal(12), Parameter(12)}) {
MakeOp<ScanAllByLabelPropertyValue>(
nullptr, NextSymbol(), label, property,
nullptr, NextSymbol(), label, property, "property",
// once we make expression const-folding this test case will fail
storage_.Create<UnaryPlusOperator>(const_val));
EXPECT_COST(20 * CardParam::kFilter *
@ -133,7 +133,8 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeUpperConstant) {
AddVertices(100, 30, 20);
for (auto const_val : {Literal(12), Parameter(12)}) {
MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
nullopt, InclusiveBound(const_val));
"property", nullopt,
InclusiveBound(const_val));
// cardinality estimation is exact for very small indexes
EXPECT_COST(13 * CostParam::MakeScanAllByLabelPropertyRange);
}
@ -143,12 +144,14 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeLowerConstant) {
AddVertices(100, 30, 20);
for (auto const_val : {Literal(17), Parameter(17)}) {
MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
InclusiveBound(const_val), nullopt);
"property", InclusiveBound(const_val),
nullopt);
// cardinality estimation is exact for very small indexes
EXPECT_COST(3 * CostParam::MakeScanAllByLabelPropertyRange);
}
}
TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
AddVertices(100, 30, 20);
for (auto const_val : {Literal(12), Parameter(12)}) {
@ -156,7 +159,7 @@ TEST_F(QueryCostEstimator, ScanAllByLabelPropertyRangeConstExpr) {
utils::MakeBoundInclusive(static_cast<Expression *>(
storage_.Create<UnaryPlusOperator>(const_val))));
MakeOp<ScanAllByLabelPropertyRange>(nullptr, NextSymbol(), label, property,
bound, nullopt);
"property", bound, nullopt);
EXPECT_COST(20 * CardParam::kFilter *
CostParam::MakeScanAllByLabelPropertyRange);
}

View File

@ -48,17 +48,24 @@ class ExpressionEvaluatorTest : public ::testing::Test {
frame[symbol] = value;
return id;
}
template <class TExpression>
auto Eval(TExpression *expr) {
ctx.properties = NamesToProperties(storage.properties_, dba.get());
ctx.labels = NamesToLabels(storage.labels_, dba.get());
return expr->Accept(eval);
}
};
TEST_F(ExpressionEvaluatorTest, OrOperator) {
auto *op =
storage.Create<OrOperator>(storage.Create<PrimitiveLiteral>(true),
storage.Create<PrimitiveLiteral>(false));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<OrOperator>(storage.Create<PrimitiveLiteral>(true),
storage.Create<PrimitiveLiteral>(true));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), true);
}
@ -66,11 +73,11 @@ TEST_F(ExpressionEvaluatorTest, XorOperator) {
auto *op =
storage.Create<XorOperator>(storage.Create<PrimitiveLiteral>(true),
storage.Create<PrimitiveLiteral>(false));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<XorOperator>(storage.Create<PrimitiveLiteral>(true),
storage.Create<PrimitiveLiteral>(true));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), false);
}
@ -78,11 +85,11 @@ TEST_F(ExpressionEvaluatorTest, AndOperator) {
auto *op =
storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(true),
storage.Create<PrimitiveLiteral>(true));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(false),
storage.Create<PrimitiveLiteral>(true));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), false);
}
@ -91,7 +98,7 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorShortCircuit) {
auto *op =
storage.Create<AndOperator>(storage.Create<PrimitiveLiteral>(false),
storage.Create<PrimitiveLiteral>(5));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueBool(), false);
}
{
@ -102,7 +109,7 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorShortCircuit) {
// raise due to `5`. This differs from neo4j, where they evaluate both
// sides and return `false` without checking for type of the first
// expression.
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
}
@ -112,20 +119,20 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorNull) {
auto *op = storage.Create<AndOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>(5));
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
{
auto *op = storage.Create<AndOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>(true));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
auto *op = storage.Create<AndOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>(false));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_TRUE(value.IsBool());
EXPECT_EQ(value.ValueBool(), false);
}
@ -134,21 +141,21 @@ TEST_F(ExpressionEvaluatorTest, AndOperatorNull) {
TEST_F(ExpressionEvaluatorTest, AdditionOperator) {
auto *op = storage.Create<AdditionOperator>(
storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 5);
}
TEST_F(ExpressionEvaluatorTest, SubtractionOperator) {
auto *op = storage.Create<SubtractionOperator>(
storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), -1);
}
TEST_F(ExpressionEvaluatorTest, MultiplicationOperator) {
auto *op = storage.Create<MultiplicationOperator>(
storage.Create<PrimitiveLiteral>(2), storage.Create<PrimitiveLiteral>(3));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 6);
}
@ -156,14 +163,14 @@ TEST_F(ExpressionEvaluatorTest, DivisionOperator) {
auto *op =
storage.Create<DivisionOperator>(storage.Create<PrimitiveLiteral>(50),
storage.Create<PrimitiveLiteral>(10));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 5);
}
TEST_F(ExpressionEvaluatorTest, ModOperator) {
auto *op = storage.Create<ModOperator>(storage.Create<PrimitiveLiteral>(65),
storage.Create<PrimitiveLiteral>(10));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 5);
}
@ -171,15 +178,15 @@ TEST_F(ExpressionEvaluatorTest, EqualOperator) {
auto *op =
storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), false);
op = storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), true);
op = storage.Create<EqualOperator>(storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), false);
}
@ -187,30 +194,30 @@ TEST_F(ExpressionEvaluatorTest, NotEqualOperator) {
auto *op =
storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), false);
op = storage.Create<NotEqualOperator>(storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), true);
}
TEST_F(ExpressionEvaluatorTest, LessOperator) {
auto *op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), false);
op = storage.Create<LessOperator>(storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), false);
}
@ -218,15 +225,15 @@ TEST_F(ExpressionEvaluatorTest, GreaterOperator) {
auto *op =
storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), false);
op = storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), false);
op = storage.Create<GreaterOperator>(storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), true);
}
@ -234,15 +241,15 @@ TEST_F(ExpressionEvaluatorTest, LessEqualOperator) {
auto *op =
storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), true);
op = storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), true);
op = storage.Create<LessEqualOperator>(storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), false);
}
@ -250,17 +257,17 @@ TEST_F(ExpressionEvaluatorTest, GreaterEqualOperator) {
auto *op = storage.Create<GreaterEqualOperator>(
storage.Create<PrimitiveLiteral>(10),
storage.Create<PrimitiveLiteral>(15));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), false);
op = storage.Create<GreaterEqualOperator>(
storage.Create<PrimitiveLiteral>(15),
storage.Create<PrimitiveLiteral>(15));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), true);
op = storage.Create<GreaterEqualOperator>(
storage.Create<PrimitiveLiteral>(20),
storage.Create<PrimitiveLiteral>(15));
auto val3 = op->Accept(eval);
auto val3 = Eval(op);
ASSERT_EQ(val3.ValueBool(), true);
}
@ -272,14 +279,14 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
// Element exists in list.
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>(2), list_literal);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueBool(), true);
}
{
// Element doesn't exist in list.
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>("x"), list_literal);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueBool(), false);
}
{
@ -290,7 +297,7 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
// Element doesn't exist in list with null element.
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>("x"), list_literal);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
@ -298,14 +305,14 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>("x"),
storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
// Null literal.
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null), list_literal);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
@ -313,7 +320,7 @@ TEST_F(ExpressionEvaluatorTest, InListOperator) {
auto *op = storage.Create<InListOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<ListLiteral>(std::vector<Expression *>()));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_FALSE(value.ValueBool());
}
}
@ -327,28 +334,28 @@ TEST_F(ExpressionEvaluatorTest, ListIndexing) {
// Legal indexing.
auto *op = storage.Create<SubscriptOperator>(
list_literal, storage.Create<PrimitiveLiteral>(2));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueInt(), 3);
}
{
// Out of bounds indexing.
auto *op = storage.Create<SubscriptOperator>(
list_literal, storage.Create<PrimitiveLiteral>(4));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
// Out of bounds indexing with negative bound.
auto *op = storage.Create<SubscriptOperator>(
list_literal, storage.Create<PrimitiveLiteral>(-100));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
// Legal indexing with negative index.
auto *op = storage.Create<SubscriptOperator>(
list_literal, storage.Create<PrimitiveLiteral>(-2));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueInt(), 3);
}
{
@ -356,51 +363,48 @@ TEST_F(ExpressionEvaluatorTest, ListIndexing) {
auto *op = storage.Create<SubscriptOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>(-2));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
// Indexing with incompatible type.
auto *op = storage.Create<SubscriptOperator>(
list_literal, storage.Create<PrimitiveLiteral>("bla"));
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
}
TEST_F(ExpressionEvaluatorTest, MapIndexing) {
auto *map_literal = storage.Create<MapLiteral>(
std::unordered_map<std::pair<std::string, storage::Property>,
Expression *>{{std::make_pair("a", dba->Property("a")),
storage.Create<PrimitiveLiteral>(1)},
{std::make_pair("b", dba->Property("b")),
storage.Create<PrimitiveLiteral>(2)},
{std::make_pair("c", dba->Property("c")),
storage.Create<PrimitiveLiteral>(3)}});
auto *map_literal =
storage.Create<MapLiteral>(std::unordered_map<PropertyIx, Expression *>{
{storage.GetPropertyIx("a"), storage.Create<PrimitiveLiteral>(1)},
{storage.GetPropertyIx("b"), storage.Create<PrimitiveLiteral>(2)},
{storage.GetPropertyIx("c"), storage.Create<PrimitiveLiteral>(3)}});
{
// Legal indexing.
auto *op = storage.Create<SubscriptOperator>(
map_literal, storage.Create<PrimitiveLiteral>("b"));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_EQ(value.ValueInt(), 2);
}
{
// Legal indexing, non-existing key.
auto *op = storage.Create<SubscriptOperator>(
map_literal, storage.Create<PrimitiveLiteral>("z"));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
{
// Wrong key type.
auto *op = storage.Create<SubscriptOperator>(
map_literal, storage.Create<PrimitiveLiteral>(42));
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
{
// Indexing with Null.
auto *op = storage.Create<SubscriptOperator>(
map_literal, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
}
@ -419,46 +423,46 @@ TEST_F(ExpressionEvaluatorTest, VertexAndEdgeIndexing) {
// Legal indexing.
auto *op1 = storage.Create<SubscriptOperator>(
vertex_id, storage.Create<PrimitiveLiteral>("prop"));
auto value1 = op1->Accept(eval);
auto value1 = Eval(op1);
EXPECT_EQ(value1.ValueInt(), 42);
auto *op2 = storage.Create<SubscriptOperator>(
edge_id, storage.Create<PrimitiveLiteral>("prop"));
auto value2 = op2->Accept(eval);
auto value2 = Eval(op2);
EXPECT_EQ(value2.ValueInt(), 43);
}
{
// Legal indexing, non-existing key.
auto *op1 = storage.Create<SubscriptOperator>(
vertex_id, storage.Create<PrimitiveLiteral>("blah"));
auto value1 = op1->Accept(eval);
auto value1 = Eval(op1);
EXPECT_TRUE(value1.IsNull());
auto *op2 = storage.Create<SubscriptOperator>(
edge_id, storage.Create<PrimitiveLiteral>("blah"));
auto value2 = op2->Accept(eval);
auto value2 = Eval(op2);
EXPECT_TRUE(value2.IsNull());
}
{
// Wrong key type.
auto *op1 = storage.Create<SubscriptOperator>(
vertex_id, storage.Create<PrimitiveLiteral>(1));
EXPECT_THROW(op1->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op1), QueryRuntimeException);
auto *op2 = storage.Create<SubscriptOperator>(
edge_id, storage.Create<PrimitiveLiteral>(1));
EXPECT_THROW(op2->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op2), QueryRuntimeException);
}
{
// Indexing with Null.
auto *op1 = storage.Create<SubscriptOperator>(
vertex_id, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto value1 = op1->Accept(eval);
auto value1 = Eval(op1);
EXPECT_TRUE(value1.IsNull());
auto *op2 = storage.Create<SubscriptOperator>(
edge_id, storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto value2 = op2->Accept(eval);
auto value2 = Eval(op2);
EXPECT_TRUE(value2.IsNull());
}
}
@ -481,7 +485,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(2),
storage.Create<PrimitiveLiteral>(4));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre(3, 4));
}
{
@ -489,7 +493,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(2),
storage.Create<PrimitiveLiteral>(-1));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre(3));
}
{
@ -497,7 +501,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(2),
storage.Create<PrimitiveLiteral>(-4));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre());
}
{
@ -505,21 +509,21 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(-100),
storage.Create<PrimitiveLiteral>(10));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre(1, 2, 3, 4));
}
{
// Lower bound undefined.
auto *op = storage.Create<ListSlicingOperator>(
list_literal, nullptr, storage.Create<PrimitiveLiteral>(3));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre(1, 2, 3));
}
{
// Upper bound undefined.
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(-2), nullptr);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_THAT(extract_ints(value), ElementsAre(3, 4));
}
{
@ -527,21 +531,21 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>("mirko"));
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
{
// List of illegal type.
auto *op = storage.Create<ListSlicingOperator>(
storage.Create<PrimitiveLiteral>("a"),
storage.Create<PrimitiveLiteral>(-2), nullptr);
EXPECT_THROW(op->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(op), QueryRuntimeException);
}
{
// Null value list with undefined upper bound.
auto *op = storage.Create<ListSlicingOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null),
storage.Create<PrimitiveLiteral>(-2), nullptr);
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
;
}
@ -550,7 +554,7 @@ TEST_F(ExpressionEvaluatorTest, ListSlicingOperator) {
auto *op = storage.Create<ListSlicingOperator>(
list_literal, storage.Create<PrimitiveLiteral>(-2),
storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto value = op->Accept(eval);
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
;
}
@ -565,7 +569,7 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
storage.Create<PrimitiveLiteral>(2));
auto *op = storage.Create<IfOperator>(condition_true, then_expression,
else_expression);
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 10);
}
{
@ -574,7 +578,7 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
storage.Create<PrimitiveLiteral>(3));
auto *op = storage.Create<IfOperator>(condition_false, then_expression,
else_expression);
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 20);
}
{
@ -583,39 +587,39 @@ TEST_F(ExpressionEvaluatorTest, IfOperator) {
storage.Create<PrimitiveLiteral>(3));
auto *op = storage.Create<IfOperator>(condition_exception, then_expression,
else_expression);
ASSERT_THROW(op->Accept(eval), QueryRuntimeException);
ASSERT_THROW(Eval(op), QueryRuntimeException);
}
}
TEST_F(ExpressionEvaluatorTest, NotOperator) {
auto *op =
storage.Create<NotOperator>(storage.Create<PrimitiveLiteral>(false));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueBool(), true);
}
TEST_F(ExpressionEvaluatorTest, UnaryPlusOperator) {
auto *op =
storage.Create<UnaryPlusOperator>(storage.Create<PrimitiveLiteral>(5));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), 5);
}
TEST_F(ExpressionEvaluatorTest, UnaryMinusOperator) {
auto *op =
storage.Create<UnaryMinusOperator>(storage.Create<PrimitiveLiteral>(5));
auto value = op->Accept(eval);
auto value = Eval(op);
ASSERT_EQ(value.ValueInt(), -5);
}
TEST_F(ExpressionEvaluatorTest, IsNullOperator) {
auto *op =
storage.Create<IsNullOperator>(storage.Create<PrimitiveLiteral>(1));
auto val1 = op->Accept(eval);
auto val1 = Eval(op);
ASSERT_EQ(val1.ValueBool(), false);
op = storage.Create<IsNullOperator>(
storage.Create<PrimitiveLiteral>(PropertyValue::Null));
auto val2 = op->Accept(eval);
auto val2 = Eval(op);
ASSERT_EQ(val2.ValueBool(), true);
}
@ -630,26 +634,26 @@ TEST_F(ExpressionEvaluatorTest, LabelsTest) {
frame[node_symbol] = v1;
{
auto *op = storage.Create<LabelsTest>(
identifier,
std::vector<storage::Label>{dba->Label("DOG"), dba->Label("ANIMAL")});
auto value = op->Accept(eval);
identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
storage.GetLabelIx("ANIMAL")});
auto value = Eval(op);
EXPECT_EQ(value.ValueBool(), true);
}
{
auto *op = storage.Create<LabelsTest>(
identifier,
std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
dba->Label("ANIMAL")});
auto value = op->Accept(eval);
identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
storage.GetLabelIx("BAD_DOG"),
storage.GetLabelIx("ANIMAL")});
auto value = Eval(op);
EXPECT_EQ(value.ValueBool(), false);
}
{
frame[node_symbol] = TypedValue::Null;
auto *op = storage.Create<LabelsTest>(
identifier,
std::vector<storage::Label>{dba->Label("DOG"), dba->Label("BAD_DOG"),
dba->Label("ANIMAL")});
auto value = op->Accept(eval);
identifier, std::vector<LabelIx>{storage.GetLabelIx("DOG"),
storage.GetLabelIx("BAD_DOG"),
storage.GetLabelIx("ANIMAL")});
auto value = Eval(op);
EXPECT_TRUE(value.IsNull());
}
}
@ -660,7 +664,7 @@ TEST_F(ExpressionEvaluatorTest, Aggregation) {
auto aggr_sym = symbol_table.CreateSymbol("aggr", true);
symbol_table[*aggr] = aggr_sym;
frame[aggr_sym] = TypedValue(1);
auto value = aggr->Accept(eval);
auto value = Eval(aggr);
EXPECT_EQ(value.ValueInt(), 1);
}
@ -669,7 +673,7 @@ TEST_F(ExpressionEvaluatorTest, ListLiteral) {
std::vector<Expression *>{storage.Create<PrimitiveLiteral>(1),
storage.Create<PrimitiveLiteral>("bla"),
storage.Create<PrimitiveLiteral>(true)});
TypedValue result = list_literal->Accept(eval);
TypedValue result = Eval(list_literal);
ASSERT_TRUE(result.IsList());
auto &result_elems = result.ValueList();
ASSERT_EQ(3, result_elems.size());
@ -684,7 +688,7 @@ TEST_F(ExpressionEvaluatorTest, ListLiteral) {
TEST_F(ExpressionEvaluatorTest, ParameterLookup) {
ctx.parameters.Add(0, 42);
auto *param_lookup = storage.Create<ParameterLookup>(0);
auto value = param_lookup->Accept(eval);
auto value = Eval(param_lookup);
ASSERT_TRUE(value.IsInt());
EXPECT_EQ(value.ValueInt(), 42);
}
@ -697,7 +701,7 @@ TEST_F(ExpressionEvaluatorTest, All) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*all->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = all->Accept(eval);
auto value = Eval(all);
ASSERT_TRUE(value.IsBool());
EXPECT_FALSE(value.ValueBool());
}
@ -707,7 +711,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionAllNullList) {
auto *all = ALL("x", LITERAL(PropertyValue::Null), WHERE(LITERAL(true)));
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*all->identifier_] = x_sym;
auto value = all->Accept(eval);
auto value = Eval(all);
EXPECT_TRUE(value.IsNull());
}
@ -716,7 +720,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionAllWhereWrongType) {
auto *all = ALL("x", LIST(LITERAL(1)), WHERE(LITERAL(2)));
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*all->identifier_] = x_sym;
EXPECT_THROW(all->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(all), QueryRuntimeException);
}
TEST_F(ExpressionEvaluatorTest, FunctionSingle) {
@ -727,7 +731,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingle) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*single->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = single->Accept(eval);
auto value = Eval(single);
ASSERT_TRUE(value.IsBool());
EXPECT_TRUE(value.ValueBool());
}
@ -740,7 +744,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingle2) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*single->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = single->Accept(eval);
auto value = Eval(single);
ASSERT_TRUE(value.IsBool());
EXPECT_FALSE(value.ValueBool());
}
@ -751,7 +755,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionSingleNullList) {
SINGLE("x", LITERAL(PropertyValue::Null), WHERE(LITERAL(true)));
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*single->identifier_] = x_sym;
auto value = single->Accept(eval);
auto value = Eval(single);
EXPECT_TRUE(value.IsNull());
}
@ -767,7 +771,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionReduce) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*reduce->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = reduce->Accept(eval);
auto value = Eval(reduce);
ASSERT_TRUE(value.IsInt());
EXPECT_EQ(value.ValueInt(), 3);
}
@ -781,7 +785,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtract) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*extract->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = extract->Accept(eval);
auto value = Eval(extract);
EXPECT_TRUE(value.IsList());
;
auto result = value.ValueList();
@ -798,7 +802,7 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtractNull) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*extract->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
auto value = extract->Accept(eval);
auto value = Eval(extract);
EXPECT_TRUE(value.IsNull());
}
@ -809,40 +813,37 @@ TEST_F(ExpressionEvaluatorTest, FunctionExtractExceptions) {
const auto x_sym = symbol_table.CreateSymbol("x", true);
symbol_table[*extract->identifier_] = x_sym;
symbol_table[*ident_x] = x_sym;
EXPECT_THROW(extract->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(extract), QueryRuntimeException);
}
TEST_F(ExpressionEvaluatorTest, Coalesce) {
// coalesce()
EXPECT_THROW(COALESCE()->Accept(eval), QueryRuntimeException);
EXPECT_THROW(Eval(COALESCE()), QueryRuntimeException);
// coalesce(null, null)
EXPECT_TRUE(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null))
->Accept(eval)
.IsNull());
EXPECT_TRUE(
Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null)))
.IsNull());
// coalesce(null, 2, 3)
EXPECT_EQ(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3))
->Accept(eval)
EXPECT_EQ(Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3)))
.ValueInt(),
2);
// coalesce(null, 2, assert(false), 3)
EXPECT_EQ(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
FN("ASSERT", LITERAL(false)), LITERAL(3))
->Accept(eval)
EXPECT_EQ(Eval(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
FN("ASSERT", LITERAL(false)), LITERAL(3)))
.ValueInt(),
2);
// (null, assert(false))
EXPECT_THROW(COALESCE(LITERAL(TypedValue::Null), FN("ASSERT", LITERAL(false)))
->Accept(eval),
QueryRuntimeException);
EXPECT_THROW(
Eval(COALESCE(LITERAL(TypedValue::Null), FN("ASSERT", LITERAL(false)))),
QueryRuntimeException);
// coalesce([null, null])
EXPECT_FALSE(COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
TypedValue::Null, TypedValue::Null})))
->Accept(eval)
EXPECT_FALSE(Eval(COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
TypedValue::Null, TypedValue::Null}))))
.IsNull());
}
@ -858,8 +859,9 @@ class ExpressionEvaluatorPropertyLookup : public ExpressionEvaluatorTest {
void SetUp() { symbol_table[*identifier] = symbol; }
auto Value(std::pair<std::string, storage::Property> property) {
auto *op = storage.Create<PropertyLookup>(identifier, property);
return op->Accept(eval);
auto *op = storage.Create<PropertyLookup>(
identifier, storage.GetPropertyIx(property.first));
return Eval(op);
}
};
@ -915,7 +917,7 @@ class FunctionTest : public ExpressionEvaluatorTest {
const std::string &function_name,
const std::vector<Expression *> &expressions) {
auto *op = storage.Create<Function>(function_name, expressions);
return op->Accept(eval);
return Eval(op);
}
TypedValue EvaluateFunction(const std::string &function_name,

View File

@ -43,9 +43,9 @@ class Planner {
public:
template <class TDbAccessor>
Planner(std::vector<SingleQueryPart> single_query_parts,
PlanningContext<TDbAccessor> &context) {
PlanningContext<TDbAccessor> context) {
plan_ = MakeLogicalPlanForSingleQuery<RuleBasedPlanner>(single_query_parts,
context);
&context);
}
auto &plan() { return *plan_; }
@ -68,7 +68,7 @@ auto CheckPlan(query::CypherQuery *query, AstStorage &storage,
TChecker... checker) {
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TPlanner>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TPlanner>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, checker...);
}
@ -86,7 +86,7 @@ TYPED_TEST(TestPlanner, MatchNodeReturn) {
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), RETURN(as_n)));
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce());
}
@ -99,7 +99,7 @@ TYPED_TEST(TestPlanner, CreateNodeReturn) {
auto symbol_table = query::MakeSymbolTable(query);
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc,
ExpectProduce());
}
@ -108,7 +108,7 @@ TYPED_TEST(TestPlanner, CreateExpand) {
// Test CREATE (n) -[r :rel1]-> (m)
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto *query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
CheckPlan<TypeParam>(query, storage, ExpectCreateNode(),
@ -127,7 +127,7 @@ TYPED_TEST(TestPlanner, CreateNodeExpandNode) {
// Test CREATE (n) -[r :rel]-> (m), (l)
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("rel");
auto relationship = "rel";
auto *query = QUERY(SINGLE_QUERY(CREATE(
PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")),
PATTERN(NODE("l")))));
@ -139,7 +139,7 @@ TYPED_TEST(TestPlanner, CreateNamedPattern) {
// Test CREATE p = (n) -[r :rel]-> (m)
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("rel");
auto relationship = "rel";
auto *query = QUERY(SINGLE_QUERY(CREATE(NAMED_PATTERN(
"p", NODE("n"), EDGE("r", Direction::OUT, {relationship}), NODE("m")))));
CheckPlan<TypeParam>(query, storage, ExpectCreateNode(), ExpectCreateExpand(),
@ -150,7 +150,7 @@ TYPED_TEST(TestPlanner, MatchCreateExpand) {
// Test MATCH (n) CREATE (n) -[r :rel1]-> (m)
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))),
CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {relationship}),
@ -162,12 +162,12 @@ TYPED_TEST(TestPlanner, MatchLabeledNodes) {
// Test MATCH (n :label) RETURN n
AstStorage storage;
FakeDbAccessor dba;
auto label = dba.Label("label");
auto label = "label";
auto *as_n = NEXPR("n", IDENT("n"));
auto *query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))), RETURN(as_n)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAllByLabel(),
ExpectProduce());
}
@ -176,14 +176,14 @@ TYPED_TEST(TestPlanner, MatchPathReturn) {
// Test MATCH (n) -[r :relationship]- (m) RETURN n
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto *as_n = NEXPR("n", IDENT("n"));
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::BOTH, {relationship}),
NODE("m"))),
RETURN(as_n)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
ExpectProduce());
}
@ -192,7 +192,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
// Test MATCH p = (n) -[r :relationship]- (m) RETURN p
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto *as_p = NEXPR("p", IDENT("p"));
auto *query = QUERY(SINGLE_QUERY(
MATCH(NAMED_PATTERN("p", NODE("n"),
@ -200,7 +200,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternReturn) {
NODE("m"))),
RETURN(as_p)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
ExpectConstructNamedPath(), ExpectProduce());
}
@ -209,7 +209,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
// Test MATCH p = (n) -[r :relationship]- (m) WHERE 2 = p RETURN p
AstStorage storage;
FakeDbAccessor dba;
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto *as_p = NEXPR("p", IDENT("p"));
auto *query = QUERY(SINGLE_QUERY(
MATCH(NAMED_PATTERN("p", NODE("n"),
@ -217,7 +217,7 @@ TYPED_TEST(TestPlanner, MatchNamedPatternWithPredicateReturn) {
NODE("m"))),
WHERE(EQ(LITERAL(2), IDENT("p"))), RETURN(as_p)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
ExpectConstructNamedPath(), ExpectFilter(), ExpectProduce());
}
@ -238,7 +238,7 @@ TYPED_TEST(TestPlanner, OptionalMatchNamedPatternReturn) {
std::vector<Symbol> optional_symbols{get_symbol(pattern), get_symbol(node_n),
get_symbol(edge), get_symbol(node_m)};
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
std::list<BaseOpChecker *> optional{new ExpectScanAll(), new ExpectExpand(),
new ExpectConstructNamedPath()};
CheckPlan(planner.plan(), symbol_table,
@ -255,7 +255,7 @@ TYPED_TEST(TestPlanner, MatchWhereReturn) {
MATCH(PATTERN(NODE("n"))),
WHERE(LESS(PROPERTY_LOOKUP("n", property), LITERAL(42))), RETURN(as_n)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectFilter(),
ExpectProduce());
}
@ -273,7 +273,7 @@ TYPED_TEST(TestPlanner, MatchNodeSet) {
AstStorage storage;
FakeDbAccessor dba;
auto prop = dba.Property("prop");
auto label = dba.Label("label");
auto label = "label";
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
SET(PROPERTY_LOOKUP("n", prop), LITERAL(42)),
SET("n", IDENT("n")), SET("n", {label})));
@ -286,7 +286,7 @@ TYPED_TEST(TestPlanner, MatchRemove) {
AstStorage storage;
FakeDbAccessor dba;
auto prop = dba.Property("prop");
auto label = dba.Label("label");
auto label = "label";
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
REMOVE(PROPERTY_LOOKUP("n", prop)),
REMOVE("n", {label})));
@ -351,7 +351,7 @@ TYPED_TEST(TestPlanner, MultiMatch) {
MATCH(PATTERN(node_j, edge_e, node_i, edge_f, node_h)), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// Multiple MATCH clauses form a Cartesian product, so the uniqueness should
// not cross MATCH boundaries.
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
@ -370,7 +370,7 @@ TYPED_TEST(TestPlanner, MultiMatchSameStart) {
// MATCH clause.
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
ExpectProduce());
}
@ -384,7 +384,7 @@ TYPED_TEST(TestPlanner, MatchWithReturn) {
// No accumulation since we only do reads.
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
ExpectProduce());
}
@ -400,7 +400,7 @@ TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
WHERE(LESS(PROPERTY_LOOKUP("new", prop), LITERAL(42))), RETURN(as_new)));
// No accumulation since we only do reads.
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
ExpectFilter(), ExpectProduce());
}
@ -408,8 +408,8 @@ TYPED_TEST(TestPlanner, MatchWithWhereReturn) {
TYPED_TEST(TestPlanner, CreateMultiExpand) {
// Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
FakeDbAccessor dba;
auto r = dba.EdgeType("r");
auto p = dba.EdgeType("p");
auto r = "r";
auto p = "p";
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE(PATTERN(NODE("n"), EDGE("r", Direction::OUT, {r}), NODE("m")),
@ -446,7 +446,7 @@ TYPED_TEST(TestPlanner, MatchReturnSum) {
MATCH(PATTERN(NODE("n"))), RETURN(sum, AS("sum"), n_prop2, AS("group"))));
auto aggr = ExpectAggregate({sum}, {n_prop2});
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), aggr,
ExpectProduce());
}
@ -463,7 +463,7 @@ TYPED_TEST(TestPlanner, CreateWithSum) {
auto symbol_table = query::MakeSymbolTable(query);
auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
auto aggr = ExpectAggregate({sum}, {});
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// We expect both the accumulation and aggregation because the part before
// WITH updates the database.
CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
@ -473,7 +473,7 @@ TYPED_TEST(TestPlanner, CreateWithSum) {
TYPED_TEST(TestPlanner, MatchWithCreate) {
// Test MATCH (n) WITH n AS a CREATE (a) -[r :r]-> (b)
FakeDbAccessor dba;
auto r_type = dba.EdgeType("r");
auto r_type = "r";
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))), WITH("n", AS("a")),
@ -492,7 +492,7 @@ TYPED_TEST(TestPlanner, MatchReturnSkipLimit) {
RETURN(as_n, SKIP(LITERAL(2)), LIMIT(LITERAL(1)))));
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
ExpectSkip(), ExpectLimit());
}
@ -507,7 +507,7 @@ TYPED_TEST(TestPlanner, CreateWithSkipReturnLimit) {
auto symbol_table = query::MakeSymbolTable(query);
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// Since we have a write query, we need to have Accumulate. This is a bit
// different than Neo4j 3.0, which optimizes WITH followed by RETURN as a
// single RETURN clause and then moves Skip and Limit before Accumulate.
@ -530,7 +530,7 @@ TYPED_TEST(TestPlanner, CreateReturnSumSkipLimit) {
auto symbol_table = query::MakeSymbolTable(query);
auto acc = ExpectAccumulate({symbol_table.at(*n_prop->expression_)});
auto aggr = ExpectAggregate({sum}, {});
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
ExpectProduce(), ExpectSkip(), ExpectLimit());
}
@ -545,7 +545,7 @@ TYPED_TEST(TestPlanner, MatchReturnOrderBy) {
auto ret = RETURN(as_m, ORDER_BY(PROPERTY_LOOKUP("n", prop)));
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), ret));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectProduce(),
ExpectOrderBy());
}
@ -555,7 +555,7 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
// WITH n AS new ORDER BY new.prop, r.prop WHERE m.prop < 42
FakeDbAccessor dba;
auto prop = dba.Property("prop");
auto r_type = dba.EdgeType("r");
auto r_type = "r";
AstStorage storage;
auto ident_n = IDENT("n");
auto new_prop = PROPERTY_LOOKUP("new", prop);
@ -573,7 +573,7 @@ TYPED_TEST(TestPlanner, CreateWithOrderByWhere) {
symbol_table.at(*r_prop->expression_), // `r` in ORDER BY
symbol_table.at(*m_prop->expression_), // `m` in WHERE
});
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(),
ExpectCreateExpand(), acc, ExpectProduce(), ExpectOrderBy(),
ExpectFilter());
@ -595,7 +595,7 @@ TYPED_TEST(TestPlanner, MatchMerge) {
// ON MATCH SET n.prop = 42 ON CREATE SET m = n
// RETURN n AS n
FakeDbAccessor dba;
auto r_type = dba.EdgeType("r");
auto r_type = "r";
auto prop = dba.Property("prop");
AstStorage storage;
auto ident_n = IDENT("n");
@ -612,7 +612,7 @@ TYPED_TEST(TestPlanner, MatchMerge) {
auto symbol_table = query::MakeSymbolTable(query);
// We expect Accumulate after Merge, because it is considered as a write.
auto acc = ExpectAccumulate({symbol_table.at(*ident_n)});
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(),
ExpectMerge(on_match, on_create), acc, ExpectProduce());
for (auto &op : on_match) delete op;
@ -647,7 +647,7 @@ TYPED_TEST(TestPlanner, MatchUnwindReturn) {
RETURN(as_n, as_x)));
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectUnwind(),
ExpectProduce());
}
@ -675,7 +675,7 @@ TYPED_TEST(TestPlanner, CreateWithDistinctSumWhereReturn) {
auto symbol_table = query::MakeSymbolTable(query);
auto acc = ExpectAccumulate({symbol_table.at(*node_n->identifier_)});
auto aggr = ExpectAggregate({sum}, {});
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectCreateNode(), acc, aggr,
ExpectProduce(), ExpectDistinct(), ExpectFilter(), ExpectProduce());
}
@ -687,10 +687,10 @@ TYPED_TEST(TestPlanner, MatchCrossReferenceVariable) {
AstStorage storage;
auto node_n = NODE("n");
auto m_prop = PROPERTY_LOOKUP("m", prop.second);
node_n->properties_[prop] = m_prop;
node_n->properties_[storage.GetPropertyIx(prop.first)] = m_prop;
auto node_m = NODE("m");
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
node_m->properties_[prop] = n_prop;
node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
auto *query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n), PATTERN(node_m)), RETURN("n")));
// We expect both ScanAll to come before filters (2 are joined into one),
@ -710,7 +710,7 @@ TYPED_TEST(TestPlanner, MatchWhereBeforeExpand) {
WHERE(LESS(PROPERTY_LOOKUP("n", prop), LITERAL(42))), RETURN(as_n)));
// We expect Filter to come immediately after ScanAll, since it only uses `n`.
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectFilter(),
ExpectExpand(), ExpectProduce());
}
@ -757,7 +757,7 @@ TYPED_TEST(TestPlanner, MatchReturnAsterisk) {
auto query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("e"), NODE("m"))), ret));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
ExpectProduce());
std::vector<std::string> output_names;
@ -778,7 +778,7 @@ TYPED_TEST(TestPlanner, MatchReturnAsteriskSum) {
ret->body_.all_identifiers = true;
auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))), ret));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
auto *produce = dynamic_cast<Produce *>(&planner.plan());
ASSERT_TRUE(produce);
const auto &named_expressions = produce->named_expressions_;
@ -802,7 +802,7 @@ TYPED_TEST(TestPlanner, UnwindMergeNodeProperty) {
AstStorage storage;
FakeDbAccessor dba;
auto node_n = NODE("n");
node_n->properties_[PROPERTY_PAIR("prop")] = IDENT("i");
node_n->properties_[storage.GetPropertyIx("prop")] = IDENT("i");
auto *query = QUERY(
SINGLE_QUERY(UNWIND(LIST(LITERAL(1)), AS("i")), MERGE(PATTERN(node_n))));
std::list<BaseOpChecker *> on_match{new ExpectScanAll(), new ExpectFilter()};
@ -859,9 +859,9 @@ TYPED_TEST(TestPlanner, MapLiteralAggregationReturn) {
FakeDbAccessor dba;
auto sum = SUM(LITERAL(2));
auto group_by_literal = LITERAL(42);
auto *query =
QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum}), AS("result"),
group_by_literal, AS("group_by"))));
auto *query = QUERY(
SINGLE_QUERY(RETURN(MAP({storage.GetPropertyIx("sum"), sum}),
AS("result"), group_by_literal, AS("group_by"))));
auto aggr = ExpectAggregate({sum}, {group_by_literal});
CheckPlan<TypeParam>(query, storage, aggr, ExpectProduce());
}
@ -926,10 +926,10 @@ TYPED_TEST(TestPlanner, MapWithAggregationAndGroupBy) {
FakeDbAccessor dba;
auto sum = SUM(LITERAL(2));
auto group_by_literal = LITERAL(42);
auto *query =
QUERY(SINGLE_QUERY(RETURN(MAP({PROPERTY_PAIR("sum"), sum},
{PROPERTY_PAIR("lit"), group_by_literal}),
AS("result"))));
auto *query = QUERY(
SINGLE_QUERY(RETURN(MAP({storage.GetPropertyIx("sum"), sum},
{storage.GetPropertyIx("lit"), group_by_literal}),
AS("result"))));
auto aggr = ExpectAggregate({sum}, {group_by_literal});
CheckPlan<TypeParam>(query, storage, aggr, ExpectProduce());
}
@ -943,13 +943,13 @@ TYPED_TEST(TestPlanner, AtomIndexedLabelProperty) {
auto not_indexed = PROPERTY_PAIR("not_indexed");
dba.SetIndexCount(label, 1);
dba.SetIndexCount(label, property.second, 1);
auto node = NODE("n", label);
auto node = NODE("n", "label");
auto lit_42 = LITERAL(42);
node->properties_[property] = lit_42;
node->properties_[not_indexed] = LITERAL(0);
node->properties_[storage.GetPropertyIx(property.first)] = lit_42;
node->properties_[storage.GetPropertyIx(not_indexed.first)] = LITERAL(0);
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node)), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
ExpectFilter(), ExpectProduce());
@ -965,15 +965,16 @@ TYPED_TEST(TestPlanner, AtomPropertyWhereLabelIndexing) {
dba.SetIndexCount(label, property.second, 0);
auto node = NODE("n");
auto lit_42 = LITERAL(42);
node->properties_[property] = lit_42;
node->properties_[storage.GetPropertyIx(property.first)] = lit_42;
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(node)),
WHERE(AND(PROPERTY_LOOKUP("n", not_indexed),
storage.Create<query::LabelsTest>(
IDENT("n"), std::vector<storage::Label>{label}))),
IDENT("n"),
std::vector<query::LabelIx>{storage.GetLabelIx("label")}))),
RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
ExpectFilter(), ExpectProduce());
@ -988,10 +989,10 @@ TYPED_TEST(TestPlanner, WhereIndexedLabelProperty) {
dba.SetIndexCount(label, property.second, 0);
auto lit_42 = LITERAL(42);
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n", label))),
MATCH(PATTERN(NODE("n", "label"))),
WHERE(EQ(PROPERTY_LOOKUP("n", property), lit_42)), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyValue(label, property, lit_42),
ExpectProduce());
@ -1010,12 +1011,12 @@ TYPED_TEST(TestPlanner, BestPropertyIndexed) {
dba.SetIndexCount(label, better.second, 0);
auto lit_42 = LITERAL(42);
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
SINGLE_QUERY(MATCH(PATTERN(NODE("n", "label"))),
WHERE(AND(EQ(PROPERTY_LOOKUP("n", property), LITERAL(1)),
EQ(PROPERTY_LOOKUP("n", better), lit_42))),
RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyValue(label, better, lit_42),
ExpectFilter(), ExpectProduce());
@ -1035,12 +1036,12 @@ TYPED_TEST(TestPlanner, MultiPropertyIndexScan) {
auto lit_1 = LITERAL(1);
auto lit_2 = LITERAL(2);
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n", label1)), PATTERN(NODE("m", label2))),
MATCH(PATTERN(NODE("n", "label1")), PATTERN(NODE("m", "label2"))),
WHERE(AND(EQ(PROPERTY_LOOKUP("n", prop1), lit_1),
EQ(PROPERTY_LOOKUP("m", prop2), lit_2))),
RETURN("n", "m")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyValue(label1, prop1, lit_1),
ExpectScanAllByLabelPropertyValue(label2, prop2, lit_2),
@ -1062,10 +1063,12 @@ TYPED_TEST(TestPlanner, WhereIndexedLabelPropertyRange) {
auto upper_bound) {
// Shadow the first storage, so that the query is created in this one.
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", label))),
storage.GetLabelIx("label");
storage.GetPropertyIx("property");
auto *query = QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n", "label"))),
WHERE(rel_expr), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table,
ExpectScanAllByLabelPropertyRange(label, property, lower_bound,
upper_bound),
@ -1105,11 +1108,11 @@ TYPED_TEST(TestPlanner, UnableToUsePropertyIndex) {
dba.SetIndexCount(label, property, 0);
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n", label))),
MATCH(PATTERN(NODE("n", "label"))),
WHERE(EQ(PROPERTY_LOOKUP("n", property), PROPERTY_LOOKUP("n", property))),
RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// We can only get ScanAllByLabelIndex, because we are comparing properties
// with those on the same node.
CheckPlan(planner.plan(), symbol_table, ExpectScanAllByLabel(),
@ -1125,11 +1128,11 @@ TYPED_TEST(TestPlanner, SecondPropertyIndex) {
AstStorage storage;
auto n_prop = PROPERTY_LOOKUP("n", property);
auto m_prop = PROPERTY_LOOKUP("m", property);
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n", label)), PATTERN(NODE("m", label))),
WHERE(EQ(m_prop, n_prop)), RETURN("n")));
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n", "label")), PATTERN(NODE("m", "label"))),
WHERE(EQ(m_prop, n_prop)), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(
planner.plan(), symbol_table, ExpectScanAllByLabel(),
// Note: We are scanning for m, therefore property should equal n_prop.
@ -1171,11 +1174,11 @@ TYPED_TEST(TestPlanner, MatchExpandVariableNoBounds) {
TYPED_TEST(TestPlanner, MatchExpandVariableInlinedFilter) {
// Test MATCH (n) -[r :type * {prop: 42}]-> (m) RETURN r
FakeDbAccessor dba;
auto type = dba.EdgeType("type");
auto type = "type";
auto prop = PROPERTY_PAIR("prop");
AstStorage storage;
auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH, {type});
edge->properties_[prop] = LITERAL(42);
edge->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
CheckPlan<TypeParam>(
@ -1187,11 +1190,12 @@ TYPED_TEST(TestPlanner, MatchExpandVariableInlinedFilter) {
TYPED_TEST(TestPlanner, MatchExpandVariableNotInlinedFilter) {
// Test MATCH (n) -[r :type * {prop: m.prop}]-> (m) RETURN r
FakeDbAccessor dba;
auto type = dba.EdgeType("type");
auto type = "type";
auto prop = PROPERTY_PAIR("prop");
AstStorage storage;
auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH, {type});
edge->properties_[prop] = EQ(PROPERTY_LOOKUP("m", prop), LITERAL(42));
edge->properties_[storage.GetPropertyIx(prop.first)] =
EQ(PROPERTY_LOOKUP("m", prop), LITERAL(42));
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("r")));
CheckPlan<TypeParam>(query, storage, ExpectScanAll(), ExpectExpandVariable(),
@ -1210,7 +1214,7 @@ TYPED_TEST(TestPlanner, MatchExpandVariableTotalWeightSymbol) {
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), RETURN("*")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
auto *root = dynamic_cast<Produce *>(&planner.plan());
ASSERT_TRUE(root);
@ -1245,11 +1249,11 @@ TYPED_TEST(TestPlanner, UnwindMatchVariable) {
TYPED_TEST(TestPlanner, MatchBfs) {
// Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
FakeDbAccessor dba;
auto edge_type = dba.EdgeType("type");
AstStorage storage;
auto edge_type = storage.GetEdgeTypeIx("type");
auto *bfs = storage.Create<query::EdgeAtom>(
IDENT("r"), query::EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
std::vector<storage::EdgeType>{edge_type});
std::vector<query::EdgeTypeIx>{edge_type});
bfs->filter_lambda_.inner_edge = IDENT("r");
bfs->filter_lambda_.inner_node = IDENT("n");
bfs->filter_lambda_.expression = IDENT("n");
@ -1258,7 +1262,7 @@ TYPED_TEST(TestPlanner, MatchBfs) {
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), bfs, NODE("m"))), RETURN(as_r)));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpandBfs(),
ExpectProduce());
}
@ -1266,12 +1270,12 @@ TYPED_TEST(TestPlanner, MatchBfs) {
TYPED_TEST(TestPlanner, MatchDoubleScanToExpandExisting) {
// Test MATCH (n) -[r]- (m :label) RETURN r
FakeDbAccessor dba;
auto label = dba.Label("label");
auto label = "label";
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m", label))), RETURN("r")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// We expect 2x ScanAll and then Expand, since we are guessing that is
// faster (due to low label index vertex count).
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(),
@ -1288,12 +1292,12 @@ TYPED_TEST(TestPlanner, MatchScanToExpand) {
FLAGS_query_vertex_count_to_expand_existing + 1);
dba.SetIndexCount(label, FLAGS_query_vertex_count_to_expand_existing + 1);
AstStorage storage;
auto node_m = NODE("m", label);
node_m->properties_[std::make_pair("property", property)] = LITERAL(1);
auto node_m = NODE("m", "label");
node_m->properties_[storage.GetPropertyIx("property")] = LITERAL(1);
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), EDGE("r"), node_m)), RETURN("r")));
auto symbol_table = query::MakeSymbolTable(query);
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
// We expect 1x ScanAll and then Expand, since we are guessing that
// is faster (due to high label index vertex count).
CheckPlan(planner.plan(), symbol_table, ExpectScanAll(), ExpectExpand(),
@ -1327,7 +1331,7 @@ TYPED_TEST(TestPlanner, ReturnAsteriskOmitsLambdaSymbols) {
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"), edge, NODE("m"))), ret));
auto symbol_table = query::MakeSymbolTable(query);
FakeDbAccessor dba;
auto planner = MakePlanner<TypeParam>(dba, storage, symbol_table, query);
auto planner = MakePlanner<TypeParam>(&dba, storage, symbol_table, query);
auto *produce = dynamic_cast<Produce *>(&planner.plan());
ASSERT_TRUE(produce);
std::vector<std::string> outputs;

View File

@ -51,11 +51,11 @@ TEST(QueryPlan, Accumulate) {
auto n_p = PROPERTY_LOOKUP("n", prop);
symbol_table[*n_p->expression_] = n.sym_;
auto set_n_p =
std::make_shared<plan::SetProperty>(r_m.op_, n_p, ADD(n_p, one));
std::make_shared<plan::SetProperty>(r_m.op_, prop, n_p, ADD(n_p, one));
auto m_p = PROPERTY_LOOKUP("m", prop);
symbol_table[*m_p->expression_] = r_m.node_sym_;
auto set_m_p =
std::make_shared<plan::SetProperty>(set_n_p, m_p, ADD(m_p, one));
std::make_shared<plan::SetProperty>(set_n_p, prop, m_p, ADD(m_p, one));
std::shared_ptr<LogicalOperator> last_op = set_m_p;
if (accumulate) {
@ -68,7 +68,8 @@ TEST(QueryPlan, Accumulate) {
auto m_p_ne = NEXPR("m.p", m_p);
symbol_table[*m_p_ne] = symbol_table.CreateSymbol("m_p_ne", true);
auto produce = MakeProduce(last_op, n_p_ne, m_p_ne);
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
std::vector<int> results_data;
for (const auto &row : results)
for (const auto &column : row)
@ -97,7 +98,8 @@ TEST(QueryPlan, AccumulateAdvance) {
auto accumulate = std::make_shared<Accumulate>(
create, std::vector<Symbol>{node.symbol}, advance);
auto match = MakeScanAll(storage, symbol_table, "m", accumulate);
EXPECT_EQ(advance ? 1 : 0, PullAll(match.op_, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(advance ? 1 : 0, PullAll(*match.op_, &context));
};
check(false);
check(true);
@ -187,7 +189,8 @@ class QueryPlanAggregateOps : public ::testing::Test {
auto produce =
MakeAggregationProduce(n.op_, symbol_table, storage,
aggregation_expressions, ops, group_bys, {});
return CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
return CollectProduce(*produce, &context);
}
};
@ -330,7 +333,8 @@ TEST(QueryPlan, AggregateGroupByValues) {
MakeAggregationProduce(n.op_, symbol_table, storage, {n_p},
{Aggregation::Op::COUNT}, {n_p}, {n.sym_});
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(results.size(), group_by_vals.size() - 2);
TypedValue::unordered_set result_group_bys;
for (const auto &row : results) {
@ -378,7 +382,8 @@ TEST(QueryPlan, AggregateMultipleGroupBy) {
{Aggregation::Op::COUNT},
{n_p1, n_p2, n_p3}, {n.sym_});
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 2 * 3 * 5);
}
@ -394,7 +399,8 @@ TEST(QueryPlan, AggregateNoInput) {
auto produce = MakeAggregationProduce(nullptr, symbol_table, storage, {two},
{Aggregation::Op::COUNT}, {}, {});
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(1, results.size());
EXPECT_EQ(1, results[0].size());
EXPECT_EQ(TypedValue::Type::Int, results[0][0].type());
@ -427,7 +433,8 @@ TEST(QueryPlan, AggregateCountEdgeCases) {
auto count = [&]() {
auto produce = MakeAggregationProduce(n.op_, symbol_table, storage, {n_p},
{Aggregation::Op::COUNT}, {}, {});
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
if (results.size() == 0) return -1L;
EXPECT_EQ(1, results.size());
EXPECT_EQ(1, results[0].size());
@ -487,7 +494,8 @@ TEST(QueryPlan, AggregateFirstValueTypes) {
auto aggregate = [&](Expression *expression, Aggregation::Op aggr_op) {
auto produce = MakeAggregationProduce(n.op_, symbol_table, storage,
{expression}, {aggr_op}, {}, {});
CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
CollectProduce(*produce, &context);
};
// everything except for COUNT and COLLECT fails on a Vertex
@ -545,7 +553,8 @@ TEST(QueryPlan, AggregateTypes) {
auto aggregate = [&](Expression *expression, Aggregation::Op aggr_op) {
auto produce = MakeAggregationProduce(n.op_, symbol_table, storage,
{expression}, {aggr_op}, {}, {});
CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
CollectProduce(*produce, &context);
};
// everything except for COUNT and COLLECT fails on a Vertex
@ -602,7 +611,8 @@ TEST(QueryPlan, Unwind) {
symbol_table[*y_ne] = symbol_table.CreateSymbol("y_ne", true);
auto produce = MakeProduce(unwind_1, x_ne, y_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(4, results.size());
const std::vector<int> expected_x_card{3, 3, 3, 1};
auto expected_x_card_it = expected_x_card.begin();

View File

@ -31,23 +31,24 @@ TEST(QueryPlan, Skip) {
auto n = MakeScanAll(storage, symbol_table, "n1");
auto skip = std::make_shared<plan::Skip>(n.op_, LITERAL(2));
EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(0, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(0, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(0, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(1, PullAll(*skip, &context));
for (int i = 0; i < 10; ++i) dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(11, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(11, PullAll(*skip, &context));
}
TEST(QueryPlan, Limit) {
@ -60,23 +61,24 @@ TEST(QueryPlan, Limit) {
auto n = MakeScanAll(storage, symbol_table, "n1");
auto skip = std::make_shared<plan::Limit>(n.op_, LITERAL(2));
EXPECT_EQ(0, PullAll(skip, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(0, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(1, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(2, PullAll(*skip, &context));
dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(2, PullAll(*skip, &context));
for (int i = 0; i < 10; ++i) dba->InsertVertex();
dba->AdvanceCommand();
EXPECT_EQ(2, PullAll(skip, *dba, symbol_table));
EXPECT_EQ(2, PullAll(*skip, &context));
}
TEST(QueryPlan, CreateLimit) {
@ -98,7 +100,8 @@ TEST(QueryPlan, CreateLimit) {
auto c = std::make_shared<CreateNode>(n.op_, m);
auto skip = std::make_shared<plan::Limit>(c, LITERAL(1));
EXPECT_EQ(1, PullAll(skip, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*skip, &context));
dba->AdvanceCommand();
EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
}
@ -156,7 +159,8 @@ TEST(QueryPlan, OrderBy) {
auto n_p_ne = NEXPR("n.p", n_p);
symbol_table[*n_p_ne] = symbol_table.CreateSymbol("n.p", true);
auto produce = MakeProduce(order_by, n_p_ne);
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(values.size(), results.size());
for (int j = 0; j < results.size(); ++j)
EXPECT_TRUE(TypedValue::BoolEqual{}(results[j][0], values[j]));
@ -210,7 +214,8 @@ TEST(QueryPlan, OrderByMultiple) {
auto n_p2_ne = NEXPR("n.p2", n_p2);
symbol_table[*n_p2_ne] = symbol_table.CreateSymbol("n.p2", true);
auto produce = MakeProduce(order_by, n_p1_ne, n_p2_ne);
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(N * N, results.size());
for (int j = 0; j < N * N; ++j) {
ASSERT_EQ(results[j][0].type(), TypedValue::Type::Int);
@ -261,6 +266,7 @@ TEST(QueryPlan, OrderByExceptions) {
auto order_by = std::make_shared<plan::OrderBy>(
n.op_, std::vector<SortItem>{{Ordering::ASC, n_p}},
std::vector<Symbol>{});
EXPECT_THROW(PullAll(order_by, dba, symbol_table), QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_THROW(PullAll(*order_by, &context), QueryRuntimeException);
}
}

View File

@ -341,10 +341,10 @@ std::list<std::unique_ptr<BaseOpChecker>> MakeCheckers(T arg, Rest &&... rest) {
}
template <class TPlanner, class TDbAccessor>
TPlanner MakePlanner(const TDbAccessor &dba, AstStorage &storage,
TPlanner MakePlanner(TDbAccessor *dba, AstStorage &storage,
SymbolTable &symbol_table, CypherQuery *query) {
auto planning_context =
MakePlanningContext(storage, symbol_table, query, dba);
MakePlanningContext(&storage, &symbol_table, query, dba);
auto query_parts = CollectQueryParts(symbol_table, storage, query);
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
return TPlanner(single_query_parts, planning_context);

View File

@ -17,26 +17,33 @@ using namespace query::plan;
using Bound = ScanAllByLabelPropertyRange::Bound;
Context MakeContext(const AstStorage &storage, const SymbolTable &symbol_table,
database::GraphDbAccessor *dba) {
Context context(*dba);
context.symbol_table_ = symbol_table;
context.evaluation_context_.properties =
NamesToProperties(storage.properties_, dba);
context.evaluation_context_.labels = NamesToLabels(storage.labels_, dba);
return context;
}
/** Helper function that collects all the results from the given Produce. */
std::vector<std::vector<TypedValue>> CollectProduce(
Produce *produce, SymbolTable &symbol_table,
database::GraphDbAccessor &db_accessor) {
Frame frame(symbol_table.max_position());
std::vector<std::vector<TypedValue>> CollectProduce(const Produce &produce,
Context *context) {
Frame frame(context->symbol_table_.max_position());
// top level node in the operator tree is a produce (return)
// so stream out results
// collect the symbols from the return clause
std::vector<Symbol> symbols;
for (auto named_expression : produce->named_expressions_)
symbols.emplace_back(symbol_table[*named_expression]);
for (auto named_expression : produce.named_expressions_)
symbols.emplace_back(context->symbol_table_[*named_expression]);
Context context(db_accessor);
context.symbol_table_ = symbol_table;
// stream out results
auto cursor = produce->MakeCursor(db_accessor);
auto cursor = produce.MakeCursor(context->db_accessor_);
std::vector<std::vector<TypedValue>> results;
while (cursor->Pull(frame, context)) {
while (cursor->Pull(frame, *context)) {
std::vector<TypedValue> values;
for (auto &symbol : symbols) values.emplace_back(frame[symbol]);
results.emplace_back(values);
@ -45,14 +52,11 @@ std::vector<std::vector<TypedValue>> CollectProduce(
return results;
}
int PullAll(std::shared_ptr<LogicalOperator> logical_op,
database::GraphDbAccessor &db, SymbolTable &symbol_table) {
Frame frame(symbol_table.max_position());
auto cursor = logical_op->MakeCursor(db);
int PullAll(const LogicalOperator &logical_op, Context *context) {
Frame frame(context->symbol_table_.max_position());
auto cursor = logical_op.MakeCursor(context->db_accessor_);
int count = 0;
Context context(db);
context.symbol_table_ = symbol_table;
while (cursor->Pull(frame, context)) count++;
while (cursor->Pull(frame, *context)) count++;
return count;
}
@ -114,6 +118,7 @@ ScanAllTuple MakeScanAllByLabel(
ScanAllTuple MakeScanAllByLabelPropertyRange(
AstStorage &storage, SymbolTable &symbol_table, std::string identifier,
storage::Label label, storage::Property property,
const std::string &property_name,
std::experimental::optional<Bound> lower_bound,
std::experimental::optional<Bound> upper_bound,
std::shared_ptr<LogicalOperator> input = {nullptr},
@ -122,7 +127,8 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
auto symbol = symbol_table.CreateSymbol(identifier, true);
symbol_table[*node->identifier_] = symbol;
auto logical_op = std::make_shared<ScanAllByLabelPropertyRange>(
input, symbol, label, property, lower_bound, upper_bound, graph_view);
input, symbol, label, property, property_name, lower_bound, upper_bound,
graph_view);
return ScanAllTuple{node, logical_op, symbol};
}
@ -134,14 +140,15 @@ ScanAllTuple MakeScanAllByLabelPropertyRange(
*/
ScanAllTuple MakeScanAllByLabelPropertyValue(
AstStorage &storage, SymbolTable &symbol_table, std::string identifier,
storage::Label label, storage::Property property, Expression *value,
storage::Label label, storage::Property property,
const std::string &property_name, Expression *value,
std::shared_ptr<LogicalOperator> input = {nullptr},
GraphView graph_view = GraphView::OLD) {
auto node = NODE(identifier);
auto symbol = symbol_table.CreateSymbol(identifier, true);
symbol_table[*node->identifier_] = symbol;
auto logical_op = std::make_shared<ScanAllByLabelPropertyValue>(
input, symbol, label, property, value, graph_view);
input, symbol, label, property, property_name, value, graph_view);
return ScanAllTuple{node, logical_op, symbol};
}

View File

@ -33,7 +33,8 @@ TEST(QueryPlan, CreateNodeWithAttributes) {
node.properties.emplace_back(property.second, LITERAL(42));
auto create = std::make_shared<CreateNode>(nullptr, node);
PullAll(create, dba, symbol_table);
Context context = MakeContext(storage, symbol_table, &dba);
PullAll(*create, &context);
dba.AdvanceCommand();
// count the number of vertices
@ -79,7 +80,8 @@ TEST(QueryPlan, CreateReturn) {
symbol_table[*named_expr_n->expression_] = node.symbol;
auto produce = MakeProduce(create, named_expr_n, named_expr_n_p);
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(1, results.size());
EXPECT_EQ(2, results[0].size());
EXPECT_EQ(TypedValue::Type::Vertex, results[0][0].type());
@ -130,7 +132,8 @@ TEST(QueryPlan, CreateExpand) {
auto create_op = std::make_shared<CreateNode>(nullptr, n);
auto create_expand =
std::make_shared<CreateExpand>(m, r, create_op, n.symbol, cycle);
PullAll(create_expand, dba, symbol_table);
Context context = MakeContext(storage, symbol_table, &dba);
PullAll(*create_expand, &context);
dba.AdvanceCommand();
EXPECT_EQ(CountIterable(dba.Vertices(false)) - before_v,
@ -185,7 +188,8 @@ TEST(QueryPlan, MatchCreateNode) {
auto create_node = std::make_shared<CreateNode>(n_scan_all.op_, m);
EXPECT_EQ(CountIterable(dba->Vertices(false)), 3);
PullAll(create_node, *dba, symbol_table);
Context context = MakeContext(storage, symbol_table, dba.get());
PullAll(*create_node, &context);
dba->AdvanceCommand();
EXPECT_EQ(CountIterable(dba->Vertices(false)), 6);
}
@ -227,7 +231,8 @@ TEST(QueryPlan, MatchCreateExpand) {
auto create_expand = std::make_shared<CreateExpand>(m, r, n_scan_all.op_,
n_scan_all.sym_, cycle);
PullAll(create_expand, *dba, symbol_table);
Context context = MakeContext(storage, symbol_table, dba.get());
PullAll(*create_expand, &context);
dba->AdvanceCommand();
EXPECT_EQ(CountIterable(dba->Vertices(false)) - before_v,
@ -266,7 +271,8 @@ TEST(QueryPlan, Delete) {
symbol_table[*n_get] = n.sym_;
auto delete_op = std::make_shared<plan::Delete>(
n.op_, std::vector<Expression *>{n_get}, false);
EXPECT_THROW(PullAll(delete_op, *dba, symbol_table), QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*delete_op, &context), QueryRuntimeException);
dba->AdvanceCommand();
EXPECT_EQ(4, CountIterable(dba->Vertices(false)));
EXPECT_EQ(6, CountIterable(dba->Edges(false)));
@ -280,8 +286,7 @@ TEST(QueryPlan, Delete) {
auto delete_op = std::make_shared<plan::Delete>(
n.op_, std::vector<Expression *>{n_get}, true);
Frame frame(symbol_table.max_position());
Context context(*dba);
context.symbol_table_ = symbol_table;
Context context = MakeContext(storage, symbol_table, dba.get());
delete_op->MakeCursor(*dba)->Pull(frame, context);
dba->AdvanceCommand();
EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
@ -298,7 +303,8 @@ TEST(QueryPlan, Delete) {
symbol_table[*r_get] = r_m.edge_sym_;
auto delete_op = std::make_shared<plan::Delete>(
r_m.op_, std::vector<Expression *>{r_get}, false);
PullAll(delete_op, *dba, symbol_table);
Context context = MakeContext(storage, symbol_table, dba.get());
PullAll(*delete_op, &context);
dba->AdvanceCommand();
EXPECT_EQ(3, CountIterable(dba->Vertices(false)));
EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@ -311,7 +317,8 @@ TEST(QueryPlan, Delete) {
symbol_table[*n_get] = n.sym_;
auto delete_op = std::make_shared<plan::Delete>(
n.op_, std::vector<Expression *>{n_get}, false);
PullAll(delete_op, *dba, symbol_table);
Context context = MakeContext(storage, symbol_table, dba.get());
PullAll(*delete_op, &context);
dba->AdvanceCommand();
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@ -359,7 +366,8 @@ TEST(QueryPlan, DeleteTwiceDeleteBlockingEdge) {
auto delete_op = std::make_shared<plan::Delete>(
r_m.op_, std::vector<Expression *>{n_get, r_get, m_get}, detach);
EXPECT_EQ(2, PullAll(delete_op, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(2, PullAll(*delete_op, &context));
dba->AdvanceCommand();
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(0, CountIterable(dba->Edges(false)));
@ -395,14 +403,14 @@ TEST(QueryPlan, DeleteReturn) {
auto delete_op = std::make_shared<plan::Delete>(
n.op_, std::vector<Expression *>{n_get}, true);
auto prop_lookup =
storage.Create<PropertyLookup>(storage.Create<Identifier>("n"), prop);
auto prop_lookup = PROPERTY_LOOKUP("n", prop);
symbol_table[*prop_lookup->expression_] = n.sym_;
auto n_p = storage.Create<NamedExpression>("n", prop_lookup);
symbol_table[*n_p] = symbol_table.CreateSymbol("bla", true);
auto produce = MakeProduce(delete_op, n_p);
auto results = CollectProduce(produce.get(), symbol_table, dba);
Context context = MakeContext(storage, symbol_table, &dba);
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(4, results.size());
dba.AdvanceCommand();
EXPECT_EQ(0, CountIterable(dba.Vertices(false)));
@ -418,7 +426,8 @@ TEST(QueryPlan, DeleteNull) {
auto once = std::make_shared<Once>();
auto delete_op = std::make_shared<plan::Delete>(
once, std::vector<Expression *>{LITERAL(TypedValue::Null)}, false);
EXPECT_EQ(1, PullAll(delete_op, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*delete_op, &context));
}
TEST(QueryPlan, DeleteAdvance) {
@ -446,7 +455,8 @@ TEST(QueryPlan, DeleteAdvance) {
n.op_, std::vector<Expression *>{n_get}, false);
auto advance = std::make_shared<Accumulate>(
delete_op, std::vector<Symbol>{n.sym_}, true);
EXPECT_THROW(PullAll(advance, *dba, symbol_table), ReconstructionException);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*advance, &context), ReconstructionException);
}
TEST(QueryPlan, SetProperty) {
@ -481,12 +491,15 @@ TEST(QueryPlan, SetProperty) {
auto n_p = PROPERTY_LOOKUP("n", prop1);
symbol_table[*n_p->expression_] = n.sym_;
auto set_n_p = std::make_shared<plan::SetProperty>(r_m.op_, n_p, literal);
auto set_n_p =
std::make_shared<plan::SetProperty>(r_m.op_, prop1, n_p, literal);
auto r_p = PROPERTY_LOOKUP("r", prop1);
symbol_table[*r_p->expression_] = r_m.edge_sym_;
auto set_r_p = std::make_shared<plan::SetProperty>(set_n_p, r_p, literal);
EXPECT_EQ(2, PullAll(set_r_p, dba, symbol_table));
auto set_r_p =
std::make_shared<plan::SetProperty>(set_n_p, prop1, r_p, literal);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(2, PullAll(*set_r_p, &context));
dba.AdvanceCommand();
EXPECT_EQ(CountIterable(dba.Edges(false)), 2);
@ -539,7 +552,8 @@ TEST(QueryPlan, SetProperties) {
std::make_shared<plan::SetProperties>(r_m.op_, n.sym_, r_ident, op);
auto set_m_to_r = std::make_shared<plan::SetProperties>(
set_r_to_n, r_m.edge_sym_, m_ident, op);
EXPECT_EQ(1, PullAll(set_m_to_r, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*set_m_to_r, &context));
dba->AdvanceCommand();
EXPECT_EQ(CountIterable(dba->Edges(false)), 1);
@ -589,7 +603,8 @@ TEST(QueryPlan, SetLabels) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto label_set = std::make_shared<plan::SetLabels>(
n.op_, n.sym_, std::vector<storage::Label>{label2, label3});
EXPECT_EQ(2, PullAll(label_set, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(2, PullAll(*label_set, &context));
for (VertexAccessor vertex : dba->Vertices(false)) {
vertex.SwitchNew();
@ -634,12 +649,13 @@ TEST(QueryPlan, RemoveProperty) {
auto n_p = PROPERTY_LOOKUP("n", prop1);
symbol_table[*n_p->expression_] = n.sym_;
auto set_n_p = std::make_shared<plan::RemoveProperty>(r_m.op_, n_p);
auto set_n_p = std::make_shared<plan::RemoveProperty>(r_m.op_, prop1, n_p);
auto r_p = PROPERTY_LOOKUP("r", prop1);
symbol_table[*r_p->expression_] = r_m.edge_sym_;
auto set_r_p = std::make_shared<plan::RemoveProperty>(set_n_p, r_p);
EXPECT_EQ(2, PullAll(set_r_p, dba, symbol_table));
auto set_r_p = std::make_shared<plan::RemoveProperty>(set_n_p, prop1, r_p);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(2, PullAll(*set_r_p, &context));
dba.AdvanceCommand();
EXPECT_EQ(CountIterable(dba.Edges(false)), 2);
@ -675,7 +691,8 @@ TEST(QueryPlan, RemoveLabels) {
auto n = MakeScanAll(storage, symbol_table, "n");
auto label_remove = std::make_shared<plan::RemoveLabels>(
n.op_, n.sym_, std::vector<storage::Label>{label1, label2});
EXPECT_EQ(2, PullAll(label_remove, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(2, PullAll(*label_remove, &context));
for (VertexAccessor vertex : dba->Vertices(false)) {
vertex.SwitchNew();
@ -706,20 +723,23 @@ TEST(QueryPlan, NodeFilterSet) {
SymbolTable symbol_table;
// MATCH (n {prop: 42}) -[r]- (m)
auto scan_all = MakeScanAll(storage, symbol_table, "n");
scan_all.node_->properties_[prop] = LITERAL(42);
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);
auto *filter_expr =
EQ(storage.Create<PropertyLookup>(scan_all.node_->identifier_, prop),
EQ(storage.Create<PropertyLookup>(scan_all.node_->identifier_,
storage.GetPropertyIx(prop.first)),
LITERAL(42));
auto node_filter = std::make_shared<Filter>(expand.op_, filter_expr);
// SET n.prop = n.prop + 1
auto set_prop = PROPERTY_LOOKUP("n", prop);
symbol_table[*set_prop->expression_] = scan_all.sym_;
auto add = ADD(set_prop, LITERAL(1));
auto set = std::make_shared<plan::SetProperty>(node_filter, set_prop, add);
EXPECT_EQ(2, PullAll(set, dba, symbol_table));
auto set = std::make_shared<plan::SetProperty>(node_filter, prop.second,
set_prop, add);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(2, PullAll(*set, &context));
dba.AdvanceCommand();
v1.Reconstruct();
auto prop_eq = v1.PropsAt(prop.second) == TypedValue(42 + 2);
@ -747,7 +767,7 @@ TEST(QueryPlan, FilterRemove) {
SymbolTable symbol_table;
// MATCH (n) -[r]- (m) WHERE n.prop < 43
auto scan_all = MakeScanAll(storage, symbol_table, "n");
scan_all.node_->properties_[prop] = LITERAL(42);
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);
@ -758,8 +778,10 @@ TEST(QueryPlan, FilterRemove) {
// REMOVE n.prop
auto rem_prop = PROPERTY_LOOKUP("n", prop);
symbol_table[*rem_prop->expression_] = scan_all.sym_;
auto rem = std::make_shared<plan::RemoveProperty>(filter, rem_prop);
EXPECT_EQ(2, PullAll(rem, dba, symbol_table));
auto rem =
std::make_shared<plan::RemoveProperty>(filter, prop.second, rem_prop);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(2, PullAll(*rem, &context));
dba.AdvanceCommand();
v1.Reconstruct();
EXPECT_EQ(v1.PropsAt(prop.second).type(), PropertyValue::Type::Null);
@ -782,7 +804,8 @@ TEST(QueryPlan, SetRemove) {
scan_all.op_, scan_all.sym_, std::vector<storage::Label>{label1, label2});
auto rem = std::make_shared<plan::RemoveLabels>(
set, scan_all.sym_, std::vector<storage::Label>{label1, label2});
EXPECT_EQ(1, PullAll(rem, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*rem, &context));
dba->AdvanceCommand();
v.Reconstruct();
EXPECT_FALSE(v.has_label(label1));
@ -817,16 +840,18 @@ TEST(QueryPlan, Merge) {
EdgeAtom::Direction::BOTH, {}, "m", false, GraphView::OLD);
auto m_p = PROPERTY_LOOKUP("m", prop);
symbol_table[*m_p->expression_] = r_m.node_sym_;
auto m_set = std::make_shared<plan::SetProperty>(r_m.op_, m_p, LITERAL(1));
auto m_set = std::make_shared<plan::SetProperty>(r_m.op_, prop.second, m_p,
LITERAL(1));
// merge_create branch
auto n_p = PROPERTY_LOOKUP("n", prop);
symbol_table[*n_p->expression_] = n.sym_;
auto n_set = std::make_shared<plan::SetProperty>(std::make_shared<Once>(),
n_p, LITERAL(2));
auto n_set = std::make_shared<plan::SetProperty>(
std::make_shared<Once>(), prop.second, n_p, LITERAL(2));
auto merge = std::make_shared<plan::Merge>(n.op_, m_set, n_set);
ASSERT_EQ(3, PullAll(merge, dba, symbol_table));
Context context = MakeContext(storage, symbol_table, &dba);
ASSERT_EQ(3, PullAll(*merge, &context));
dba.AdvanceCommand();
v1.Reconstruct();
v2.Reconstruct();
@ -854,7 +879,8 @@ TEST(QueryPlan, MergeNoInput) {
auto merge = std::make_shared<plan::Merge>(nullptr, create, create);
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(1, PullAll(merge, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*merge, &context));
dba->AdvanceCommand();
EXPECT_EQ(1, CountIterable(dba->Vertices(false)));
}
@ -869,10 +895,12 @@ TEST(QueryPlan, SetPropertyOnNull) {
auto prop = PROPERTY_PAIR("property");
auto null = LITERAL(TypedValue::Null);
auto literal = LITERAL(42);
auto n_prop = storage.Create<PropertyLookup>(null, prop);
auto n_prop = PROPERTY_LOOKUP(null, prop);
auto once = std::make_shared<Once>();
auto set_op = std::make_shared<plan::SetProperty>(once, n_prop, literal);
EXPECT_EQ(1, PullAll(set_op, dba, symbol_table));
auto set_op =
std::make_shared<plan::SetProperty>(once, prop.second, n_prop, literal);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(1, PullAll(*set_op, &context));
}
TEST(QueryPlan, SetPropertiesOnNull) {
@ -889,7 +917,8 @@ TEST(QueryPlan, SetPropertiesOnNull) {
auto set_op = std::make_shared<plan::SetProperties>(
optional, n.sym_, n_ident, plan::SetProperties::Op::REPLACE);
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*set_op, &context));
}
TEST(QueryPlan, SetLabelsOnNull) {
@ -907,7 +936,8 @@ TEST(QueryPlan, SetLabelsOnNull) {
auto set_op = std::make_shared<plan::SetLabels>(
optional, n.sym_, std::vector<storage::Label>{label});
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(1, PullAll(set_op, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*set_op, &context));
}
TEST(QueryPlan, RemovePropertyOnNull) {
@ -919,10 +949,12 @@ TEST(QueryPlan, RemovePropertyOnNull) {
SymbolTable symbol_table;
auto prop = PROPERTY_PAIR("property");
auto null = LITERAL(TypedValue::Null);
auto n_prop = storage.Create<PropertyLookup>(null, prop);
auto n_prop = PROPERTY_LOOKUP(null, prop);
auto once = std::make_shared<Once>();
auto remove_op = std::make_shared<plan::RemoveProperty>(once, n_prop);
EXPECT_EQ(1, PullAll(remove_op, dba, symbol_table));
auto remove_op =
std::make_shared<plan::RemoveProperty>(once, prop.second, n_prop);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(1, PullAll(*remove_op, &context));
}
TEST(QueryPlan, RemoveLabelsOnNull) {
@ -940,7 +972,8 @@ TEST(QueryPlan, RemoveLabelsOnNull) {
auto remove_op = std::make_shared<plan::RemoveLabels>(
optional, n.sym_, std::vector<storage::Label>{label});
EXPECT_EQ(0, CountIterable(dba->Vertices(false)));
EXPECT_EQ(1, PullAll(remove_op, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*remove_op, &context));
}
TEST(QueryPlan, DeleteSetProperty) {
@ -962,9 +995,10 @@ TEST(QueryPlan, DeleteSetProperty) {
auto prop = PROPERTY_PAIR("property");
auto n_prop = PROPERTY_LOOKUP("n", prop);
symbol_table[*n_prop->expression_] = n.sym_;
auto set_op =
std::make_shared<plan::SetProperty>(delete_op, n_prop, LITERAL(42));
EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
auto set_op = std::make_shared<plan::SetProperty>(delete_op, prop.second,
n_prop, LITERAL(42));
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
}
TEST(QueryPlan, DeleteSetPropertiesFromMap) {
@ -986,16 +1020,16 @@ TEST(QueryPlan, DeleteSetPropertiesFromMap) {
auto prop = PROPERTY_PAIR("property");
auto n_prop = PROPERTY_LOOKUP("n", prop);
symbol_table[*n_prop->expression_] = n.sym_;
std::unordered_map<std::pair<std::string, storage::Property>, Expression *>
prop_map;
prop_map.emplace(prop, LITERAL(42));
std::unordered_map<PropertyIx, Expression *> prop_map;
prop_map.emplace(storage.GetPropertyIx(prop.first), LITERAL(42));
auto *rhs = storage.Create<MapLiteral>(prop_map);
symbol_table[*rhs] = n.sym_;
for (auto op_type :
{plan::SetProperties::Op::REPLACE, plan::SetProperties::Op::UPDATE}) {
auto set_op =
std::make_shared<plan::SetProperties>(delete_op, n.sym_, rhs, op_type);
EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
}
}
@ -1027,7 +1061,8 @@ TEST(QueryPlan, DeleteSetPropertiesFromVertex) {
{plan::SetProperties::Op::REPLACE, plan::SetProperties::Op::UPDATE}) {
auto set_op =
std::make_shared<plan::SetProperties>(delete_op, n.sym_, rhs, op_type);
EXPECT_THROW(PullAll(set_op, dba, symbol_table), QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_THROW(PullAll(*set_op, &context), QueryRuntimeException);
}
}
@ -1048,7 +1083,8 @@ TEST(QueryPlan, DeleteRemoveLabels) {
n.op_, std::vector<Expression *>{n_get}, false);
std::vector<storage::Label> labels{dba->Label("label")};
auto rem_op = std::make_shared<plan::RemoveLabels>(delete_op, n.sym_, labels);
EXPECT_THROW(PullAll(rem_op, *dba, symbol_table), QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*rem_op, &context), QueryRuntimeException);
}
TEST(QueryPlan, DeleteRemoveProperty) {
@ -1070,6 +1106,8 @@ TEST(QueryPlan, DeleteRemoveProperty) {
auto prop = PROPERTY_PAIR("property");
auto n_prop = PROPERTY_LOOKUP("n", prop);
symbol_table[*n_prop->expression_] = n.sym_;
auto rem_op = std::make_shared<plan::RemoveProperty>(delete_op, n_prop);
EXPECT_THROW(PullAll(rem_op, dba, symbol_table), QueryRuntimeException);
auto rem_op =
std::make_shared<plan::RemoveProperty>(delete_op, prop.second, n_prop);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_THROW(PullAll(*rem_op, &context), QueryRuntimeException);
}

View File

@ -38,7 +38,8 @@ class MatchReturnFixture : public testing::Test {
template <typename TResult>
std::vector<TResult> Results(std::shared_ptr<Produce> &op) {
std::vector<TResult> res;
for (const auto &row : CollectProduce(op.get(), symbol_table, *dba_))
Context context = MakeContext(storage, symbol_table, dba_.get());
for (const auto &row : CollectProduce(*op, &context))
res.emplace_back(row[0].Value<TResult>());
return res;
}
@ -56,7 +57,8 @@ TEST_F(MatchReturnFixture, MatchReturn) {
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
return PullAll(produce, *dba_, symbol_table);
Context context = MakeContext(storage, symbol_table, dba_.get());
return PullAll(*produce, &context);
};
EXPECT_EQ(2, test_pull_count(GraphView::NEW));
@ -111,8 +113,8 @@ TEST(QueryPlan, MatchReturnCartesian) {
symbol_table[*return_m] =
symbol_table.CreateSymbol("named_expression_2", true);
auto produce = MakeProduce(m.op_, return_n, return_m);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 4);
// ensure the result ordering is OK:
// "n" from the results is the same for the first two rows, while "m" isn't
@ -138,7 +140,8 @@ TEST(QueryPlan, StandaloneReturn) {
auto produce = MakeProduce(std::shared_ptr<LogicalOperator>(nullptr), output);
symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 1);
EXPECT_EQ(results[0].size(), 1);
EXPECT_EQ(results[0][0].Value<int64_t>(), 42);
@ -176,8 +179,8 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
// make a scan all
auto n = MakeScanAll(storage, symbol_table, "n");
n.node_->labels_.emplace_back(label);
n.node_->properties_[property] = LITERAL(42);
n.node_->labels_.emplace_back(storage.GetLabelIx(dba.LabelName(label)));
n.node_->properties_[storage.GetPropertyIx(property.first)] = LITERAL(42);
// node filtering
auto *filter_expr =
@ -191,14 +194,15 @@ TEST(QueryPlan, NodeFilterLabelsAndProperties) {
symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
auto produce = MakeProduce(node_filter, output);
EXPECT_EQ(1, PullAll(produce, dba, symbol_table));
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(1, PullAll(*produce, &context));
// test that filtering works with old records
v4.Reconstruct();
v4.add_label(label);
EXPECT_EQ(1, PullAll(produce, dba, symbol_table));
EXPECT_EQ(1, PullAll(*produce, &context));
dba.AdvanceCommand();
EXPECT_EQ(2, PullAll(produce, dba, symbol_table));
EXPECT_EQ(2, PullAll(*produce, &context));
}
TEST(QueryPlan, NodeFilterMultipleLabels) {
@ -231,8 +235,8 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
// make a scan all
auto n = MakeScanAll(storage, symbol_table, "n");
n.node_->labels_.emplace_back(label1);
n.node_->labels_.emplace_back(label2);
n.node_->labels_.emplace_back(storage.GetLabelIx(dba->LabelName(label1)));
n.node_->labels_.emplace_back(storage.GetLabelIx(dba->LabelName(label2)));
// node filtering
auto *filter_expr =
@ -247,7 +251,8 @@ TEST(QueryPlan, NodeFilterMultipleLabels) {
symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
symbol_table[*output->expression_] = n.sym_;
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 2);
}
@ -286,7 +291,8 @@ TEST(QueryPlan, Cartesian) {
auto produce = MakeProduce(cartesian_op, return_n, return_m);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 9);
for (int i = 0; i < 3; ++i) {
for (int j = 0; j < 3; ++j) {
@ -320,8 +326,8 @@ TEST(QueryPlan, CartesianEmptySet) {
std::make_shared<Cartesian>(n.op_, left_symbols, m.op_, right_symbols);
auto produce = MakeProduce(cartesian_op, return_n, return_m);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 0);
}
@ -368,8 +374,8 @@ TEST(QueryPlan, CartesianThreeWay) {
l.op_, l_symbols);
auto produce = MakeProduce(cartesian_op_2, return_n, return_m, return_l);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 27);
int id = 0;
for (int i = 0; i < 3; ++i) {
@ -419,8 +425,8 @@ TEST_F(ExpandFixture, Expand) {
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
auto produce = MakeProduce(r_m.op_, output);
return PullAll(produce, *dba_, symbol_table);
Context context = MakeContext(storage, symbol_table, dba_.get());
return PullAll(*produce, &context);
};
// test that expand works well for both old and new graph state
@ -456,7 +462,8 @@ TEST_F(ExpandFixture, ExpandPath) {
auto produce = MakeProduce(path, output);
std::vector<query::Path> expected_paths{{v1, r2, v3}, {v1, r1, v2}};
auto results = CollectProduce(produce.get(), symbol_table, *dba_);
Context context = MakeContext(storage, symbol_table, dba_.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(results.size(), 2);
std::vector<query::Path> results_paths;
for (const auto &result : results)
@ -546,9 +553,10 @@ class QueryPlanExpandVariable : public testing::Test {
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_, storage.Create<query::LabelsTest>(
n_from.node_->identifier_,
std::vector<storage::Label>{labels[layer]}));
n_from.op_,
storage.Create<query::LabelsTest>(
n_from.node_->identifier_, std::vector<LabelIx>{storage.GetLabelIx(
dba_->LabelName(labels[layer]))}));
auto n_to = NODE(node_to);
auto n_to_sym = symbol_table.CreateSymbol(node_to, true);
@ -592,8 +600,7 @@ class QueryPlanExpandVariable : public testing::Test {
auto GetResults(std::shared_ptr<LogicalOperator> input_op, Symbol symbol) {
Frame frame(symbol_table.max_position());
auto cursor = input_op->MakeCursor(*dba_);
Context context(*dba_);
context.symbol_table_ = symbol_table;
Context context = MakeContext(storage, symbol_table, dba_.get());
std::vector<TResult> results;
while (cursor->Pull(frame, context))
results.emplace_back(frame[symbol].Value<TResult>());
@ -710,8 +717,8 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessSingleAndVariableExpansion) {
if (add_uniqueness_check) {
auto last_symbol = symbols.back();
symbols.pop_back();
last_op = std::make_shared<EdgeUniquenessFilter>(
last_op, last_symbol, symbols);
last_op =
std::make_shared<EdgeUniquenessFilter>(last_op, last_symbol, symbols);
}
return GetEdgeListSizes(last_op, var_length_sym);
@ -741,8 +748,8 @@ TEST_F(QueryPlanExpandVariable, EdgeUniquenessTwoVariableExpansions) {
AddMatch<ExpandVariable>(first, "n2", layer, direction, {}, lower,
upper, e2, "m2", GraphView::OLD);
if (add_uniqueness_check) {
last_op = std::make_shared<EdgeUniquenessFilter>(
last_op, e2, std::vector<Symbol>{e1});
last_op = std::make_shared<EdgeUniquenessFilter>(last_op, e2,
std::vector<Symbol>{e1});
}
return GetEdgeListSizes(last_op, e2);
@ -929,8 +936,7 @@ class QueryPlanExpandWeightedShortestPath : public testing::Test {
Frame frame(symbol_table.max_position());
auto cursor = last_op->MakeCursor(dba);
std::vector<ResultType> results;
Context context(dba);
context.symbol_table_ = symbol_table;
Context context = MakeContext(storage, symbol_table, &dba);
while (cursor->Pull(frame, context)) {
results.push_back(ResultType{std::vector<EdgeAccessor>(),
frame[node_sym].Value<VertexAccessor>(),
@ -1231,8 +1237,8 @@ TEST(QueryPlan, ExpandOptional) {
symbol_table[*m_ne->expression_] = r_m.node_sym_;
symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
auto produce = MakeProduce(optional, n_ne, r_ne, m_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(4, results.size());
int v1_is_n_count = 0;
for (auto &row : results) {
@ -1268,8 +1274,8 @@ TEST(QueryPlan, OptionalMatchEmptyDB) {
auto optional = std::make_shared<plan::Optional>(nullptr, n.op_,
std::vector<Symbol>{n.sym_});
auto produce = MakeProduce(optional, n_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(1, results.size());
EXPECT_EQ(results[0][0].type(), TypedValue::Type::Null);
}
@ -1298,7 +1304,8 @@ TEST(QueryPlan, OptionalMatchEmptyDBExpandFromNode) {
symbol_table[*m_ne->expression_] = r_m.node_sym_;
symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
auto produce = MakeProduce(r_m.op_, m_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(0, results.size());
}
@ -1317,8 +1324,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
SymbolTable symbol_table;
// OPTIONAL MATCH (n :missing)
auto n = MakeScanAll(storage, symbol_table, "n");
auto label_missing = dba->Label("missing");
n.node_->labels_.emplace_back(label_missing);
auto label_missing = "missing";
n.node_->labels_.emplace_back(storage.GetLabelIx(label_missing));
auto *filter_expr =
storage.Create<LabelsTest>(n.node_->identifier_, n.node_->labels_);
@ -1347,7 +1354,8 @@ TEST(QueryPlan, OptionalMatchThenExpandToMissingNode) {
symbol_table[*m_ne->expression_] = m.sym_;
symbol_table[*m_ne] = symbol_table.CreateSymbol("m", true);
auto produce = MakeProduce(expand, m_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(0, results.size());
}
@ -1383,8 +1391,8 @@ TEST(QueryPlan, ExpandExistingNode) {
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
auto produce = MakeProduce(r_n.op_, output);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), expected_result_count);
};
@ -1409,7 +1417,8 @@ TEST(QueryPlan, ExpandBothCycleEdgeCase) {
auto r_ =
MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::BOTH, {}, "_", false, GraphView::OLD);
EXPECT_EQ(1, PullAll(r_.op_, *dba, symbol_table));
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(1, PullAll(*r_.op_, &context));
}
TEST(QueryPlan, EdgeFilter) {
@ -1458,8 +1467,9 @@ TEST(QueryPlan, EdgeFilter) {
auto r_m = MakeExpand(storage, symbol_table, n.op_, n.sym_, "r",
EdgeAtom::Direction::OUT, {edge_type}, "m", false,
GraphView::OLD);
r_m.edge_->edge_types_.push_back(edge_type);
r_m.edge_->properties_[prop] = LITERAL(42);
r_m.edge_->edge_types_.push_back(
storage.GetEdgeTypeIx(dba.EdgeTypeName(edge_type)));
r_m.edge_->properties_[storage.GetPropertyIx(prop.first)] = LITERAL(42);
auto *filter_expr =
EQ(PROPERTY_LOOKUP(r_m.edge_->identifier_, prop), LITERAL(42));
auto edge_filter = std::make_shared<Filter>(r_m.op_, filter_expr);
@ -1470,8 +1480,8 @@ TEST(QueryPlan, EdgeFilter) {
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
auto produce = MakeProduce(edge_filter, output);
return PullAll(produce, dba, symbol_table);
Context context = MakeContext(storage, symbol_table, &dba);
return PullAll(*produce, &context);
};
EXPECT_EQ(1, test_filter());
@ -1512,8 +1522,8 @@ TEST(QueryPlan, EdgeFilterMultipleTypes) {
// fill up the symbol table
symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
symbol_table[*output->expression_] = r_m.node_sym_;
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 2);
}
@ -1533,8 +1543,7 @@ TEST(QueryPlan, Filter) {
SymbolTable symbol_table;
auto n = MakeScanAll(storage, symbol_table, "n");
auto e =
storage.Create<PropertyLookup>(storage.Create<Identifier>("n"), property);
auto e = PROPERTY_LOOKUP("n", property);
symbol_table[*e->expression_] = n.sym_;
auto f = std::make_shared<Filter>(n.op_, e);
@ -1543,8 +1552,8 @@ TEST(QueryPlan, Filter) {
symbol_table[*output->expression_] = n.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("named_expression_1", true);
auto produce = MakeProduce(f, output);
EXPECT_EQ(CollectProduce(produce.get(), symbol_table, dba).size(), 2);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(CollectProduce(*produce, &context).size(), 2);
}
TEST(QueryPlan, EdgeUniquenessFilter) {
@ -1575,7 +1584,8 @@ TEST(QueryPlan, EdgeUniquenessFilter) {
if (edge_uniqueness)
last_op = std::make_shared<EdgeUniquenessFilter>(
last_op, r2_n3.edge_sym_, std::vector<Symbol>{r1_n2.edge_sym_});
return PullAll(last_op, *dba, symbol_table);
Context context = MakeContext(storage, symbol_table, dba.get());
return PullAll(*last_op, &context);
};
EXPECT_EQ(2, check_expand_results(false));
@ -1607,8 +1617,8 @@ TEST(QueryPlan, Distinct) {
auto x_ne = NEXPR("x", x_expr);
symbol_table[*x_ne] = symbol_table.CreateSymbol("x_ne", true);
auto produce = MakeProduce(distinct, x_ne);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(output.size(), results.size());
auto output_it = output.begin();
for (const auto &row : results) {
@ -1647,7 +1657,8 @@ TEST(QueryPlan, ScanAllByLabel) {
auto produce = MakeProduce(scan_all_by_label.op_, output);
symbol_table[*output->expression_] = scan_all_by_label.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("n", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(results.size(), 1);
auto result_row = results[0];
ASSERT_EQ(result_row.size(), 1);
@ -1685,14 +1696,15 @@ TEST(QueryPlan, ScanAllByLabelProperty) {
AstStorage storage;
SymbolTable symbol_table;
auto scan_all = MakeScanAllByLabelPropertyRange(
storage, symbol_table, "n", label, prop,
storage, symbol_table, "n", label, prop, "prop",
Bound{LITERAL(lower), lower_type}, Bound{LITERAL(upper), upper_type});
// RETURN n
auto output = NEXPR("n", IDENT("n"));
auto produce = MakeProduce(scan_all.op_, output);
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("n", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(results.size(), expected.size());
for (size_t i = 0; i < expected.size(); i++) {
TypedValue equal =
@ -1746,14 +1758,15 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualityNoError) {
// MATCH (n :label {prop: 42})
AstStorage storage;
SymbolTable symbol_table;
auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n",
label, prop, LITERAL(42));
auto scan_all = MakeScanAllByLabelPropertyValue(
storage, symbol_table, "n", label, prop, "prop", LITERAL(42));
// RETURN n
auto output = NEXPR("n", IDENT("n"));
auto produce = MakeProduce(scan_all.op_, output);
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("n", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
ASSERT_EQ(results.size(), 1);
const auto &row = results[0];
ASSERT_EQ(row.size(), 1);
@ -1786,9 +1799,9 @@ TEST(QueryPlan, ScanAllByLabelPropertyValueError) {
auto *ident_m = IDENT("m");
symbol_table[*ident_m] = scan_all.sym_;
auto scan_index = MakeScanAllByLabelPropertyValue(
storage, symbol_table, "n", label, prop, ident_m, scan_all.op_);
EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
QueryRuntimeException);
storage, symbol_table, "n", label, prop, "prop", ident_m, scan_all.op_);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
}
TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
@ -1816,28 +1829,29 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeError) {
{
// Lower bound isn't property value
auto scan_index = MakeScanAllByLabelPropertyRange(
storage, symbol_table, "n", label, prop,
storage, symbol_table, "n", label, prop, "prop",
Bound{ident_m, Bound::Type::INCLUSIVE}, std::experimental::nullopt,
scan_all.op_);
EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
}
{
// Upper bound isn't property value
auto scan_index = MakeScanAllByLabelPropertyRange(
storage, symbol_table, "n", label, prop, std::experimental::nullopt,
Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
QueryRuntimeException);
storage, symbol_table, "n", label, prop, "prop",
std::experimental::nullopt, Bound{ident_m, Bound::Type::INCLUSIVE},
scan_all.op_);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
}
{
// Both bounds aren't property value
auto scan_index = MakeScanAllByLabelPropertyRange(
storage, symbol_table, "n", label, prop,
storage, symbol_table, "n", label, prop, "prop",
Bound{ident_m, Bound::Type::INCLUSIVE},
Bound{ident_m, Bound::Type::INCLUSIVE}, scan_all.op_);
EXPECT_THROW(PullAll(scan_index.op_, *dba, symbol_table),
QueryRuntimeException);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_THROW(PullAll(*scan_index.op_, &context), QueryRuntimeException);
}
}
@ -1863,14 +1877,16 @@ TEST(QueryPlan, ScanAllByLabelPropertyEqualNull) {
// MATCH (n :label {prop: 42})
AstStorage storage;
SymbolTable symbol_table;
auto scan_all = MakeScanAllByLabelPropertyValue(
storage, symbol_table, "n", label, prop, LITERAL(TypedValue::Null));
auto scan_all =
MakeScanAllByLabelPropertyValue(storage, symbol_table, "n", label, prop,
"prop", LITERAL(TypedValue::Null));
// RETURN n
auto output = NEXPR("n", IDENT("n"));
auto produce = MakeProduce(scan_all.op_, output);
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("n", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 0);
}
@ -1897,7 +1913,7 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
AstStorage storage;
SymbolTable symbol_table;
auto scan_all = MakeScanAllByLabelPropertyRange(
storage, symbol_table, "n", label, prop,
storage, symbol_table, "n", label, prop, "prop",
Bound{LITERAL(TypedValue::Null), Bound::Type::INCLUSIVE},
Bound{LITERAL(TypedValue::Null), Bound::Type::EXCLUSIVE});
// RETURN n
@ -1905,7 +1921,8 @@ TEST(QueryPlan, ScanAllByLabelPropertyRangeNull) {
auto produce = MakeProduce(scan_all.op_, output);
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] = symbol_table.CreateSymbol("n", true);
auto results = CollectProduce(produce.get(), symbol_table, *dba);
Context context = MakeContext(storage, symbol_table, dba.get());
auto results = CollectProduce(*produce, &context);
EXPECT_EQ(results.size(), 0);
}
@ -1935,10 +1952,11 @@ TEST(QueryPlan, ScanAllByLabelPropertyNoValueInIndexContinuation) {
symbol_table[*x_expr] = x;
// MATCH (n :label {prop: x})
auto scan_all = MakeScanAllByLabelPropertyValue(storage, symbol_table, "n",
label, prop, x_expr, unwind);
auto scan_all = MakeScanAllByLabelPropertyValue(
storage, symbol_table, "n", label, prop, "prop", x_expr, unwind);
EXPECT_EQ(PullAll(scan_all.op_, *dba, symbol_table), 1);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(PullAll(*scan_all.op_, &context), 1);
}
TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
@ -1972,13 +1990,14 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
SymbolTable symbol_table;
auto dba = db.Access();
auto scan_all_by_label_property_value = MakeScanAllByLabelPropertyValue(
storage, symbol_table, "n", label, prop, LITERAL(prop_value));
storage, symbol_table, "n", label, prop, "prop", LITERAL(prop_value));
auto output = NEXPR("n", IDENT("n"));
auto produce = MakeProduce(scan_all_by_label_property_value.op_, output);
symbol_table[*output->expression_] = scan_all_by_label_property_value.sym_;
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
EXPECT_EQ(PullAll(produce, *dba, symbol_table), prop_count);
Context context = MakeContext(storage, symbol_table, dba.get());
EXPECT_EQ(PullAll(*produce, &context), prop_count);
};
// Make sure there are `vertex_count` results when using scan all
@ -1988,8 +2007,7 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
auto dba_ptr = db.Access();
auto &dba = *dba_ptr;
auto scan_all = MakeScanAll(storage, symbol_table, "n");
auto e = storage.Create<PropertyLookup>(storage.Create<Identifier>("n"),
std::make_pair("prop", prop));
auto e = PROPERTY_LOOKUP("n", std::make_pair("prop", prop));
symbol_table[*e->expression_] = scan_all.sym_;
auto filter =
std::make_shared<Filter>(scan_all.op_, EQ(e, LITERAL(prop_value)));
@ -1998,7 +2016,8 @@ TEST(QueryPlan, ScanAllEqualsScanAllByLabelProperty) {
symbol_table[*output->expression_] = scan_all.sym_;
symbol_table[*output] =
symbol_table.CreateSymbol("named_expression_1", true);
EXPECT_EQ(PullAll(produce, dba, symbol_table), prop_count);
Context context = MakeContext(storage, symbol_table, &dba);
EXPECT_EQ(PullAll(*produce, &context), prop_count);
};
count_with_index(prop_value1, vertex_prop_count);

View File

@ -28,112 +28,120 @@ struct ExpressionPrettyPrinterTest : public ::testing::Test {
TEST_F(ExpressionPrettyPrinterTest, Literals) {
// 1
EXPECT_EQ(ToString(LITERAL(1)), "1");
EXPECT_EQ(ToString(storage, LITERAL(1)), "1");
// "hello"
EXPECT_EQ(ToString(LITERAL("hello")), "\"hello\"");
EXPECT_EQ(ToString(storage, LITERAL("hello")), "\"hello\"");
// null
EXPECT_EQ(ToString(LITERAL(TypedValue::Null)), "null");
EXPECT_EQ(ToString(storage, LITERAL(TypedValue::Null)), "null");
// true
EXPECT_EQ(ToString(LITERAL(true)), "true");
EXPECT_EQ(ToString(storage, LITERAL(true)), "true");
// false
EXPECT_EQ(ToString(LITERAL(false)), "false");
EXPECT_EQ(ToString(storage, LITERAL(false)), "false");
// [1 null "hello"]
EXPECT_EQ(ToString(LITERAL(
(std::vector<PropertyValue>{1, PropertyValue::Null, "hello"}))),
EXPECT_EQ(ToString(storage, LITERAL((std::vector<PropertyValue>{
1, PropertyValue::Null, "hello"}))),
"[1, null, \"hello\"]");
// {hello: 1, there: 2}
EXPECT_EQ(ToString(LITERAL((std::map<std::string, PropertyValue>{
{"hello", 1}, {"there", 2}}))),
EXPECT_EQ(ToString(storage, LITERAL((std::map<std::string, PropertyValue>{
{"hello", 1}, {"there", 2}}))),
"{\"hello\": 1, \"there\": 2}");
}
TEST_F(ExpressionPrettyPrinterTest, UnaryOperators) {
// not(false)
EXPECT_EQ(ToString(NOT(LITERAL(false))), "(Not false)");
EXPECT_EQ(ToString(storage, NOT(LITERAL(false))), "(Not false)");
// +1
EXPECT_EQ(ToString(UPLUS(LITERAL(1))), "(+ 1)");
EXPECT_EQ(ToString(storage, UPLUS(LITERAL(1))), "(+ 1)");
// -1
EXPECT_EQ(ToString(UMINUS(LITERAL(1))), "(- 1)");
EXPECT_EQ(ToString(storage, UMINUS(LITERAL(1))), "(- 1)");
// null IS NULL
EXPECT_EQ(ToString(IS_NULL(LITERAL(TypedValue::Null))), "(IsNull null)");
EXPECT_EQ(ToString(storage, IS_NULL(LITERAL(TypedValue::Null))),
"(IsNull null)");
}
TEST_F(ExpressionPrettyPrinterTest, BinaryOperators) {
// and(null, 5)
EXPECT_EQ(ToString(AND(LITERAL(TypedValue::Null), LITERAL(5))),
EXPECT_EQ(ToString(storage, AND(LITERAL(TypedValue::Null), LITERAL(5))),
"(And null 5)");
// or(5, {hello: "there"}["hello"])
EXPECT_EQ(
ToString(OR(LITERAL(5),
PROPERTY_LOOKUP(MAP(std::make_pair(PROPERTY_PAIR("hello"),
LITERAL("there"))),
PROPERTY_PAIR("hello")))),
"(Or 5 (PropertyLookup {\"hello\": \"there\"} \"hello\"))");
EXPECT_EQ(ToString(storage,
OR(LITERAL(5),
PROPERTY_LOOKUP(
MAP(std::make_pair(storage.GetPropertyIx("hello"),
LITERAL("there"))),
"hello"))),
"(Or 5 (PropertyLookup {\"hello\": \"there\"} \"hello\"))");
// and(coalesce(null, 1), {hello: "there"})
EXPECT_EQ(ToString(AND(
COALESCE(LITERAL(TypedValue::Null), LITERAL(1)),
MAP(std::make_pair(PROPERTY_PAIR("hello"), LITERAL("there"))))),
"(And (Coalesce [null, 1]) {\"hello\": \"there\"})");
EXPECT_EQ(
ToString(storage, AND(COALESCE(LITERAL(TypedValue::Null), LITERAL(1)),
MAP(std::make_pair(storage.GetPropertyIx("hello"),
LITERAL("there"))))),
"(And (Coalesce [null, 1]) {\"hello\": \"there\"})");
}
TEST_F(ExpressionPrettyPrinterTest, Coalesce) {
// coalesce()
EXPECT_EQ(ToString(COALESCE()), "(Coalesce [])");
EXPECT_EQ(ToString(storage, COALESCE()), "(Coalesce [])");
// coalesce(null, null)
EXPECT_EQ(
ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(TypedValue::Null))),
"(Coalesce [null, null])");
EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null),
LITERAL(TypedValue::Null))),
"(Coalesce [null, null])");
// coalesce(null, 2, 3)
EXPECT_EQ(
ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(2), LITERAL(3))),
"(Coalesce [null, 2, 3])");
EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
LITERAL(3))),
"(Coalesce [null, 2, 3])");
// coalesce(null, 2, assert(false), 3)
EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
FN("ASSERT", LITERAL(false)), LITERAL(3))),
"(Coalesce [null, 2, (Function \"ASSERT\" [false]), 3])");
EXPECT_EQ(
ToString(storage, COALESCE(LITERAL(TypedValue::Null), LITERAL(2),
FN("ASSERT", LITERAL(false)), LITERAL(3))),
"(Coalesce [null, 2, (Function \"ASSERT\" [false]), 3])");
// coalesce(null, assert(false))
EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue::Null),
FN("ASSERT", LITERAL(false)))),
EXPECT_EQ(ToString(storage, COALESCE(LITERAL(TypedValue::Null),
FN("ASSERT", LITERAL(false)))),
"(Coalesce [null, (Function \"ASSERT\" [false])])");
// coalesce([null, null])
EXPECT_EQ(ToString(COALESCE(LITERAL(TypedValue(
std::vector<TypedValue>{TypedValue::Null, TypedValue::Null})))),
"(Coalesce [[null, null]])");
EXPECT_EQ(
ToString(storage, COALESCE(LITERAL(TypedValue(std::vector<TypedValue>{
TypedValue::Null, TypedValue::Null})))),
"(Coalesce [[null, null]])");
}
TEST_F(ExpressionPrettyPrinterTest, ParameterLookup) {
// and($hello, $there)
EXPECT_EQ(ToString(AND(PARAMETER_LOOKUP(1), PARAMETER_LOOKUP(2))),
EXPECT_EQ(ToString(storage, AND(PARAMETER_LOOKUP(1), PARAMETER_LOOKUP(2))),
"(And (ParameterLookup 1) (ParameterLookup 2))");
}
TEST_F(ExpressionPrettyPrinterTest, PropertyLookup) {
// {hello: "there"}["hello"]
EXPECT_EQ(ToString(PROPERTY_LOOKUP(
MAP(std::make_pair(PROPERTY_PAIR("hello"), LITERAL("there"))),
PROPERTY_PAIR("hello"))),
"(PropertyLookup {\"hello\": \"there\"} \"hello\")");
EXPECT_EQ(
ToString(storage, PROPERTY_LOOKUP(
MAP(std::make_pair(storage.GetPropertyIx("hello"),
LITERAL("there"))),
"hello")),
"(PropertyLookup {\"hello\": \"there\"} \"hello\")");
}
TEST_F(ExpressionPrettyPrinterTest, NamedExpression) {
// n AS 1
EXPECT_EQ(ToString(NEXPR("n", LITERAL(1))), "(NamedExpression \"n\" 1)");
EXPECT_EQ(ToString(storage, NEXPR("n", LITERAL(1))),
"(NamedExpression \"n\" 1)");
}
} // namespace

View File

@ -10,10 +10,10 @@ using namespace query;
class FakeDbAccessor {};
storage::EdgeType EDGE_TYPE(0);
storage::Label LABEL_0(0);
storage::Label LABEL_1(1);
storage::Property PROP_0(0);
const std::string EDGE_TYPE = "0";
const std::string LABEL_0 = "label0";
const std::string LABEL_1 = "label1";
const std::string PROP_0 = "prop0";
using ::testing::UnorderedElementsAre;
@ -65,7 +65,8 @@ TEST_F(TestPrivilegeExtractor, MatchNodeSetLabels) {
TEST_F(TestPrivilegeExtractor, MatchNodeSetProperty) {
auto *query = QUERY(
SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
SET(PROPERTY_LOOKUP("n", {"prop", PROP_0}), LITERAL(42))));
SET(PROPERTY_LOOKUP(storage.Create<Identifier>("n"), PROP_0),
LITERAL(42))));
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::MATCH,
AuthQuery::Privilege::SET));
@ -88,16 +89,17 @@ TEST_F(TestPrivilegeExtractor, MatchNodeRemoveLabels) {
}
TEST_F(TestPrivilegeExtractor, MatchNodeRemoveProperty) {
auto *query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(NODE("n"))),
REMOVE(PROPERTY_LOOKUP("n", {"prop", PROP_0}))));
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))),
REMOVE(PROPERTY_LOOKUP(storage.Create<Identifier>("n"), PROP_0))));
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::MATCH,
AuthQuery::Privilege::REMOVE));
}
TEST_F(TestPrivilegeExtractor, CreateIndex) {
auto *query = CREATE_INDEX_ON(LABEL_0, PROP_0);
auto *query = CREATE_INDEX_ON(storage.GetLabelIx(LABEL_0),
storage.GetPropertyIx(PROP_0));
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::INDEX));
}

View File

@ -79,7 +79,7 @@ TEST_F(TestSymbolGenerator, MatchNodeUnboundReturn) {
TEST_F(TestSymbolGenerator, CreatePropertyUnbound) {
// AST with unbound variable in create: CREATE ({prop: x})
auto node = NODE("anon");
node->properties_[PROPERTY_PAIR("prop")] = IDENT("x");
node->properties_[storage.GetPropertyIx("prop")] = IDENT("x");
auto query_ast = QUERY(SINGLE_QUERY(CREATE(PATTERN(node))));
EXPECT_THROW(query::MakeSymbolTable(query_ast), UnboundVariableError);
}
@ -130,7 +130,7 @@ TEST_F(TestSymbolGenerator, MatchCreateRedeclareNode) {
TEST_F(TestSymbolGenerator, MatchCreateRedeclareEdge) {
// AST with redeclaring a match edge variable in create:
// MATCH (n) -[r]- (m) CREATE (n) -[r :relationship]-> (l)
auto relationship = dba.EdgeType("relationship");
auto relationship = "relationship";
auto query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
CREATE(PATTERN(NODE("n"),
@ -160,10 +160,10 @@ TEST_F(TestSymbolGenerator, MatchCreateTypeMismatch) {
TEST_F(TestSymbolGenerator, CreateMultipleEdgeType) {
// Multiple edge relationship are not allowed when creating edges.
// CREATE (n) -[r :rel1 | :rel2]-> (m)
auto rel1 = dba.EdgeType("rel1");
auto rel2 = dba.EdgeType("rel2");
auto rel1 = "rel1";
auto rel2 = "rel2";
auto edge = EDGE("r", EdgeAtom::Direction::OUT, {rel1});
edge->edge_types_.emplace_back(rel2);
edge->edge_types_.emplace_back(storage.GetEdgeTypeIx(rel2));
auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(NODE("n"), edge, NODE("m")))));
EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
}
@ -171,7 +171,7 @@ TEST_F(TestSymbolGenerator, CreateMultipleEdgeType) {
TEST_F(TestSymbolGenerator, CreateBidirectionalEdge) {
// Bidirectional relationships are not allowed when creating edges.
// CREATE (n) -[r :rel1]- (m)
auto rel1 = dba.EdgeType("rel1");
auto rel1 = "rel1";
auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
NODE("n"), EDGE("r", EdgeAtom::Direction::BOTH, {rel1}), NODE("m")))));
EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
@ -270,8 +270,8 @@ TEST_F(TestSymbolGenerator, MatchWithWhereUnbound) {
TEST_F(TestSymbolGenerator, CreateMultiExpand) {
// Test CREATE (n) -[r :r]-> (m), (n) - [p :p]-> (l)
auto r_type = dba.EdgeType("r");
auto p_type = dba.EdgeType("p");
auto r_type = "r";
auto p_type = "p";
auto node_n1 = NODE("n");
auto edge_r = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
auto node_m = NODE("m");
@ -303,8 +303,8 @@ TEST_F(TestSymbolGenerator, CreateMultiExpand) {
TEST_F(TestSymbolGenerator, MatchCreateExpandLabel) {
// Test MATCH (n) CREATE (m) -[r :r]-> (n:label)
auto r_type = dba.EdgeType("r");
auto label = dba.Label("label");
auto r_type = "r";
auto label = "label";
auto query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))),
CREATE(PATTERN(NODE("m"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}),
@ -314,9 +314,9 @@ TEST_F(TestSymbolGenerator, MatchCreateExpandLabel) {
TEST_F(TestSymbolGenerator, CreateExpandProperty) {
// Test CREATE (n) -[r :r]-> (n {prop: 42})
auto r_type = dba.EdgeType("r");
auto r_type = "r";
auto n_prop = NODE("n");
n_prop->properties_[PROPERTY_PAIR("prop")] = LITERAL(42);
n_prop->properties_[storage.GetPropertyIx("prop")] = LITERAL(42);
auto query = QUERY(SINGLE_QUERY(CREATE(PATTERN(
NODE("n"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}), n_prop))));
EXPECT_THROW(query::MakeSymbolTable(query), SemanticException);
@ -365,7 +365,7 @@ TEST_F(TestSymbolGenerator, MatchPropCreateNodeProp) {
auto node_n = NODE("n");
auto node_m = NODE("m");
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
node_m->properties_[prop] = n_prop;
node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
auto query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), CREATE(PATTERN(node_m))));
auto symbol_table = query::MakeSymbolTable(query);
@ -379,7 +379,7 @@ TEST_F(TestSymbolGenerator, MatchPropCreateNodeProp) {
TEST_F(TestSymbolGenerator, CreateNodeEdge) {
// Test CREATE (n), (n) -[r :r]-> (n)
auto r_type = dba.EdgeType("r");
auto r_type = "r";
auto node_1 = NODE("n");
auto node_2 = NODE("n");
auto edge = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
@ -397,7 +397,7 @@ TEST_F(TestSymbolGenerator, CreateNodeEdge) {
TEST_F(TestSymbolGenerator, MatchWithCreate) {
// Test MATCH (n) WITH n AS m CREATE (m) -[r :r]-> (m)
auto r_type = dba.EdgeType("r");
auto r_type = "r";
auto node_1 = NODE("n");
auto node_2 = NODE("m");
auto edge = EDGE("r", EdgeAtom::Direction::OUT, {r_type});
@ -512,7 +512,7 @@ TEST_F(TestSymbolGenerator, MergeVariableError) {
TEST_F(TestSymbolGenerator, MergeVariableErrorEdge) {
// Test MATCH (n) -[r]- (m) MERGE (a) -[r :rel]- (b)
auto rel = dba.EdgeType("rel");
auto rel = "rel";
auto query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r"), NODE("m"))),
MERGE(PATTERN(NODE("a"), EDGE("r", EdgeAtom::Direction::BOTH, {rel}),
@ -531,7 +531,7 @@ TEST_F(TestSymbolGenerator, MergeEdgeWithoutType) {
TEST_F(TestSymbolGenerator, MergeOnMatchOnCreate) {
// Test MATCH (n) MERGE (n) -[r :rel]- (m) ON MATCH SET n.prop = 42
// ON CREATE SET m.prop = 42 RETURN r AS r
auto rel = dba.EdgeType("rel");
auto rel = "rel";
auto prop = dba.Property("prop");
auto match_n = NODE("n");
auto merge_n = NODE("n");
@ -600,10 +600,10 @@ TEST_F(TestSymbolGenerator, MatchCrossReferenceVariable) {
auto prop = PROPERTY_PAIR("prop");
auto node_n = NODE("n");
auto m_prop = PROPERTY_LOOKUP("m", prop.second);
node_n->properties_[prop] = m_prop;
node_n->properties_[storage.GetPropertyIx(prop.first)] = m_prop;
auto node_m = NODE("m");
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
node_m->properties_[prop] = n_prop;
node_m->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
auto ident_n = IDENT("n");
auto as_n = AS("n");
auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n), PATTERN(node_m)),
@ -661,8 +661,8 @@ TEST_F(TestSymbolGenerator, MatchReturnAsteriskNoUserVariables) {
TEST_F(TestSymbolGenerator, MatchMergeExpandLabel) {
// Test MATCH (n) MERGE (m) -[r :r]-> (n:label)
auto r_type = dba.EdgeType("r");
auto label = dba.Label("label");
auto r_type = "r";
auto label = "label";
auto query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"))),
MERGE(PATTERN(NODE("m"), EDGE("r", EdgeAtom::Direction::OUT, {r_type}),
@ -675,7 +675,7 @@ TEST_F(TestSymbolGenerator, MatchEdgeWithIdentifierInProperty) {
auto prop = PROPERTY_PAIR("prop");
auto edge = EDGE("r");
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
edge->properties_[prop] = n_prop;
edge->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
auto node_n = NODE("n");
auto query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n, edge, NODE("m"))), RETURN("r")));
@ -768,7 +768,7 @@ TEST_F(TestSymbolGenerator, MatchPropertySameIdentifier) {
auto prop = PROPERTY_PAIR("prop");
auto node_n = NODE("n");
auto n_prop = PROPERTY_LOOKUP("n", prop.second);
node_n->properties_[prop] = n_prop;
node_n->properties_[storage.GetPropertyIx(prop.first)] = n_prop;
auto query = QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n)), RETURN("n")));
auto symbol_table = query::MakeSymbolTable(query);
auto n = symbol_table.at(*node_n->identifier_);
@ -885,7 +885,7 @@ TEST_F(TestSymbolGenerator, MatchBfsReturn) {
auto *n_prop = PROPERTY_LOOKUP("n", prop);
auto *bfs = storage.Create<EdgeAtom>(
IDENT("r"), EdgeAtom::Type::BREADTH_FIRST, EdgeAtom::Direction::OUT,
std::vector<storage::EdgeType>{});
std::vector<EdgeTypeIx>{});
bfs->filter_lambda_.inner_edge = IDENT("r");
bfs->filter_lambda_.inner_node = IDENT("n");
bfs->filter_lambda_.expression = r_prop;
@ -991,7 +991,7 @@ TEST_F(TestSymbolGenerator, MatchWShortestReturn) {
auto *r_filter = PROPERTY_LOOKUP("r", filter);
auto *shortest = storage.Create<EdgeAtom>(
IDENT("r"), EdgeAtom::Type::WEIGHTED_SHORTEST_PATH,
EdgeAtom::Direction::OUT, std::vector<storage::EdgeType>{});
EdgeAtom::Direction::OUT, std::vector<EdgeTypeIx>{});
{
shortest->weight_lambda_.inner_edge = IDENT("r");
shortest->weight_lambda_.inner_node = IDENT("n");

View File

@ -58,21 +58,22 @@ void AssertRows(const std::vector<std::vector<TypedValue>> &datum,
void CheckPlansProduce(
size_t expected_plan_count, query::CypherQuery *query, AstStorage &storage,
database::GraphDbAccessor &dba,
database::GraphDbAccessor *dba,
std::function<void(const std::vector<std::vector<TypedValue>> &)> check) {
auto symbol_table = query::MakeSymbolTable(query);
auto planning_context =
MakePlanningContext(storage, symbol_table, query, dba);
MakePlanningContext(&storage, &symbol_table, query, dba);
auto query_parts = CollectQueryParts(symbol_table, storage, query);
EXPECT_TRUE(query_parts.query_parts.size() > 0);
auto single_query_parts = query_parts.query_parts.at(0).single_query_parts;
auto plans = MakeLogicalPlanForSingleQuery<VariableStartPlanner>(
single_query_parts, planning_context);
single_query_parts, &planning_context);
EXPECT_EQ(std::distance(plans.begin(), plans.end()), expected_plan_count);
for (const auto &plan : plans) {
auto *produce = dynamic_cast<Produce *>(plan.get());
ASSERT_TRUE(produce);
auto results = CollectProduce(produce, symbol_table, dba);
Context context = MakeContext(storage, symbol_table, dba);
auto results = CollectProduce(*produce, &context);
check(results);
}
}
@ -91,7 +92,7 @@ TEST(TestVariableStartPlanner, MatchReturn) {
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))),
RETURN("n")));
// We have 2 nodes `n` and `m` from which we could start, so expect 2 plans.
CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
// We expect to produce only a single (v1) node.
AssertRows(results, {{v1}});
});
@ -115,7 +116,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
EDGE("e", Direction::OUT), NODE("l"))),
RETURN("n")));
// We have 3 nodes: `n`, `m` and `l` from which we could start.
CheckPlansProduce(3, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(3, query, storage, dba.get(), [&](const auto &results) {
// We expect to produce only a single (v1) node.
AssertRows(results, {{v1}});
});
@ -127,7 +128,7 @@ TEST(TestVariableStartPlanner, MatchTripletPatternReturn) {
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m")),
PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))),
RETURN("n")));
CheckPlansProduce(3, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(3, query, storage, dba.get(), [&](const auto &results) {
AssertRows(results, {{v1}});
});
}
@ -151,7 +152,7 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchReturn) {
RETURN("n", "l")));
// We have 2 nodes `n` and `m` from which we could start the MATCH, and 2
// nodes for OPTIONAL MATCH. This should produce 2 * 2 plans.
CheckPlansProduce(4, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(4, query, storage, dba.get(), [&](const auto &results) {
// We expect to produce 2 rows:
// * (v1), (v3)
// * (v2), null
@ -165,7 +166,8 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
// Graph (v1) -[:r]-> (v2)
auto v1 = dba->InsertVertex();
auto v2 = dba->InsertVertex();
auto r_type = dba->EdgeType("r");
auto r_type_name = "r";
auto r_type = dba->EdgeType(r_type_name);
dba->InsertEdge(v1, v2, r_type);
dba->AdvanceCommand();
// Test MATCH (n) -[r]-> (m) OPTIONAL MATCH (m) -[e]-> (l)
@ -174,11 +176,12 @@ TEST(TestVariableStartPlanner, MatchOptionalMatchMergeReturn) {
auto *query = QUERY(SINGLE_QUERY(
MATCH(PATTERN(NODE("n"), EDGE("r", Direction::OUT), NODE("m"))),
OPTIONAL_MATCH(PATTERN(NODE("m"), EDGE("e", Direction::OUT), NODE("l"))),
MERGE(PATTERN(NODE("u"), EDGE("q", Direction::OUT, {r_type}), NODE("v"))),
MERGE(PATTERN(NODE("u"), EDGE("q", Direction::OUT, {r_type_name}),
NODE("v"))),
RETURN("n", "m", "l", "u", "v")));
// Since MATCH, OPTIONAL MATCH and MERGE each have 2 nodes from which we can
// start, we generate 2 * 2 * 2 plans.
CheckPlansProduce(8, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(8, query, storage, dba.get(), [&](const auto &results) {
// We expect to produce a single row: (v1), (v2), null, (v1), (v2)
AssertRows(results, {{v1, v2, TypedValue::Null, v1, v2}});
});
@ -201,7 +204,7 @@ TEST(TestVariableStartPlanner, MatchWithMatchReturn) {
RETURN("n", "m", "l")));
// We can start from 2 nodes in each match. Since WITH separates query parts,
// we expect to get 2 plans for each, which totals 2 * 2.
CheckPlansProduce(4, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(4, query, storage, dba.get(), [&](const auto &results) {
// We expect to produce a single row: (v1), (v1), (v2)
AssertRows(results, {{v1, v1, v2}});
});
@ -226,7 +229,7 @@ TEST(TestVariableStartPlanner, MatchVariableExpand) {
TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1]
TypedValue r2_list(std::vector<TypedValue>{r2}); // [r2]
TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2}); // [r1, r2]
CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
AssertRows(results, {{r1_list}, {r2_list}, {r1_r2_list}});
});
}
@ -255,7 +258,7 @@ TEST(TestVariableStartPlanner, MatchVariableExpandReferenceNode) {
// We expect to get a single column with the following rows:
TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1] (v1 -[*..1]-> v2)
TypedValue r2_list(std::vector<TypedValue>{r2}); // [r2] (v2 -[*..2]-> v3)
CheckPlansProduce(2, query, storage, dba, [&](const auto &results) {
CheckPlansProduce(2, query, storage, &dba, [&](const auto &results) {
AssertRows(results, {{r1_list}, {r2_list}});
});
}
@ -276,13 +279,13 @@ TEST(TestVariableStartPlanner, MatchVariableExpandBoth) {
AstStorage storage;
auto edge = EDGE_VARIABLE("r", Type::DEPTH_FIRST, Direction::BOTH);
auto node_n = NODE("n");
node_n->properties_[std::make_pair("id", id)] = LITERAL(1);
node_n->properties_[storage.GetPropertyIx("id")] = LITERAL(1);
auto *query =
QUERY(SINGLE_QUERY(MATCH(PATTERN(node_n, edge, NODE("m"))), RETURN("r")));
// We expect to get a single column with the following rows:
TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1]
TypedValue r1_r2_list(std::vector<TypedValue>{r1, r2}); // [r1, r2]
CheckPlansProduce(2, query, storage, *dba, [&](const auto &results) {
CheckPlansProduce(2, query, storage, dba.get(), [&](const auto &results) {
AssertRows(results, {{r1_list}, {r1_r2_list}});
});
}
@ -306,7 +309,7 @@ TEST(TestVariableStartPlanner, MatchBfs) {
AstStorage storage;
auto *bfs = storage.Create<query::EdgeAtom>(
IDENT("r"), EdgeAtom::Type::BREADTH_FIRST, Direction::OUT,
std::vector<storage::EdgeType>{});
std::vector<query::EdgeTypeIx>{});
bfs->filter_lambda_.inner_edge = IDENT("r");
bfs->filter_lambda_.inner_node = IDENT("n");
bfs->filter_lambda_.expression = NEQ(PROPERTY_LOOKUP("n", id), LITERAL(3));
@ -315,7 +318,7 @@ TEST(TestVariableStartPlanner, MatchBfs) {
SINGLE_QUERY(MATCH(PATTERN(NODE("n"), bfs, NODE("m"))), RETURN("r")));
// We expect to get a single column with the following rows:
TypedValue r1_list(std::vector<TypedValue>{r1}); // [r1]
CheckPlansProduce(2, query, storage, dba, [&](const auto &results) {
CheckPlansProduce(2, query, storage, &dba, [&](const auto &results) {
AssertRows(results, {{r1_list}});
});
}