Separate query types in AST and interpreter

Summary:
`Query` is now an abstract class which has `CypherQuery`,
`ExplainQuery`, `IndexQuery`, `AuthQuery` and `StreamQuery` as derived
classes. Only `CypherQuery` is forwarded to planner and the rest of the
queries are handled directly in the interpreter. This enabled us to
remove auth, explain and stream operators, clean up `Context` class and
remove coupling between `Results` class and plan cache. This should make
it easier to add similar functionality because no logical operator
boilerplate is needed. It should also be easier to separate community
and enterprise features for open source.

Remove Explain logical operator
Separate IndexQuery in AST
Handle index creation in interpreter
Remove CreateIndex operator and ast nodes
Remove plan cache reference from Results
Move auth queries out of operator tree
Remove auth from context
Fix tests, separate stream queries
Remove in_explicit_transaction and streams from context

Reviewers: teon.banek, mferencevic, msantl

Reviewed By: teon.banek, mferencevic

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1664
This commit is contained in:
Marin Tomic 2018-10-19 16:18:44 +02:00
parent 187da7f910
commit eee8b57daf
37 changed files with 1546 additions and 2948 deletions

View File

@ -5,14 +5,6 @@
#include "query/frontend/semantic/symbol_table.hpp"
#include "query/parameters.hpp"
namespace auth {
class Auth;
} // namespace auth
namespace integrations::kafka {
class Streams;
} // namespace integrations::kafka
namespace query {
struct EvaluationContext {
@ -22,8 +14,6 @@ struct EvaluationContext {
class Context {
public:
// Since we also return some information from context (is_index_created_) we
// need to be sure that we have only one Context instance per query.
Context(const Context &) = delete;
Context &operator=(const Context &) = delete;
Context(Context &&) = default;
@ -32,13 +22,8 @@ class Context {
explicit Context(database::GraphDbAccessor &db_accessor)
: db_accessor_(db_accessor) {}
database::GraphDbAccessor &db_accessor_;
bool in_explicit_transaction_ = false;
bool is_index_created_ = false;
SymbolTable symbol_table_;
EvaluationContext evaluation_context_;
auth::Auth *auth_ = nullptr;
integrations::kafka::Streams *kafka_streams_ = nullptr;
};
struct ParsingContext {

View File

@ -2,7 +2,9 @@
#include "database/distributed/distributed_graph_db.hpp"
#include "distributed/plan_dispatcher.hpp"
#include "query/frontend/semantic/symbol_generator.hpp"
#include "query/plan/distributed.hpp"
#include "query/plan/distributed_pretty_print.hpp"
#include "query/plan/planner.hpp"
#include "query/plan/rule_based_planner.hpp"
#include "query/plan/vertex_count_cache.hpp"
@ -57,21 +59,33 @@ DistributedInterpreter::DistributedInterpreter(database::Master *db)
: plan_dispatcher_(&db->plan_dispatcher()) {}
std::unique_ptr<LogicalPlan> DistributedInterpreter::MakeLogicalPlan(
Query *query, AstStorage ast_storage, Context *context) {
auto vertex_counts = plan::MakeVertexCountCache(context->db_accessor_);
auto planning_context = plan::MakePlanningContext(
ast_storage, context->symbol_table_, query, vertex_counts);
CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
SymbolTable symbol_table;
SymbolGenerator symbol_generator(symbol_table);
query->Accept(symbol_generator);
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, context->evaluation_context_.parameters,
FLAGS_query_cost_planner);
auto plan = MakeDistributedPlan(*tmp_logical_plan, context->symbol_table_,
next_plan_id_);
planning_context, parameters, FLAGS_query_cost_planner);
auto plan =
MakeDistributedPlan(*tmp_logical_plan, symbol_table, next_plan_id_);
VLOG(10) << "[Interpreter] Created plan for distributed execution "
<< next_plan_id_ - 1;
return std::make_unique<DistributedLogicalPlan>(std::move(plan), cost,
plan_dispatcher_);
}
void DistributedInterpreter::PrettyPrintPlan(
const database::GraphDbAccessor &dba,
const plan::LogicalOperator *plan_root, std::ostream *out) {
plan::DistributedPrettyPrint(dba, plan_root, out);
}
} // namespace query

View File

@ -17,8 +17,12 @@ class DistributedInterpreter final : public Interpreter {
DistributedInterpreter(database::Master *db);
private:
std::unique_ptr<LogicalPlan> MakeLogicalPlan(Query *, AstStorage,
Context *) override;
std::unique_ptr<LogicalPlan> MakeLogicalPlan(
CypherQuery *, AstStorage, const Parameters &,
database::GraphDbAccessor *) override;
void PrettyPrintPlan(const database::GraphDbAccessor &,
const plan::LogicalOperator *, std::ostream *) override;
std::atomic<int64_t> next_plan_id_{0};
distributed::PlanDispatcher *plan_dispatcher_{nullptr};

View File

@ -1460,9 +1460,26 @@ cpp<#
(:serialize :capnp))
(lcp:define-class query (tree)
((explain :bool :initval "false" :scope :public
:documentation "True if this is an EXPLAIN query.")
(single-query "SingleQuery *" :initval "nullptr" :scope :public
()
(:abstractp t)
(:public
#>cpp
Query() = default;
virtual Query *Clone(AstStorage &storage) const = 0;
cpp<#)
(:protected
#>cpp
explicit Query(int uid) : Tree(uid) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class cypher-query (query)
((single-query "SingleQuery *" :initval "nullptr" :scope :public
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "SingleQuery *")
@ -1475,7 +1492,7 @@ cpp<#
:documentation "Contains remaining queries that should form and union with `single_query_`."))
(:public
#>cpp
Query() = default;
CypherQuery() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
bool Accept(HierarchicalTreeVisitor &visitor) override {
@ -1490,20 +1507,18 @@ cpp<#
return visitor.PostVisit(*this);
}
// Creates deep copy of whole ast.
Query *Clone(AstStorage &storage) const override {
auto *query = storage.Create<Query>();
query->single_query_ = single_query_->Clone(storage);
CypherQuery *Clone(AstStorage &storage) const override {
auto *cypher_query = storage.Create<CypherQuery>();
cypher_query->single_query_ = single_query_->Clone(storage);
for (auto *cypher_union : cypher_unions_) {
query->cypher_unions_.push_back(cypher_union->Clone(storage));
cypher_query->cypher_unions_.push_back(cypher_union->Clone(storage));
}
query->explain_ = explain_;
return query;
return cypher_query;
}
cpp<#)
(:protected
#>cpp
explicit Query(int uid) : Tree(uid) {}
explicit CypherQuery(int uid) : Query(uid) {}
cpp<#)
(:private
#>cpp
@ -1511,6 +1526,80 @@ cpp<#
cpp<#)
(:serialize :capnp))
(lcp:define-class explain-query (query)
((cypher-query "CypherQuery *" :initval "nullptr" :scope :public
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "CypherQuery *")
:documentation "CypherQuery to be explained."))
(:public
#>cpp
ExplainQuery() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
bool Accept(HierarchicalTreeVisitor &visitor) override {
if (visitor.PreVisit(*this)) {
cypher_query_->Accept(visitor);
}
return visitor.PostVisit(*this);
}
ExplainQuery *Clone(AstStorage &storage) const override {
auto *explain_query = storage.Create<ExplainQuery>();
explain_query->cypher_query_ = cypher_query_->Clone(storage);
return explain_query;
}
cpp<#)
(:protected
#>cpp
explicit ExplainQuery(int uid) : Query(uid) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class index-query (query)
((action "Action" :scope :public
:capnp-save (lcp:capnp-save-enum "capnp::IndexQuery::Action"
"IndexQuery::Action"
'(create create-unique))
:capnp-load (lcp:capnp-load-enum "capnp::IndexQuery::Action"
"IndexQuery::Action"
'(create create-unique)))
(label "storage::Label" :scope :public)
(properties "std::vector<storage::Property>" :scope :public
:capnp-save (lcp:capnp-save-vector "storage::capnp::Common" "storage::Property")
:capnp-load (lcp:capnp-load-vector "storage::capnp::Common" "storage::Property")))
(:public
(lcp:define-enum action (create create-unique)
(:serialize :capnp))
#>cpp
IndexQuery() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
IndexQuery *Clone(AstStorage &storage) const override {
return storage.Create<IndexQuery>(action_, label_, properties_);
}
cpp<#)
(:protected
#>cpp
explicit IndexQuery(int uid) : Query(uid) {}
IndexQuery(int uid, Action action, storage::Label label,
std::vector<storage::Property> properties)
: Query(uid), action_(action), label_(label), properties_(properties) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class create (clause)
((patterns "std::vector<Pattern *>"
:scope :public
@ -2119,63 +2208,22 @@ cpp<#
cpp<#)
(:serialize :capnp))
(lcp:define-class create-index (clause)
((label "storage::Label" :scope :public)
(property "storage::Property" :scope :public))
(:public
#>cpp
CreateIndex() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
CreateIndex *Clone(AstStorage &storage) const override {
return storage.Create<CreateIndex>(label_, property_);
}
cpp<#)
(:protected
#>cpp
explicit CreateIndex(int uid) : Clause(uid) {}
CreateIndex(int uid, storage::Label label, storage::Property property)
: Clause(uid), label_(label), property_(property) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class create-unique-index (clause)
((label "storage::Label" :scope :public)
(properties "std::vector<storage::Property>" :scope :public
:capnp-save (lcp:capnp-save-vector "storage::capnp::Common" "storage::Property")
:capnp-load (lcp:capnp-load-vector "storage::capnp::Common" "storage::Property")))
(:public
#>cpp
CreateUniqueIndex() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
CreateUniqueIndex *Clone(AstStorage &storage) const override {
return storage.Create<CreateUniqueIndex>(label_, properties_);
}
cpp<#)
(:protected
#>cpp
explicit CreateUniqueIndex(int uid) : Clause(uid) {}
CreateUniqueIndex(int uid, storage::Label label,
const std::vector<storage::Property> &properties)
: Clause(uid), label_(label), properties_(properties) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class auth-query (clause)
((action "Action" :scope :public)
(lcp:define-class auth-query (query)
((action "Action" :scope :public
:capnp-save (lcp:capnp-save-enum "capnp::AuthQuery::Action"
"AuthQuery::Action"
'(create-role drop-role show-roles create-user
set-password drop-user show-users set-role
clear-role grant-privilege deny-privilege
revoke-privilege show-privileges
show-role-for-user show-users-for-role))
:capnp-load (lcp:capnp-load-enum "capnp::AuthQuery::Action"
"AuthQuery::Action"
'(create-role drop-role show-roles create-user
set-password drop-user show-users set-role
clear-role grant-privilege deny-privilege
revoke-privilege show-privileges
show-role-for-user show-users-for-role)))
(user "std::string" :scope :public)
(role "std::string" :scope :public)
(user-or-role "std::string" :scope :public)
@ -2280,12 +2328,12 @@ cpp<#
cpp<#)
(:protected
#>cpp
explicit AuthQuery(int uid) : Clause(uid) {}
explicit AuthQuery(int uid) : Query(uid) {}
explicit AuthQuery(int uid, Action action, std::string user, std::string role,
std::string user_or_role, Expression *password,
std::vector<Privilege> privileges)
: Clause(uid),
: Query(uid),
action_(action),
user_(user),
role_(role),
@ -2309,139 +2357,84 @@ const std::vector<AuthQuery::Privilege> kPrivilegesAll = {
AuthQuery::Privilege::STREAM};
cpp<#
(lcp:define-class create-stream (clause)
((stream-name "std::string" :scope :public)
(stream-uri "Expression *" :scope :public
(lcp:define-class stream-query (query)
((action "Action" :scope :public
:capnp-save (lcp:capnp-save-enum "capnp::StreamQuery::Action"
"StreamQuery::Action"
'(create-stream drop-stream show-streams
start-stream stop-stream test-stream
start-all-streams stop-all-streams))
:capnp-load (lcp:capnp-load-enum "capnp::StreamQuery::Action"
"StreamQuery::Action"
'(create-stream drop-stream show-streams
start-stream stop-stream test-stream
start-all-streams stop-all-streams)))
(stream-name "std::string" :scope :public)
(stream-uri "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(stream-topic "Expression *" :scope :public
(stream-topic "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(transform-uri "Expression *" :scope :public
(transform-uri "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(batch-interval-in-ms "Expression *" :scope :public
(batch-interval-in-ms "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(batch-size "Expression *" :scope :public
(batch-size "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
:capnp-load (load-ast-pointer "Expression *"))
(limit-batches "Expression *" :scope :public :initval "nullptr"
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
(:public
(lcp:define-enum action (create-stream drop-stream show-streams start-stream
stop-stream start-all-streams stop-all-streams test-stream)
(:serialize :capnp))
#>cpp
CreateStream() = default;
StreamQuery() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
CreateStream *Clone(AstStorage &storage) const override {
return storage.Create<CreateStream>(
stream_name_, stream_uri_->Clone(storage),
stream_topic_->Clone(storage), transform_uri_->Clone(storage),
batch_interval_in_ms_ ? batch_interval_in_ms_->Clone(storage) : nullptr,
batch_size_ ? batch_size_->Clone(storage) : nullptr);
StreamQuery *Clone(AstStorage &storage) const override {
auto *stream_uri = stream_uri_ ? stream_uri_->Clone(storage) : nullptr;
auto *stream_topic = stream_topic_ ? stream_topic_->Clone(storage) : nullptr;
auto *transform_uri =
transform_uri_ ? transform_uri_->Clone(storage) : nullptr;
auto *batch_interval_in_ms =
batch_interval_in_ms_ ? batch_interval_in_ms_->Clone(storage) : nullptr;
auto *batch_size = batch_size_ ? batch_size_->Clone(storage) : nullptr;
auto *limit_batches =
limit_batches_ ? limit_batches_->Clone(storage) : nullptr;
return storage.Create<StreamQuery>(
action_, stream_name_, stream_uri, stream_topic, transform_uri,
batch_interval_in_ms, batch_size, limit_batches);
}
cpp<#)
(:protected
#>cpp
explicit CreateStream(int uid) : Clause(uid) {}
CreateStream(int uid, std::string stream_name, Expression *stream_uri,
Expression *stream_topic, Expression *transform_uri,
Expression *batch_interval_in_ms, Expression *batch_size)
: Clause(uid),
StreamQuery(int uid) : Query(uid) {}
StreamQuery(int uid, Action action, std::string stream_name,
Expression *stream_uri, Expression *stream_topic,
Expression *transform_uri, Expression *batch_interval_in_ms,
Expression *batch_size, Expression *limit_batches)
: Query(uid),
action_(action),
stream_name_(std::move(stream_name)),
stream_uri_(stream_uri),
stream_topic_(stream_topic),
transform_uri_(transform_uri),
batch_interval_in_ms_(batch_interval_in_ms),
batch_size_(batch_size) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class drop-stream (clause)
((stream-name "std::string" :scope :public))
(:public
#>cpp
DropStream() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
DropStream *Clone(AstStorage &storage) const override {
return storage.Create<DropStream>(stream_name_);
}
cpp<#)
(:protected
#>cpp
explicit DropStream(int uid) : Clause(uid) {}
DropStream(int uid, std::string stream_name)
: Clause(uid), stream_name_(std::move(stream_name)) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class show-streams (clause)
()
(:public
#>cpp
ShowStreams() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
ShowStreams *Clone(AstStorage &storage) const override {
return storage.Create<ShowStreams>();
}
cpp<#)
(:protected
#>cpp
explicit ShowStreams(int uid) : Clause(uid) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class start-stop-stream (clause)
((stream-name "std::string" :scope :public)
(is-start :bool :scope :public)
(limit-batches "Expression *" :scope :public
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
(:public
#>cpp
StartStopStream() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
StartStopStream *Clone(AstStorage &storage) const override {
return storage.Create<StartStopStream>(
stream_name_, is_start_,
limit_batches_ ? limit_batches_->Clone(storage) : nullptr);
}
cpp<#)
(:protected
#>cpp
explicit StartStopStream(int uid) : Clause(uid) {}
StartStopStream(int uid, std::string stream_name, bool is_start,
Expression *limit_batches)
: Clause(uid),
stream_name_(std::move(stream_name)),
is_start_(is_start),
batch_size_(batch_size),
limit_batches_(limit_batches) {}
cpp<#)
(:private
@ -2450,67 +2443,6 @@ cpp<#
cpp<#)
(:serialize :capnp))
(lcp:define-class start-stop-all-streams (clause)
((is-start :bool :scope :public))
(:public
#>cpp
StartStopAllStreams() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
StartStopAllStreams *Clone(AstStorage &storage) const override {
return storage.Create<StartStopAllStreams>(is_start_);
}
cpp<#)
(:protected
#>cpp
explicit StartStopAllStreams(int uid) : Clause(uid) {}
StartStopAllStreams(int uid, bool is_start)
: Clause(uid), is_start_(is_start) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
(lcp:define-class test-stream (clause)
((stream-name "std::string" :scope :public)
(limit-batches "Expression *" :scope :public
:capnp-type "Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
(:public
#>cpp
TestStream() = default;
DEFVISITABLE(TreeVisitor<TypedValue>);
DEFVISITABLE(HierarchicalTreeVisitor);
TestStream *Clone(AstStorage &storage) const override {
return storage.Create<TestStream>(
stream_name_,
limit_batches_ ? limit_batches_->Clone(storage) : nullptr);
}
cpp<#)
(:protected
#>cpp
explicit TestStream(int uid) : Clause(uid) {}
TestStream(int uid, std::string stream_name, Expression *limit_batches)
: Clause(uid),
stream_name_(std::move(stream_name)),
limit_batches_(limit_batches) {}
cpp<#)
(:private
#>cpp
friend class AstStorage;
cpp<#)
(:serialize :capnp))
#>cpp
#undef CLONE_BINARY_EXPRESSION
#undef CLONE_UNARY_EXPRESSION

View File

@ -5,7 +5,7 @@
namespace query {
// Forward declares for Tree visitors.
class Query;
class CypherQuery;
class SingleQuery;
class CypherUnion;
class NamedExpression;
@ -60,22 +60,17 @@ class RemoveProperty;
class RemoveLabels;
class Merge;
class Unwind;
class CreateIndex;
class CreateUniqueIndex;
class AuthQuery;
class CreateStream;
class DropStream;
class ShowStreams;
class StartStopStream;
class StartStopAllStreams;
class TestStream;
class ExplainQuery;
class IndexQuery;
class StreamQuery;
using TreeCompositeVisitor = ::utils::CompositeVisitor<
Query, SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator,
AndOperator, NotOperator, AdditionOperator, SubtractionOperator,
MultiplicationOperator, DivisionOperator, ModOperator, NotEqualOperator,
EqualOperator, LessOperator, GreaterOperator, LessEqualOperator,
GreaterEqualOperator, InListOperator, SubscriptOperator,
CypherQuery, ExplainQuery, SingleQuery, CypherUnion, NamedExpression,
OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator,
SubtractionOperator, MultiplicationOperator, DivisionOperator, ModOperator,
NotEqualOperator, EqualOperator, LessOperator, GreaterOperator,
LessEqualOperator, GreaterEqualOperator, InListOperator, SubscriptOperator,
ListSlicingOperator, IfOperator, UnaryPlusOperator, UnaryMinusOperator,
IsNullOperator, ListLiteral, MapLiteral, PropertyLookup, LabelsTest,
Aggregation, Function, Reduce, Extract, All, Single, Create, Match, Return,
@ -84,9 +79,7 @@ using TreeCompositeVisitor = ::utils::CompositeVisitor<
using TreeLeafVisitor =
::utils::LeafVisitor<Identifier, PrimitiveLiteral, ParameterLookup,
CreateIndex, CreateUniqueIndex, AuthQuery,
CreateStream, DropStream, ShowStreams, StartStopStream,
StartStopAllStreams, TestStream>;
IndexQuery, AuthQuery, StreamQuery>;
class HierarchicalTreeVisitor : public TreeCompositeVisitor,
public TreeLeafVisitor {
@ -99,18 +92,17 @@ class HierarchicalTreeVisitor : public TreeCompositeVisitor,
template <typename TResult>
using TreeVisitor = ::utils::Visitor<
TResult, Query, SingleQuery, CypherUnion, NamedExpression, OrOperator,
XorOperator, AndOperator, NotOperator, AdditionOperator,
SubtractionOperator, MultiplicationOperator, DivisionOperator, ModOperator,
NotEqualOperator, EqualOperator, LessOperator, GreaterOperator,
LessEqualOperator, GreaterEqualOperator, InListOperator, SubscriptOperator,
ListSlicingOperator, IfOperator, UnaryPlusOperator, UnaryMinusOperator,
IsNullOperator, ListLiteral, MapLiteral, PropertyLookup, LabelsTest,
Aggregation, Function, Reduce, Extract, All, Single, ParameterLookup,
Create, Match, Return, With, Pattern, NodeAtom, EdgeAtom, Delete, Where,
SetProperty, SetProperties, SetLabels, RemoveProperty, RemoveLabels, Merge,
Unwind, Identifier, PrimitiveLiteral, CreateIndex, CreateUniqueIndex,
AuthQuery, CreateStream, DropStream, ShowStreams, StartStopStream,
StartStopAllStreams, TestStream>;
TResult, CypherQuery, ExplainQuery, SingleQuery, CypherUnion,
NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator,
AdditionOperator, SubtractionOperator, MultiplicationOperator,
DivisionOperator, ModOperator, NotEqualOperator, EqualOperator,
LessOperator, GreaterOperator, LessEqualOperator, GreaterEqualOperator,
InListOperator, SubscriptOperator, ListSlicingOperator, IfOperator,
UnaryPlusOperator, UnaryMinusOperator, IsNullOperator, ListLiteral,
MapLiteral, PropertyLookup, LabelsTest, Aggregation, Function, Reduce,
Extract, All, Single, ParameterLookup, Create, Match, Return, With, Pattern,
NodeAtom, EdgeAtom, Delete, Where, SetProperty, SetProperties, SetLabels,
RemoveProperty, RemoveLabels, Merge, Unwind, Identifier, PrimitiveLiteral,
IndexQuery, AuthQuery, StreamQuery>;
} // namespace query

View File

@ -27,17 +27,21 @@ const std::string CypherMainVisitor::kAnonPrefix = "anon";
antlrcpp::Any CypherMainVisitor::visitExplainQuery(
MemgraphCypher::ExplainQueryContext *ctx) {
visitChildren(ctx);
CHECK(query_);
query_->explain_ = true;
return query_;
CHECK(ctx->children.size() == 2)
<< "ExplainQuery should have exactly two children!";
auto *cypher_query = ctx->children[1]->accept(this).as<CypherQuery *>();
auto *explain_query = storage_->Create<ExplainQuery>();
explain_query->cypher_query_ = cypher_query;
query_ = explain_query;
return explain_query;
}
antlrcpp::Any CypherMainVisitor::visitCypherQuery(
MemgraphCypher::CypherQueryContext *ctx) {
query_ = storage_->Create<Query>();
DCHECK(ctx->singleQuery()) << "Expected single query.";
query_->single_query_ = ctx->singleQuery()->accept(this).as<SingleQuery *>();
auto *cypher_query = storage_->Create<CypherQuery>();
CHECK(ctx->singleQuery()) << "Expected single query.";
cypher_query->single_query_ =
ctx->singleQuery()->accept(this).as<SingleQuery *>();
// Check that union and union all dont mix
bool has_union = false;
@ -51,59 +55,64 @@ antlrcpp::Any CypherMainVisitor::visitCypherQuery(
if (has_union && has_union_all) {
throw SemanticException("Invalid combination of UNION and UNION ALL.");
}
query_->cypher_unions_.push_back(child->accept(this).as<CypherUnion *>());
cypher_query->cypher_unions_.push_back(
child->accept(this).as<CypherUnion *>());
}
return query_;
query_ = cypher_query;
return cypher_query;
}
antlrcpp::Any CypherMainVisitor::visitIndexQuery(
MemgraphCypher::IndexQueryContext *ctx) {
query_ = storage_->Create<Query>();
query_->single_query_ = storage_->Create<SingleQuery>();
if (ctx->createIndex()) {
query_->single_query_->clauses_.emplace_back(
ctx->createIndex()->accept(this).as<CreateIndex *>());
} else {
DCHECK(ctx->createUniqueIndex()) << "Expected CREATE UNIQUE INDEX";
query_->single_query_->clauses_.emplace_back(
ctx->createUniqueIndex()->accept(this).as<CreateUniqueIndex *>());
CHECK(ctx->children.size() == 1)
<< "IndexQuery should have exactly one child!";
auto *index_query = ctx->children[0]->accept(this).as<IndexQuery *>();
query_ = index_query;
return index_query;
}
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};
return index_query;
}
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->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);
}
return query_;
return index_query;
}
antlrcpp::Any CypherMainVisitor::visitAuthQuery(
MemgraphCypher::AuthQueryContext *ctx) {
query_ = storage_->Create<Query>();
query_->single_query_ = storage_->Create<SingleQuery>();
CHECK(ctx->children.size() == 1)
<< "AuthQuery should have exactly one child!";
query_->single_query_->clauses_.push_back(
ctx->children[0]->accept(this).as<AuthQuery *>());
return query_;
auto *auth_query = ctx->children[0]->accept(this).as<AuthQuery *>();
query_ = auth_query;
return auth_query;
}
antlrcpp::Any CypherMainVisitor::visitStreamQuery(
MemgraphCypher::StreamQueryContext *ctx) {
query_ = storage_->Create<Query>();
query_->single_query_ = storage_->Create<SingleQuery>();
Clause *clause = nullptr;
if (ctx->createStream()) {
clause = ctx->createStream()->accept(this).as<CreateStream *>();
} else if (ctx->dropStream()) {
clause = ctx->dropStream()->accept(this).as<DropStream *>();
} else if (ctx->showStreams()) {
clause = ctx->showStreams()->accept(this).as<ShowStreams *>();
} else if (ctx->startStopStream()) {
clause = ctx->startStopStream()->accept(this).as<StartStopStream *>();
} else if (ctx->startStopAllStreams()) {
clause =
ctx->startStopAllStreams()->accept(this).as<StartStopAllStreams *>();
} else if (ctx->testStream()) {
clause = ctx->testStream()->accept(this).as<TestStream *>();
}
query_->single_query_->clauses_ = {clause};
return query_;
CHECK(ctx->children.size() == 1)
<< "StreamQuery should have exactly one child!";
auto *stream_query = ctx->children[0]->accept(this).as<StreamQuery *>();
query_ = stream_query;
return stream_query;
}
antlrcpp::Any CypherMainVisitor::visitCypherUnion(
@ -138,7 +147,6 @@ antlrcpp::Any CypherMainVisitor::visitSingleQuery(
bool has_update = false;
bool has_return = false;
bool has_optional_match = false;
bool has_create_index = false;
for (Clause *clause : single_query->clauses_) {
if (dynamic_cast<Unwind *>(clause)) {
@ -178,19 +186,11 @@ antlrcpp::Any CypherMainVisitor::visitSingleQuery(
throw SemanticException("RETURN can't be put before WITH.");
}
has_update = has_return = has_optional_match = false;
} else if (dynamic_cast<CreateIndex *>(clause) ||
dynamic_cast<CreateUniqueIndex *>(clause)) {
// If there is CreateIndex clause then there shouldn't be anything else.
if (single_query->clauses_.size() != 1U) {
throw SemanticException(
"CREATE INDEX must be the only clause in a query.");
}
has_create_index = true;
} else {
DLOG(FATAL) << "Can't happen";
}
}
if (!has_update && !has_return && !has_create_index) {
if (!has_update && !has_return) {
throw SemanticException(
"Query should either create or update something, or return results!");
}
@ -266,33 +266,6 @@ antlrcpp::Any CypherMainVisitor::visitCreate(
return create;
}
/**
* @return CreateIndex*
*/
antlrcpp::Any CypherMainVisitor::visitCreateIndex(
MemgraphCypher::CreateIndexContext *ctx) {
std::pair<std::string, storage::Property> key =
ctx->propertyKeyName()->accept(this);
return storage_->Create<CreateIndex>(
dba_->Label(ctx->labelName()->accept(this)), key.second);
}
/**
* @return CreateUniqueIndex*
*/
antlrcpp::Any CypherMainVisitor::visitCreateUniqueIndex(
MemgraphCypher::CreateUniqueIndexContext *ctx) {
std::vector<storage::Property> properties;
properties.reserve(ctx->propertyKeyName().size());
for (const auto &prop_name : ctx->propertyKeyName()) {
std::pair<std::string, storage::Property> name_key =
prop_name->accept(this);
properties.push_back(name_key.second);
}
return storage_->Create<CreateUniqueIndex>(
dba_->Label(ctx->labelName()->accept(this)), properties);
}
/**
* @return std::string
*/
@ -522,39 +495,33 @@ antlrcpp::Any CypherMainVisitor::visitShowUsersForRole(
}
/**
* @return CreateStream*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitCreateStream(
MemgraphCypher::CreateStreamContext *ctx) {
std::string stream_name(ctx->streamName()->getText());
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::CREATE_STREAM;
stream_query->stream_name_ = ctx->streamName()->getText();
if (!ctx->streamUri->StringLiteral()) {
throw SyntaxException("Stream URI should be a string literal.");
}
Expression *stream_uri = ctx->streamUri->accept(this);
stream_query->stream_uri_ = ctx->streamUri->accept(this);
if (!ctx->streamTopic->StringLiteral()) {
throw SyntaxException("Topic should be a string literal.");
}
Expression *stream_topic = ctx->streamTopic->accept(this);
stream_query->stream_topic_ = ctx->streamTopic->accept(this);
if (!ctx->transformUri->StringLiteral()) {
throw SyntaxException("Transform URI should be a string literal.");
}
Expression *transform_uri = ctx->transformUri->accept(this);
Expression *batch_interval_in_ms = nullptr;
stream_query->transform_uri_ = ctx->transformUri->accept(this);
if (ctx->batchIntervalOption()) {
batch_interval_in_ms = ctx->batchIntervalOption()->accept(this);
stream_query->batch_interval_in_ms_ =
ctx->batchIntervalOption()->accept(this);
}
Expression *batch_size = nullptr;
if (ctx->batchSizeOption()) {
batch_size = ctx->batchSizeOption()->accept(this);
stream_query->batch_size_ = ctx->batchSizeOption()->accept(this);
}
return storage_->Create<CreateStream>(stream_name, stream_uri, stream_topic,
transform_uri, batch_interval_in_ms,
batch_size);
return stream_query;
}
/**
@ -582,40 +549,49 @@ antlrcpp::Any CypherMainVisitor::visitBatchSizeOption(
}
/**
* @return DropStream*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitDropStream(
MemgraphCypher::DropStreamContext *ctx) {
return storage_->Create<DropStream>(
std::string(ctx->streamName()->getText()));
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::DROP_STREAM;
stream_query->stream_name_ = ctx->streamName()->getText();
return stream_query;
}
/**
* @return ShowStreams*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitShowStreams(
MemgraphCypher::ShowStreamsContext *ctx) {
return storage_->Create<ShowStreams>();
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::SHOW_STREAMS;
return stream_query;
}
/**
* @return StartStopStream*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitStartStopStream(
MemgraphCypher::StartStopStreamContext *ctx) {
std::string stream_name(std::string(ctx->streamName()->getText()));
bool is_start = static_cast<bool>(ctx->START());
Expression *limit_batches = nullptr;
antlrcpp::Any CypherMainVisitor::visitStartStream(
MemgraphCypher::StartStreamContext *ctx) {
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::START_STREAM;
stream_query->stream_name_ = std::string(ctx->streamName()->getText());
if (ctx->limitBatchesOption()) {
if (!is_start) {
throw SyntaxException("STOP STREAM can't set batch limit.");
}
limit_batches = ctx->limitBatchesOption()->accept(this);
stream_query->limit_batches_ = ctx->limitBatchesOption()->accept(this);
}
return stream_query;
}
return storage_->Create<StartStopStream>(stream_name, is_start,
limit_batches);
/**
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitStopStream(
MemgraphCypher::StopStreamContext *ctx) {
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::STOP_STREAM;
stream_query->stream_name_ = std::string(ctx->streamName()->getText());
return stream_query;
}
/**
@ -631,12 +607,37 @@ antlrcpp::Any CypherMainVisitor::visitLimitBatchesOption(
}
/*
* @return StartStopAllStreams*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitStartStopAllStreams(
MemgraphCypher::StartStopAllStreamsContext *ctx) {
bool is_start = static_cast<bool>(ctx->START());
return storage_->Create<StartStopAllStreams>(is_start);
antlrcpp::Any CypherMainVisitor::visitStartAllStreams(
MemgraphCypher::StartAllStreamsContext *ctx) {
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::START_ALL_STREAMS;
return stream_query;
}
/*
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitStopAllStreams(
MemgraphCypher::StopAllStreamsContext *ctx) {
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::STOP_ALL_STREAMS;
return stream_query;
}
/**
* @return StreamQuery*
*/
antlrcpp::Any CypherMainVisitor::visitTestStream(
MemgraphCypher::TestStreamContext *ctx) {
auto *stream_query = storage_->Create<StreamQuery>();
stream_query->action_ = StreamQuery::Action::TEST_STREAM;
stream_query->stream_name_ = std::string(ctx->streamName()->getText());
if (ctx->limitBatchesOption()) {
stream_query->limit_batches_ = ctx->limitBatchesOption()->accept(this);
}
return stream_query;
}
antlrcpp::Any CypherMainVisitor::visitCypherReturn(
@ -649,21 +650,6 @@ antlrcpp::Any CypherMainVisitor::visitCypherReturn(
return return_clause;
}
/**
* @return TestStream*
*/
antlrcpp::Any CypherMainVisitor::visitTestStream(
MemgraphCypher::TestStreamContext *ctx) {
std::string stream_name(std::string(ctx->streamName()->getText()));
Expression *limit_batches = nullptr;
if (ctx->limitBatchesOption()) {
limit_batches = ctx->limitBatchesOption()->accept(this);
}
return storage_->Create<TestStream>(stream_name, limit_batches);
}
antlrcpp::Any CypherMainVisitor::visitReturnBody(
MemgraphCypher::ReturnBodyContext *ctx) {
ReturnBody body;

View File

@ -135,30 +135,30 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
}
/**
* @return Query*
* @return CypherQuery*
*/
antlrcpp::Any visitCypherQuery(
MemgraphCypher::CypherQueryContext *ctx) override;
/**
* @return Query*
* @return IndexQuery*
*/
antlrcpp::Any visitIndexQuery(
MemgraphCypher::IndexQueryContext *ctx) override;
/**
* @return Query*
* @return ExplainQuery*
*/
antlrcpp::Any visitExplainQuery(
MemgraphCypher::ExplainQueryContext *ctx) override;
/**
* @return Query*
* @return AuthQuery*
*/
antlrcpp::Any visitAuthQuery(MemgraphCypher::AuthQueryContext *ctx) override;
/**
* @return Query*
* @return StreamQuery*
*/
antlrcpp::Any visitStreamQuery(
MemgraphCypher::StreamQueryContext *ctx) override;
@ -214,7 +214,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
antlrcpp::Any visitShowRoles(MemgraphCypher::ShowRolesContext *ctx) override;
/**
* @return CreateIndex*
* @return IndexQuery*
*/
antlrcpp::Any visitCreateIndex(
MemgraphCypher::CreateIndexContext *ctx) override;
@ -299,7 +299,7 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
MemgraphCypher::ShowUsersForRoleContext *ctx) override;
/**
* @return CreateStream*
* @return StreamQuery*
*/
antlrcpp::Any visitCreateStream(
MemgraphCypher::CreateStreamContext *ctx) override;
@ -311,34 +311,46 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
MemgraphCypher::BatchSizeOptionContext *ctx) override;
/**
* @return DropStream*
* @return StreamQuery*
*/
antlrcpp::Any visitDropStream(
MemgraphCypher::DropStreamContext *ctx) override;
/**
* @return ShowStreams*
* @return StreamQuery*
*/
antlrcpp::Any visitShowStreams(
MemgraphCypher::ShowStreamsContext *ctx) override;
/**
* @return StartStopStream*
* @return StreamQuery*
*/
antlrcpp::Any visitStartStopStream(
MemgraphCypher::StartStopStreamContext *ctx) override;
antlrcpp::Any visitStartStream(
MemgraphCypher::StartStreamContext *ctx) override;
/**
* @return StartStopAllStreams*
* @return StreamQuery*
*/
antlrcpp::Any visitStartStopAllStreams(
MemgraphCypher::StartStopAllStreamsContext *ctx) override;
antlrcpp::Any visitStopStream(
MemgraphCypher::StopStreamContext *ctx) override;
antlrcpp::Any visitLimitBatchesOption(
MemgraphCypher::LimitBatchesOptionContext *ctx) override;
/**
* @return TestStream*
* @return StreamQuery*
*/
antlrcpp::Any visitStartAllStreams(
MemgraphCypher::StartAllStreamsContext *ctx) override;
/**
* @return StreamQuery*
*/
antlrcpp::Any visitStopAllStreams(
MemgraphCypher::StopAllStreamsContext *ctx) override;
/**
* @return StreamQuery*
*/
antlrcpp::Any visitTestStream(
MemgraphCypher::TestStreamContext *ctx) override;

View File

@ -110,8 +110,10 @@ showUsersForRole : SHOW USERS FOR role=userOrRoleName ;
streamQuery : createStream
| dropStream
| showStreams
| startStopStream
| startStopAllStreams
| startStream
| stopStream
| startAllStreams
| stopAllStreams
| testStream
;
@ -129,10 +131,14 @@ dropStream : DROP STREAM streamName ;
showStreams : SHOW STREAMS ;
startStopStream : ( START | STOP ) STREAM streamName ( limitBatchesOption )? ;
startStream : START STREAM streamName ( limitBatchesOption )? ;
stopStream : STOP STREAM streamName ;
limitBatchesOption : LIMIT limitBatches=literal BATCHES ;
startStopAllStreams : ( START | STOP ) ALL STREAMS ;
startAllStreams : START ALL STREAMS ;
stopAllStreams : STOP ALL STREAMS ;
testStream : K_TEST STREAM streamName ( limitBatchesOption )? ;

View File

@ -50,12 +50,7 @@ class PrivilegeExtractor : public HierarchicalTreeVisitor {
bool Visit(PrimitiveLiteral &) override { return true; }
bool Visit(ParameterLookup &) override { return true; }
bool Visit(CreateIndex &) override {
AddPrivilege(AuthQuery::Privilege::INDEX);
return true;
}
bool Visit(CreateUniqueIndex &) override {
bool Visit(IndexQuery &) override {
AddPrivilege(AuthQuery::Privilege::INDEX);
return true;
}
@ -64,27 +59,7 @@ class PrivilegeExtractor : public HierarchicalTreeVisitor {
AddPrivilege(AuthQuery::Privilege::AUTH);
return true;
}
bool Visit(CreateStream &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}
bool Visit(DropStream &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}
bool Visit(ShowStreams &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}
bool Visit(StartStopStream &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}
bool Visit(StartStopAllStreams &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}
bool Visit(TestStream &) override {
bool Visit(StreamQuery &) override {
AddPrivilege(AuthQuery::Privilege::STREAM);
return true;
}

View File

@ -218,23 +218,11 @@ bool SymbolGenerator::PostVisit(Match &) {
return true;
}
bool SymbolGenerator::Visit(CreateIndex &) { return true; }
bool SymbolGenerator::Visit(CreateUniqueIndex &) { return true; }
bool SymbolGenerator::Visit(IndexQuery &) { return true; }
bool SymbolGenerator::Visit(AuthQuery &) { return true; }
bool SymbolGenerator::Visit(CreateStream &) { return true; }
bool SymbolGenerator::Visit(DropStream &) { return true; }
bool SymbolGenerator::Visit(ShowStreams &) { return true; }
bool SymbolGenerator::Visit(StartStopStream &) { return true; }
bool SymbolGenerator::Visit(StartStopAllStreams &) { return true; }
bool SymbolGenerator::Visit(TestStream &) { return true; }
bool SymbolGenerator::Visit(StreamQuery &) { return true; }
// Expressions

View File

@ -46,15 +46,9 @@ class SymbolGenerator : public HierarchicalTreeVisitor {
bool PostVisit(Unwind &) override;
bool PreVisit(Match &) override;
bool PostVisit(Match &) override;
bool Visit(CreateIndex &) override;
bool Visit(CreateUniqueIndex &) override;
bool Visit(IndexQuery &) override;
bool Visit(AuthQuery &) override;
bool Visit(CreateStream &) override;
bool Visit(DropStream &) override;
bool Visit(ShowStreams &) override;
bool Visit(StartStopStream &) override;
bool Visit(StartStopAllStreams &) override;
bool Visit(TestStream &) override;
bool Visit(StreamQuery &) override;
// Expressions
ReturnType Visit(Identifier &) override;

View File

@ -36,7 +36,8 @@ class ExpressionEvaluator : public TreeVisitor<TypedValue> {
LOG(FATAL) << "ExpressionEvaluator should not visit " #TREE_TYPE; \
}
BLOCK_VISIT(Query);
BLOCK_VISIT(CypherQuery);
BLOCK_VISIT(ExplainQuery);
BLOCK_VISIT(CypherUnion);
BLOCK_VISIT(SingleQuery);
BLOCK_VISIT(Create);
@ -55,15 +56,10 @@ class ExpressionEvaluator : public TreeVisitor<TypedValue> {
BLOCK_VISIT(RemoveLabels);
BLOCK_VISIT(Merge);
BLOCK_VISIT(Unwind);
BLOCK_VISIT(CreateIndex);
BLOCK_VISIT(CreateUniqueIndex);
BLOCK_VISIT(IndexQuery);
BLOCK_VISIT(AuthQuery);
BLOCK_VISIT(CreateStream);
BLOCK_VISIT(DropStream);
BLOCK_VISIT(ShowStreams);
BLOCK_VISIT(StartStopStream);
BLOCK_VISIT(StartStopAllStreams);
BLOCK_VISIT(TestStream);
BLOCK_VISIT(StreamQuery);
#undef BLOCK_VISIT

View File

@ -3,14 +3,21 @@
#include <glog/logging.h>
#include <limits>
#include "auth/auth.hpp"
#include "glue/auth.hpp"
#include "glue/communication.hpp"
#include "integrations/kafka/exceptions.hpp"
#include "integrations/kafka/streams.hpp"
#include "query/exceptions.hpp"
#include "query/frontend/ast/cypher_main_visitor.hpp"
#include "query/frontend/opencypher/parser.hpp"
#include "query/frontend/semantic/required_privileges.hpp"
#include "query/frontend/semantic/symbol_generator.hpp"
#include "query/interpret/eval.hpp"
#include "query/plan/planner.hpp"
#include "query/plan/vertex_count_cache.hpp"
#include "utils/flag_validation.hpp"
#include "utils/string.hpp"
DEFINE_HIDDEN_bool(query_cost_planner, true,
"Use the cost-estimating query planner.");
@ -20,163 +27,6 @@ DEFINE_VALIDATED_int32(query_plan_cache_ttl, 60,
namespace query {
Interpreter::CachedPlan::CachedPlan(std::unique_ptr<LogicalPlan> plan)
: plan_(std::move(plan)) {}
Interpreter::Results Interpreter::operator()(
const std::string &query, database::GraphDbAccessor &db_accessor,
const std::map<std::string, PropertyValue> &params,
bool in_explicit_transaction) {
utils::Timer frontend_timer;
EvaluationContext evaluation_context;
evaluation_context.timestamp =
std::chrono::duration_cast<std::chrono::milliseconds>(
std::chrono::system_clock::now().time_since_epoch())
.count();
// query -> stripped query
StrippedQuery stripped(query);
// Update context with provided parameters.
evaluation_context.parameters = stripped.literals();
for (const auto &param_pair : stripped.parameters()) {
auto param_it = params.find(param_pair.second);
if (param_it == params.end()) {
throw query::UnprovidedParameterError(
fmt::format("Parameter ${} not provided.", param_pair.second));
}
evaluation_context.parameters.Add(param_pair.first, param_it->second);
}
Context ctx(db_accessor);
ctx.in_explicit_transaction_ = in_explicit_transaction;
ctx.auth_ = auth_;
ctx.kafka_streams_ = kafka_streams_;
ctx.evaluation_context_ = evaluation_context;
ParsingContext parsing_context;
parsing_context.is_query_cached = true;
AstStorage ast_storage;
Query *ast =
QueryToAst(stripped, parsing_context, &ast_storage, &db_accessor);
// TODO: Maybe cache required privileges to improve performance on very simple
// queries.
auto required_privileges = query::GetRequiredPrivileges(ast);
auto frontend_time = frontend_timer.Elapsed();
// Try to get a cached plan. Note that this local shared_ptr might be the only
// owner of the CachedPlan, so ensure it lives during the whole
// interpretation.
std::shared_ptr<CachedPlan> plan{nullptr};
auto plan_cache_access = plan_cache_.access();
auto it = plan_cache_access.find(stripped.hash());
if (it != plan_cache_access.end()) {
if (it->second->IsExpired())
plan_cache_access.remove(stripped.hash());
else
plan = it->second;
}
utils::Timer planning_timer;
if (!plan) {
plan = plan_cache_access
.insert(stripped.hash(),
AstToPlan(ast, std::move(ast_storage), &ctx))
.first->second;
}
auto planning_time = planning_timer.Elapsed();
ctx.symbol_table_ = plan->symbol_table();
std::map<std::string, TypedValue> summary;
summary["parsing_time"] = frontend_time.count();
summary["planning_time"] = planning_time.count();
summary["cost_estimate"] = plan->cost();
// TODO: set summary['type'] based on transaction metadata
// the type can't be determined based only on top level LogicalOp
// (for example MATCH DELETE RETURN will have Produce as it's top)
// for now always use "rw" because something must be set, but it doesn't
// have to be correct (for Bolt clients)
summary["type"] = "rw";
auto cursor = plan->plan().MakeCursor(ctx.db_accessor_);
std::vector<std::string> header;
std::vector<Symbol> output_symbols(
plan->plan().OutputSymbols(ctx.symbol_table_));
for (const auto &symbol : output_symbols) {
// When the symbol is aliased or expanded from '*' (inside RETURN or
// WITH), then there is no token position, so use symbol name.
// Otherwise, find the name from stripped query.
header.push_back(utils::FindOr(stripped.named_expressions(),
symbol.token_position(), symbol.name())
.first);
}
return Results(std::move(ctx), plan, std::move(cursor), output_symbols,
header, summary, plan_cache_, required_privileges);
}
std::shared_ptr<Interpreter::CachedPlan> Interpreter::AstToPlan(
Query *query, AstStorage ast_storage, Context *ctx) {
SymbolGenerator symbol_generator(ctx->symbol_table_);
query->Accept(symbol_generator);
return std::make_shared<CachedPlan>(
MakeLogicalPlan(query, std::move(ast_storage), ctx));
}
Query *Interpreter::QueryToAst(const StrippedQuery &stripped,
const ParsingContext &context,
AstStorage *ast_storage,
database::GraphDbAccessor *db_accessor) {
if (!context.is_query_cached) {
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<utils::SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(
stripped.original_query());
}();
auto low_level_tree = parser->tree();
// AST -> high level tree
frontend::CypherMainVisitor visitor(context, ast_storage, db_accessor);
visitor.visit(low_level_tree);
return visitor.query();
}
auto ast_cache_accessor = ast_cache_.access();
auto ast_it = ast_cache_accessor.find(stripped.hash());
if (ast_it == ast_cache_accessor.end()) {
// stripped query -> AST
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<utils::SpinLock> guard(antlr_lock_);
try {
return std::make_unique<frontend::opencypher::Parser>(stripped.query());
} catch (const SyntaxException &e) {
// There is syntax exception in stripped query. Rerun parser with
// original query to get appropriate error messsage.
auto parser = std::make_unique<frontend::opencypher::Parser>(
stripped.original_query());
// If exception was not thrown here, it means StrippedQuery messed up
// something.
LOG(FATAL) << "Stripped query can't be parsed, original can";
return parser;
}
}();
auto low_level_tree = parser->tree();
// AST -> high level tree
CachedQuery cached_query;
frontend::CypherMainVisitor visitor(context, &cached_query.ast_storage,
db_accessor);
visitor.visit(low_level_tree);
cached_query.query = visitor.query();
// Cache it.
ast_it = ast_cache_accessor.insert(stripped.hash(), std::move(cached_query))
.first;
}
return ast_it->second.query->Clone(*ast_storage);
}
class SingleNodeLogicalPlan final : public LogicalPlan {
public:
SingleNodeLogicalPlan(std::unique_ptr<plan::LogicalOperator> root,
@ -198,18 +48,768 @@ class SingleNodeLogicalPlan final : public LogicalPlan {
SymbolTable symbol_table_;
};
Interpreter::CachedPlan::CachedPlan(std::unique_ptr<LogicalPlan> plan)
: plan_(std::move(plan)) {}
void Interpreter::PrettyPrintPlan(const database::GraphDbAccessor &dba,
const plan::LogicalOperator *plan_root,
std::ostream *out) {
plan::PrettyPrint(dba, plan_root, out);
}
struct Callback {
std::vector<std::string> header;
std::function<std::vector<std::vector<TypedValue>>()> fn;
};
TypedValue EvaluateOptionalExpression(Expression *expression,
ExpressionEvaluator *eval) {
return expression ? expression->Accept(*eval) : TypedValue::Null;
}
Callback HandleAuthQuery(AuthQuery *auth_query, auth::Auth *auth,
const EvaluationContext &evaluation_context,
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;
ExpressionEvaluator evaluator(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
AuthQuery::Action action = auth_query->action_;
std::string username = auth_query->user_;
std::string rolename = auth_query->role_;
std::string user_or_role = auth_query->user_or_role_;
std::vector<AuthQuery::Privilege> privileges = auth_query->privileges_;
auto password = EvaluateOptionalExpression(auth_query->password_, &evaluator);
Callback callback;
switch (auth_query->action_) {
case AuthQuery::Action::CREATE_USER:
callback.fn = [auth, username, password] {
CHECK(password.IsString() || password.IsNull());
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->AddUser(
username, password.IsString() ? std::experimental::make_optional(
password.ValueString())
: std::experimental::nullopt);
if (!user) {
throw QueryRuntimeException("User or role '{}' already exists.",
username);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::DROP_USER:
callback.fn = [auth, username] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->GetUser(username);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", username);
}
if (!auth->RemoveUser(username)) {
throw QueryRuntimeException("Couldn't remove user '{}'.", username);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::SET_PASSWORD:
callback.fn = [auth, username, password] {
CHECK(password.IsString() || password.IsNull());
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->GetUser(username);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", username);
}
user->UpdatePassword(
password.IsString()
? std::experimental::make_optional(password.ValueString())
: std::experimental::nullopt);
auth->SaveUser(*user);
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::CREATE_ROLE:
callback.fn = [auth, rolename] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto role = auth->AddRole(rolename);
if (!role) {
throw QueryRuntimeException("User or role '{}' already exists.",
rolename);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::DROP_ROLE:
callback.fn = [auth, rolename] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto role = auth->GetRole(rolename);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist.", rolename);
}
if (!auth->RemoveRole(rolename)) {
throw QueryRuntimeException("Couldn't remove role '{}'.", rolename);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::SHOW_USERS:
callback.header = {"user"};
callback.fn = [auth] {
std::lock_guard<std::mutex> lock(auth->WithLock());
std::vector<std::vector<TypedValue>> users;
for (const auto &user : auth->AllUsers()) {
users.push_back({user.username()});
}
return users;
};
return callback;
case AuthQuery::Action::SHOW_ROLES:
callback.header = {"role"};
callback.fn = [auth] {
std::lock_guard<std::mutex> lock(auth->WithLock());
std::vector<std::vector<TypedValue>> roles;
for (const auto &role : auth->AllRoles()) {
roles.push_back({role.rolename()});
}
return roles;
};
return callback;
case AuthQuery::Action::SET_ROLE:
callback.fn = [auth, username, rolename] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->GetUser(username);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist .", username);
}
auto role = auth->GetRole(rolename);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist .", rolename);
}
if (user->role()) {
throw QueryRuntimeException(
"User '{}' is already a member of role '{}'.", username,
user->role()->rolename());
}
user->SetRole(*role);
auth->SaveUser(*user);
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::CLEAR_ROLE:
callback.fn = [auth, username] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->GetUser(username);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist .", username);
}
user->ClearRole();
auth->SaveUser(*user);
return std::vector<std::vector<TypedValue>>();
};
return callback;
case AuthQuery::Action::GRANT_PRIVILEGE:
case AuthQuery::Action::DENY_PRIVILEGE:
case AuthQuery::Action::REVOKE_PRIVILEGE: {
callback.fn = [auth, user_or_role, action, privileges] {
std::lock_guard<std::mutex> lock(auth->WithLock());
std::vector<auth::Permission> permissions;
for (const auto &privilege : privileges) {
permissions.push_back(glue::PrivilegeToPermission(privilege));
}
auto user = auth->GetUser(user_or_role);
auto role = auth->GetRole(user_or_role);
if (!user && !role) {
throw QueryRuntimeException("User or role '{}' doesn't exist.",
user_or_role);
}
if (user) {
for (const auto &permission : permissions) {
// TODO (mferencevic): should we first check that the privilege
// is granted/denied/revoked before unconditionally
// granting/denying/revoking it?
if (action == AuthQuery::Action::GRANT_PRIVILEGE) {
user->permissions().Grant(permission);
} else if (action == AuthQuery::Action::DENY_PRIVILEGE) {
user->permissions().Deny(permission);
} else {
user->permissions().Revoke(permission);
}
}
auth->SaveUser(*user);
} else {
for (const auto &permission : permissions) {
// TODO (mferencevic): should we first check that the privilege
// is granted/denied/revoked before unconditionally
// granting/denying/revoking it?
if (action == AuthQuery::Action::GRANT_PRIVILEGE) {
role->permissions().Grant(permission);
} else if (action == AuthQuery::Action::DENY_PRIVILEGE) {
role->permissions().Deny(permission);
} else {
role->permissions().Revoke(permission);
}
}
auth->SaveRole(*role);
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
}
case AuthQuery::Action::SHOW_PRIVILEGES:
callback.header = {"privilege", "effective", "description"};
callback.fn = [auth, user_or_role] {
std::lock_guard<std::mutex> lock(auth->WithLock());
std::vector<std::vector<TypedValue>> grants;
auto user = auth->GetUser(user_or_role);
auto role = auth->GetRole(user_or_role);
if (!user && !role) {
throw QueryRuntimeException("User or role '{}' doesn't exist.",
user_or_role);
}
if (user) {
const auto &permissions = user->GetPermissions();
for (const auto &privilege : kPrivilegesAll) {
auto permission = glue::PrivilegeToPermission(privilege);
auto effective = permissions.Has(permission);
if (permissions.Has(permission) != auth::PermissionLevel::NEUTRAL) {
std::vector<std::string> description;
auto user_level = user->permissions().Has(permission);
if (user_level == auth::PermissionLevel::GRANT) {
description.push_back("GRANTED TO USER");
} else if (user_level == auth::PermissionLevel::DENY) {
description.push_back("DENIED TO USER");
}
if (user->role()) {
auto role_level = user->role()->permissions().Has(permission);
if (role_level == auth::PermissionLevel::GRANT) {
description.push_back("GRANTED TO ROLE");
} else if (role_level == auth::PermissionLevel::DENY) {
description.push_back("DENIED TO ROLE");
}
}
grants.push_back({auth::PermissionToString(permission),
auth::PermissionLevelToString(effective),
utils::Join(description, ", ")});
}
}
} else {
const auto &permissions = role->permissions();
for (const auto &privilege : kPrivilegesAll) {
auto permission = glue::PrivilegeToPermission(privilege);
auto effective = permissions.Has(permission);
if (effective != auth::PermissionLevel::NEUTRAL) {
std::string description;
if (effective == auth::PermissionLevel::GRANT) {
description = "GRANTED TO ROLE";
} else if (effective == auth::PermissionLevel::DENY) {
description = "DENIED TO ROLE";
}
grants.push_back({auth::PermissionToString(permission),
auth::PermissionLevelToString(effective),
description});
}
}
}
return grants;
};
return callback;
case AuthQuery::Action::SHOW_ROLE_FOR_USER:
callback.header = {"role"};
callback.fn = [auth, username] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto user = auth->GetUser(username);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist .", username);
}
return std::vector<std::vector<TypedValue>>{std::vector<TypedValue>{
user->role() ? user->role()->rolename() : "null"}};
};
return callback;
case AuthQuery::Action::SHOW_USERS_FOR_ROLE:
callback.header = {"users"};
callback.fn = [auth, rolename] {
std::lock_guard<std::mutex> lock(auth->WithLock());
auto role = auth->GetRole(rolename);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist.", rolename);
}
std::vector<std::vector<TypedValue>> users;
for (const auto &user : auth->AllUsersForRole(rolename)) {
users.emplace_back(std::vector<TypedValue>{user.username()});
}
return users;
};
return callback;
default:
break;
}
}
Callback HandleStreamQuery(StreamQuery *stream_query,
integrations::kafka::Streams *streams,
const EvaluationContext &evaluation_context,
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;
ExpressionEvaluator eval(&frame, symbol_table, evaluation_context,
db_accessor, GraphView::OLD);
std::string stream_name = stream_query->stream_name_;
auto stream_uri =
EvaluateOptionalExpression(stream_query->stream_uri_, &eval);
auto stream_topic =
EvaluateOptionalExpression(stream_query->stream_topic_, &eval);
auto transform_uri =
EvaluateOptionalExpression(stream_query->transform_uri_, &eval);
auto batch_interval_in_ms =
EvaluateOptionalExpression(stream_query->batch_interval_in_ms_, &eval);
auto batch_size =
EvaluateOptionalExpression(stream_query->batch_size_, &eval);
auto limit_batches =
EvaluateOptionalExpression(stream_query->limit_batches_, &eval);
Callback callback;
switch (stream_query->action_) {
case StreamQuery::Action::CREATE_STREAM:
callback.fn = [streams, stream_name, stream_uri, stream_topic,
transform_uri, batch_interval_in_ms, batch_size] {
CHECK(stream_uri.IsString());
CHECK(stream_topic.IsString());
CHECK(transform_uri.IsString());
CHECK(batch_interval_in_ms.IsInt() || batch_interval_in_ms.IsNull());
CHECK(batch_size.IsInt() || batch_size.IsNull());
integrations::kafka::StreamInfo info;
info.stream_name = stream_name;
info.stream_uri = stream_uri.ValueString();
info.stream_topic = stream_topic.ValueString();
info.transform_uri = transform_uri.ValueString();
info.batch_interval_in_ms = batch_interval_in_ms.IsInt()
? std::experimental::make_optional(
batch_interval_in_ms.ValueInt())
: std::experimental::nullopt;
info.batch_size =
batch_size.IsInt()
? std::experimental::make_optional(batch_size.ValueInt())
: std::experimental::nullopt;
try {
streams->Create(info);
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::DROP_STREAM:
callback.fn = [streams, stream_name] {
try {
streams->Drop(stream_name);
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::SHOW_STREAMS:
callback.header = {"name", "uri", "topic", "transform", "status"};
callback.fn = [streams] {
std::vector<std::vector<TypedValue>> status;
for (const auto &stream : streams->Show()) {
status.push_back(std::vector<TypedValue>{
stream.stream_name, stream.stream_uri, stream.stream_topic,
stream.transform_uri, stream.stream_status});
}
return status;
};
return callback;
case StreamQuery::Action::START_STREAM:
callback.fn = [streams, stream_name, limit_batches] {
CHECK(limit_batches.IsInt() || limit_batches.IsNull());
try {
streams->Start(stream_name, limit_batches.IsInt()
? std::experimental::make_optional(
limit_batches.ValueInt())
: std::experimental::nullopt);
} catch (integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::STOP_STREAM:
callback.fn = [streams, stream_name] {
try {
streams->Stop(stream_name);
} catch (integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::START_ALL_STREAMS:
callback.fn = [streams] {
try {
streams->StartAll();
} catch (integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::STOP_ALL_STREAMS:
callback.fn = [streams] {
try {
streams->StopAll();
} catch (integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
case StreamQuery::Action::TEST_STREAM:
callback.header = {"query", "params"};
callback.fn = [streams, stream_name, limit_batches] {
CHECK(limit_batches.IsInt() || limit_batches.IsNull());
std::vector<std::vector<TypedValue>> rows;
try {
auto results = streams->Test(
stream_name,
limit_batches.IsInt()
? std::experimental::make_optional(limit_batches.ValueInt())
: std::experimental::nullopt);
LOG(INFO) << results.size();
for (const auto &result : results) {
std::map<std::string, TypedValue> params;
for (const auto &param : result.second) {
params.emplace(param.first, glue::ToTypedValue(param.second));
}
rows.emplace_back(std::vector<TypedValue>{result.first, params});
}
} catch (integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return rows;
};
return callback;
}
}
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_;
if (properties.size() > 1) {
throw utils::NotYetImplemented("index on multiple properties");
}
Callback callback;
switch (index_query->action_) {
case IndexQuery::Action::CREATE:
case IndexQuery::Action::CREATE_UNIQUE:
callback.fn = [action, label, properties, db_accessor,
invalidate_plan_cache] {
try {
CHECK(properties.size() == 1);
db_accessor->BuildIndex(label, properties[0],
action == IndexQuery::Action::CREATE_UNIQUE);
invalidate_plan_cache();
} catch (const database::IndexConstraintViolationException &e) {
throw QueryRuntimeException(e.what());
} catch (const database::IndexExistsException &e) {
if (action == IndexQuery::Action::CREATE_UNIQUE) {
throw QueryRuntimeException(e.what());
}
// Otherwise ignore creating an existing index.
}
return std::vector<std::vector<TypedValue>>();
};
return callback;
}
} // namespace query
Interpreter::Results Interpreter::operator()(
const std::string &query_string, database::GraphDbAccessor &db_accessor,
const std::map<std::string, PropertyValue> &params,
bool in_explicit_transaction) {
utils::Timer frontend_timer;
// Strip the input query.
StrippedQuery stripped_query(query_string);
Context execution_context(db_accessor);
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();
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);
}
ParsingContext parsing_context;
parsing_context.is_query_cached = true;
AstStorage ast_storage;
Query *ast_root = ParseQuery(stripped_query.query(), query_string,
parsing_context, &ast_storage, &db_accessor);
// TODO: Maybe cache required privileges to improve performance on very
// simple queries.
auto required_privileges = query::GetRequiredPrivileges(ast_root);
auto frontend_time = frontend_timer.Elapsed();
// Build summary.
std::map<std::string, TypedValue> summary;
summary["parsing_time"] = frontend_time.count();
// TODO: set summary['type'] based on transaction metadata
// the type can't be determined based only on top level LogicalOp
// (for example MATCH DELETE RETURN will have Produce as it's top).
// For now always use "rw" because something must be set, but it doesn't
// have to be correct (for Bolt clients).
summary["type"] = "rw";
utils::Timer planning_timer;
// This local shared_ptr might be the only owner of the CachedPlan, so
// we must ensure it lives during the whole interpretation.
std::shared_ptr<CachedPlan> plan{nullptr};
if (auto *cypher_query = dynamic_cast<CypherQuery *>(ast_root)) {
plan = CypherQueryToPlan(stripped_query.hash(), cypher_query,
std::move(ast_storage),
evaluation_context.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_);
std::vector<std::string> header;
for (const auto &symbol : output_symbols) {
// When the symbol is aliased or expanded from '*' (inside RETURN or
// WITH), then there is no token position, so use symbol name.
// Otherwise, find the name from stripped query.
header.push_back(utils::FindOr(stripped_query.named_expressions(),
symbol.token_position(), symbol.name())
.first);
}
auto cursor = plan->plan().MakeCursor(db_accessor);
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, header, summary, required_privileges);
}
if (auto *explain_query = dynamic_cast<ExplainQuery *>(ast_root)) {
const std::string kExplainQueryStart = "explain ";
CHECK(utils::StartsWith(stripped_query.query(), kExplainQueryStart))
<< "Expected stripped query to start with '" << kExplainQueryStart
<< "'";
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::stringstream printed_plan;
PrettyPrintPlan(db_accessor, &cypher_query_plan->plan(), &printed_plan);
std::vector<std::vector<TypedValue>> printed_plan_rows;
for (const auto &row :
utils::Split(utils::RTrim(printed_plan.str()), "\n")) {
printed_plan_rows.push_back(std::vector<TypedValue>{row});
}
auto query_plan_symbol =
execution_context.symbol_table_.CreateSymbol("QUERY PLAN", false);
std::unique_ptr<plan::OutputTable> output_plan =
std::make_unique<plan::OutputTable>(
std::vector<Symbol>{query_plan_symbol}, printed_plan_rows);
plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
std::move(output_plan), 0.0, AstStorage{},
execution_context.symbol_table_));
auto planning_time = planning_timer.Elapsed();
summary["planning_time"] = planning_time.count();
execution_context.symbol_table_ = plan->symbol_table();
std::vector<Symbol> output_symbols{query_plan_symbol};
std::vector<std::string> header{query_plan_symbol.name()};
auto cursor = plan->plan().MakeCursor(db_accessor);
return Results(std::move(execution_context), plan, std::move(cursor),
output_symbols, header, summary, required_privileges);
}
Callback callback;
if (auto *index_query = dynamic_cast<IndexQuery *>(ast_root)) {
if (in_explicit_transaction) {
throw IndexInMulticommandTxException();
}
// Creating an index influences computed plan costs.
auto invalidate_plan_cache = [plan_cache = &this->plan_cache_] {
auto access = plan_cache->access();
for (auto &kv : access) {
access.remove(kv.first);
}
};
callback =
HandleIndexQuery(index_query, invalidate_plan_cache, &db_accessor);
} else if (auto *auth_query = dynamic_cast<AuthQuery *>(ast_root)) {
if (in_explicit_transaction) {
throw UserModificationInMulticommandTxException();
}
callback =
HandleAuthQuery(auth_query, auth_, evaluation_context, &db_accessor);
} else if (auto *stream_query = dynamic_cast<StreamQuery *>(ast_root)) {
if (in_explicit_transaction) {
throw StreamClauseInMulticommandTxException();
}
callback = HandleStreamQuery(stream_query, kafka_streams_,
evaluation_context, &db_accessor);
} else {
LOG(FATAL) << "Should not get here -- unknown query type!";
}
std::vector<Symbol> output_symbols;
for (const auto &column : callback.header) {
output_symbols.emplace_back(
execution_context.symbol_table_.CreateSymbol(column, "false"));
}
plan = std::make_shared<CachedPlan>(std::make_unique<SingleNodeLogicalPlan>(
std::make_unique<plan::OutputTable>(output_symbols, callback.fn), 0.0,
AstStorage{}, execution_context.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, required_privileges);
}
std::shared_ptr<Interpreter::CachedPlan> Interpreter::CypherQueryToPlan(
HashType query_hash, CypherQuery *query, AstStorage ast_storage,
const Parameters &parameters, database::GraphDbAccessor *db_accessor) {
auto plan_cache_access = plan_cache_.access();
auto it = plan_cache_access.find(query_hash);
if (it != plan_cache_access.end()) {
if (it->second->IsExpired()) {
plan_cache_access.remove(query_hash);
} else {
return it->second;
}
}
return plan_cache_access
.insert(query_hash,
std::make_shared<CachedPlan>(MakeLogicalPlan(
query, std::move(ast_storage), parameters, db_accessor)))
.first->second;
}
Query *Interpreter::ParseQuery(const std::string &stripped_query,
const std::string &original_query,
const ParsingContext &context,
AstStorage *ast_storage,
database::GraphDbAccessor *db_accessor) {
if (!context.is_query_cached) {
// Parse original query into antlr4 AST.
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<utils::SpinLock> guard(antlr_lock_);
return std::make_unique<frontend::opencypher::Parser>(original_query);
}();
// Convert antlr4 AST into Memgraph AST.
frontend::CypherMainVisitor visitor(context, ast_storage, db_accessor);
visitor.visit(parser->tree());
return visitor.query();
}
auto stripped_query_hash = fnv(stripped_query);
auto ast_cache_accessor = ast_cache_.access();
auto ast_it = ast_cache_accessor.find(stripped_query_hash);
if (ast_it == ast_cache_accessor.end()) {
// Parse stripped query into antlr4 AST.
auto parser = [&] {
// Be careful about unlocking since parser can throw.
std::unique_lock<utils::SpinLock> guard(antlr_lock_);
try {
return std::make_unique<frontend::opencypher::Parser>(stripped_query);
} catch (const SyntaxException &e) {
// There is syntax exception in stripped query. Rerun parser on
// the original query to get appropriate error messsage.
auto parser =
std::make_unique<frontend::opencypher::Parser>(original_query);
// If exception was not thrown here, StrippedQuery messed
// something up.
LOG(FATAL) << "Stripped query can't be parsed, but the original can.";
return parser;
}
}();
// Convert antlr4 AST into Memgraph AST.
CachedQuery cached_query;
frontend::CypherMainVisitor visitor(context, &cached_query.ast_storage,
db_accessor);
visitor.visit(parser->tree());
cached_query.query = visitor.query();
// Cache it.
ast_it =
ast_cache_accessor.insert(stripped_query_hash, std::move(cached_query))
.first;
}
return ast_it->second.query->Clone(*ast_storage);
}
std::unique_ptr<LogicalPlan> Interpreter::MakeLogicalPlan(
Query *query, AstStorage ast_storage, Context *context) {
auto vertex_counts = plan::MakeVertexCountCache(context->db_accessor_);
auto planning_context = plan::MakePlanningContext(
ast_storage, context->symbol_table_, query, vertex_counts);
CypherQuery *query, AstStorage ast_storage, const Parameters &parameters,
database::GraphDbAccessor *db_accessor) {
auto vertex_counts = plan::MakeVertexCountCache(*db_accessor);
SymbolTable symbol_table;
SymbolGenerator symbol_generator(symbol_table);
query->Accept(symbol_generator);
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, context->evaluation_context_.parameters,
FLAGS_query_cost_planner);
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), context->symbol_table_);
std::move(root), cost, std::move(ast_storage), std::move(symbol_table));
}
} // namespace query

View File

@ -74,7 +74,7 @@ class Interpreter {
Results(Context ctx, std::shared_ptr<CachedPlan> plan,
std::unique_ptr<query::plan::Cursor> cursor,
std::vector<Symbol> output_symbols, std::vector<std::string> header,
std::map<std::string, TypedValue> summary, PlanCacheT &plan_cache,
std::map<std::string, TypedValue> summary,
std::vector<AuthQuery::Privilege> privileges)
: ctx_(std::move(ctx)),
plan_(plan),
@ -83,7 +83,6 @@ class Interpreter {
output_symbols_(output_symbols),
header_(header),
summary_(summary),
plan_cache_(plan_cache),
privileges_(std::move(privileges)) {}
public:
@ -118,13 +117,6 @@ class Interpreter {
if (!return_value) {
summary_["plan_execution_time"] = execution_time_;
if (ctx_.is_index_created_) {
auto access = plan_cache_.access();
for (auto &kv : access) {
access.remove(kv.first);
}
}
cursor_->Shutdown();
}
@ -155,8 +147,6 @@ class Interpreter {
std::map<std::string, TypedValue> summary_;
double execution_time_{0};
// Gets invalidated after if an index has been built.
PlanCacheT &plan_cache_;
std::vector<AuthQuery::Privilege> privileges_;
};
@ -185,8 +175,12 @@ class Interpreter {
// high level tree -> logical plan
// AstStorage and SymbolTable may be modified during planning. The created
// LogicalPlan must take ownership of AstStorage and SymbolTable.
virtual std::unique_ptr<LogicalPlan> MakeLogicalPlan(Query *, AstStorage,
Context *);
virtual std::unique_ptr<LogicalPlan> MakeLogicalPlan(
CypherQuery *, AstStorage, const Parameters &,
database::GraphDbAccessor *);
virtual void PrettyPrintPlan(const database::GraphDbAccessor &,
const plan::LogicalOperator *, std::ostream *);
private:
ConcurrentMap<HashType, CachedQuery> ast_cache_;
@ -200,10 +194,13 @@ class Interpreter {
utils::SpinLock antlr_lock_;
// high level tree -> CachedPlan
std::shared_ptr<CachedPlan> AstToPlan(Query *query, AstStorage ast_storage,
Context *ctx);
std::shared_ptr<CachedPlan> CypherQueryToPlan(
HashType query_hash, CypherQuery *query, AstStorage ast_storage,
const Parameters &parameters, database::GraphDbAccessor *db_accessor);
// stripped query -> high level tree
Query *QueryToAst(const StrippedQuery &stripped,
Query *ParseQuery(const std::string &stripped_query,
const std::string &original_query,
const ParsingContext &context, AstStorage *ast_storage,
database::GraphDbAccessor *db_accessor);
};

View File

@ -184,14 +184,6 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
}
bool Visit(Once &) override { return true; }
bool Visit(CreateIndex &) override { return true; }
bool Visit(AuthHandler &) override { return true; }
bool Visit(CreateStream &) override { return true; }
bool Visit(DropStream &) override { return true; }
bool Visit(ShowStreams &) override { return true; }
bool Visit(StartStopStream &) override { return true; }
bool Visit(StartStopAllStreams &) override { return true; }
bool Visit(TestStream &) override { return true; }
// TODO: Cost estimate PullRemote and ProduceRemote?

View File

@ -88,24 +88,6 @@ class IndependentSubtreeFinder : public DistributedOperatorVisitor {
// These don't use any symbols
bool Visit(Once &) override { return true; }
bool Visit(CreateIndex &) override { return true; }
bool Visit(AuthHandler &) override { return true; }
bool Visit(CreateStream &) override { return true; }
bool Visit(DropStream &) override { return true; }
bool Visit(ShowStreams &) override { return true; }
bool Visit(StartStopStream &) override { return true; }
bool Visit(StartStopAllStreams &) override { return true; }
bool Visit(TestStream &) override { return true; }
// Treat Explain as if the query is planned without it
bool PreVisit(Explain &explain) override {
prev_ops_.push_back(&explain);
return true;
}
bool PostVisit(Explain &explain) override {
prev_ops_.pop_back();
return true;
}
bool PreVisit(ScanAll &scan) override {
prev_ops_.push_back(&scan);
@ -1119,30 +1101,6 @@ class DistributedPlanner : public HierarchicalLogicalOperatorVisitor {
return true;
}
// Change the pretty printer for Explain, but otherwise treat it as if the
// query is planned without it
bool PreVisit(Explain &explain) override {
CHECK(prev_ops_.empty());
prev_ops_.push_back(&explain);
explain.pretty_print_ = [](const auto &dba, auto *root, auto *out) {
return DistributedPrettyPrint(dba, root, out);
};
return true;
}
bool PostVisit(Explain &explain) override {
// Set Explain as the final operator on master.
if (ShouldSplit()) {
auto input = explain.input();
auto pull_id = AddWorkerPlan(input);
Split(explain, std::make_shared<PullRemote>(
input, pull_id,
input->OutputSymbols(distributed_plan_.symbol_table)));
}
prev_ops_.pop_back();
return false;
}
// Skip needs to skip only the first N results from *all* of the results.
// Therefore, the earliest (deepest in the plan tree) encountered Skip will
// break the plan in 2 parts.
@ -1444,22 +1402,6 @@ class DistributedPlanner : public HierarchicalLogicalOperatorVisitor {
bool Visit(Once &) override { return true; }
bool Visit(CreateIndex &) override { return true; }
bool Visit(AuthHandler &) override { return true; }
bool Visit(CreateStream &) override { return true; }
bool Visit(DropStream &) override { return true; }
bool Visit(ShowStreams &) override { return true; }
bool Visit(StartStopStream &) override { return true; }
bool Visit(StartStopAllStreams &) override { return true; }
bool Visit(TestStream &) override { return true; }
// Accumulate is used only if the query performs any writes. In such a case,
// we need to synchronize the work done on master and all workers.
// Synchronization will force applying changes to distributed storage, and

View File

@ -69,9 +69,11 @@ bool DistributedPlanPrinter::PreVisit(query::plan::Synchronize &op) {
}
void DistributedPrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root, std::ostream *out) {
const LogicalOperator *plan_root,
std::ostream *out) {
DistributedPlanPrinter printer(&dba, out);
plan_root->Accept(printer);
// FIXME(mtomic): We should make visitors that take const argument.
const_cast<LogicalOperator *>(plan_root)->Accept(printer);
}
} // namespace query::plan

View File

@ -7,10 +7,11 @@
namespace query::plan {
void DistributedPrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root, std::ostream *out);
const LogicalOperator *plan_root,
std::ostream *out);
inline void DistributedPrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root) {
const LogicalOperator *plan_root) {
DistributedPrettyPrint(dba, plan_root, &std::cout);
}

View File

@ -3016,60 +3016,6 @@ void Distinct::DistinctCursor::Reset() {
seen_rows_.clear();
}
CreateIndex::CreateIndex(storage::Label label,
const std::vector<storage::Property> &properties,
bool is_unique)
: label_(label), properties_(properties), is_unique_(is_unique) {}
bool CreateIndex::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.Visit(*this);
}
WITHOUT_SINGLE_INPUT(CreateIndex);
class CreateIndexCursor : public Cursor {
public:
CreateIndexCursor(const CreateIndex &self, database::GraphDbAccessor &db)
: self_(self), db_(db) {}
bool Pull(Frame &, Context &ctx) override {
if (did_create_) return false;
if (ctx.in_explicit_transaction_) {
throw IndexInMulticommandTxException();
}
if (self_.properties_.size() > 1)
throw utils::NotYetImplemented("index on multiple properties");
try {
CHECK(self_.properties_.size() == 1U);
db_.BuildIndex(self_.label_, self_.properties_[0], self_.is_unique_);
} catch (const database::IndexConstraintViolationException &e) {
throw QueryRuntimeException(e.what());
} catch (const database::IndexExistsException &e) {
if (self_.is_unique_) {
throw QueryRuntimeException(e.what());
}
// Otherwise ignore creating an existing index.
}
ctx.is_index_created_ = did_create_ = true;
return true;
}
void Shutdown() override {}
void Reset() override { did_create_ = false; }
private:
const CreateIndex &self_;
database::GraphDbAccessor &db_;
bool did_create_ = false;
};
std::unique_ptr<Cursor> CreateIndex::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<CreateIndexCursor>(*this, db);
}
Union::Union(const std::shared_ptr<LogicalOperator> &left_op,
const std::shared_ptr<LogicalOperator> &right_op,
const std::vector<Symbol> &union_symbols,
@ -3247,795 +3193,60 @@ std::unique_ptr<Cursor> Cartesian::MakeCursor(
return std::make_unique<CartesianCursor>(*this, db);
}
AuthHandler::AuthHandler(AuthQuery::Action action, std::string user,
std::string role, std::string user_or_role,
Expression *password,
std::vector<AuthQuery::Privilege> privileges,
Symbol user_symbol, Symbol role_symbol,
Symbol privilege_symbol, Symbol effective_symbol,
Symbol details_symbol)
: action_(action),
user_(user),
role_(role),
user_or_role_(user_or_role),
password_(password),
privileges_(privileges),
user_symbol_(user_symbol),
role_symbol_(role_symbol),
privilege_symbol_(privilege_symbol),
effective_symbol_(effective_symbol),
details_symbol_(details_symbol) {}
OutputTable::OutputTable(std::vector<Symbol> output_symbols,
std::vector<std::vector<TypedValue>> rows)
: output_symbols_(std::move(output_symbols)),
callback_([rows]() { return rows; }) {}
bool AuthHandler::Accept(HierarchicalLogicalOperatorVisitor &visitor) {
return visitor.Visit(*this);
}
OutputTable::OutputTable(
std::vector<Symbol> output_symbols,
std::function<std::vector<std::vector<TypedValue>>()> callback)
: output_symbols_(std::move(output_symbols)),
callback_(std::move(callback)) {}
std::vector<Symbol> AuthHandler::OutputSymbols(const SymbolTable &) const {
switch (action_) {
case AuthQuery::Action::SHOW_USERS:
case AuthQuery::Action::SHOW_USERS_FOR_ROLE:
return {user_symbol_};
WITHOUT_SINGLE_INPUT(OutputTable);
case AuthQuery::Action::SHOW_ROLES:
case AuthQuery::Action::SHOW_ROLE_FOR_USER:
return {role_symbol_};
case AuthQuery::Action::SHOW_PRIVILEGES:
return {privilege_symbol_, effective_symbol_, details_symbol_};
case AuthQuery::Action::CREATE_USER:
case AuthQuery::Action::DROP_USER:
case AuthQuery::Action::SET_PASSWORD:
case AuthQuery::Action::CREATE_ROLE:
case AuthQuery::Action::DROP_ROLE:
case AuthQuery::Action::SET_ROLE:
case AuthQuery::Action::CLEAR_ROLE:
case AuthQuery::Action::GRANT_PRIVILEGE:
case AuthQuery::Action::DENY_PRIVILEGE:
case AuthQuery::Action::REVOKE_PRIVILEGE:
return {};
}
}
class AuthHandlerCursor : public Cursor {
class OutputTableCursor : public Cursor {
public:
AuthHandlerCursor(const AuthHandler &self) : self_(self) {}
OutputTableCursor(const OutputTable &self) : self_(self) {}
std::vector<auth::Permission> GetAuthPermissions() {
std::vector<auth::Permission> ret;
for (const auto &privilege : self_.privileges_) {
ret.push_back(glue::PrivilegeToPermission(privilege));
}
return ret;
}
std::vector<std::tuple<std::string, std::string, std::string>>
GetGrantsForAuthUser(const auth::User &user) {
std::vector<std::tuple<std::string, std::string, std::string>> ret;
const auto &permissions = user.GetPermissions();
for (const auto &privilege : kPrivilegesAll) {
auto permission = glue::PrivilegeToPermission(privilege);
auto effective = permissions.Has(permission);
if (permissions.Has(permission) != auth::PermissionLevel::NEUTRAL) {
std::vector<std::string> description;
auto user_level = user.permissions().Has(permission);
if (user_level == auth::PermissionLevel::GRANT) {
description.push_back("GRANTED TO USER");
} else if (user_level == auth::PermissionLevel::DENY) {
description.push_back("DENIED TO USER");
}
if (user.role()) {
auto role_level = user.role()->permissions().Has(permission);
if (role_level == auth::PermissionLevel::GRANT) {
description.push_back("GRANTED TO ROLE");
} else if (role_level == auth::PermissionLevel::DENY) {
description.push_back("DENIED TO ROLE");
}
}
ret.push_back({auth::PermissionToString(permission),
auth::PermissionLevelToString(effective),
utils::Join(description, ", ")});
bool Pull(Frame &frame, Context &context) override {
if (!pulled_) {
rows_ = self_.callback_();
for (const auto &row : rows_) {
CHECK(row.size() == self_.output_symbols_.size())
<< "Wrong number of columns in row!";
}
pulled_ = true;
}
return ret;
}
std::vector<std::tuple<std::string, std::string, std::string>>
GetGrantsForAuthRole(const auth::Role &role) {
std::vector<std::tuple<std::string, std::string, std::string>> ret;
const auto &permissions = role.permissions();
for (const auto &privilege : kPrivilegesAll) {
auto permission = glue::PrivilegeToPermission(privilege);
auto effective = permissions.Has(permission);
if (effective != auth::PermissionLevel::NEUTRAL) {
std::string description;
if (effective == auth::PermissionLevel::GRANT) {
description = "GRANTED TO ROLE";
} else if (effective == auth::PermissionLevel::DENY) {
description = "DENIED TO ROLE";
}
ret.push_back({auth::PermissionToString(permission),
auth::PermissionLevelToString(effective), description});
if (current_row_ < rows_.size()) {
for (size_t i = 0; i < self_.output_symbols_.size(); ++i) {
frame[self_.output_symbols_[i]] = rows_[current_row_][i];
}
}
return ret;
}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw UserModificationInMulticommandTxException();
}
ExpressionEvaluator evaluator(&frame, ctx.symbol_table_,
ctx.evaluation_context_, &ctx.db_accessor_,
GraphView::OLD);
std::experimental::optional<std::string> password;
if (self_.password_) {
auto password_tv = self_.password_->Accept(evaluator);
if (!password_tv.IsString() && !password_tv.IsNull()) {
throw QueryRuntimeException(
"Expected string or null for password, got {}.",
password_tv.type());
}
if (password_tv.IsString()) {
password = password_tv.ValueString();
}
}
auto &auth = *ctx.auth_;
switch (self_.action_) {
case AuthQuery::Action::CREATE_USER: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.AddUser(self_.user_, password);
if (!user) {
throw QueryRuntimeException("User or role '{}' already exists.",
self_.user_);
}
return false;
}
case AuthQuery::Action::DROP_USER: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", self_.user_);
}
if (!auth.RemoveUser(self_.user_)) {
throw QueryRuntimeException("Couldn't remove user '{}'.",
self_.user_);
}
return false;
}
case AuthQuery::Action::SET_PASSWORD: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", self_.user_);
}
user->UpdatePassword(password);
auth.SaveUser(*user);
return false;
}
case AuthQuery::Action::CREATE_ROLE: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto role = auth.AddRole(self_.role_);
if (!role) {
throw QueryRuntimeException("User or role '{}' already exists.",
self_.role_);
}
return false;
}
case AuthQuery::Action::DROP_ROLE: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto role = auth.GetRole(self_.role_);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist.", self_.role_);
}
if (!auth.RemoveRole(self_.role_)) {
throw QueryRuntimeException("Couldn't remove role '{}'.",
self_.role_);
}
return false;
}
case AuthQuery::Action::SHOW_USERS: {
if (!users_) {
std::lock_guard<std::mutex> lock(auth.WithLock());
users_.emplace(auth.AllUsers());
users_it_ = users_->begin();
}
if (users_it_ == users_->end()) return false;
frame[self_.user_symbol_] = users_it_->username();
users_it_++;
return true;
}
case AuthQuery::Action::SHOW_ROLES: {
if (!roles_) {
std::lock_guard<std::mutex> lock(auth.WithLock());
roles_.emplace(auth.AllRoles());
roles_it_ = roles_->begin();
}
if (roles_it_ == roles_->end()) return false;
frame[self_.role_symbol_] = roles_it_->rolename();
roles_it_++;
return true;
}
case AuthQuery::Action::SET_ROLE: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", self_.user_);
}
auto role = auth.GetRole(self_.role_);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist.", self_.role_);
}
if (user->role()) {
throw QueryRuntimeException(
"User '{}' is already a member of role '{}'.", self_.user_,
user->role()->rolename());
}
user->SetRole(*role);
auth.SaveUser(*user);
return false;
}
case AuthQuery::Action::CLEAR_ROLE: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", self_.user_);
}
user->ClearRole();
auth.SaveUser(*user);
return false;
}
case AuthQuery::Action::GRANT_PRIVILEGE:
case AuthQuery::Action::DENY_PRIVILEGE:
case AuthQuery::Action::REVOKE_PRIVILEGE: {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_or_role_);
auto role = auth.GetRole(self_.user_or_role_);
if (!user && !role) {
throw QueryRuntimeException("User or role '{}' doesn't exist.",
self_.user_or_role_);
}
auto permissions = GetAuthPermissions();
if (user) {
for (const auto &permission : permissions) {
// TODO (mferencevic): should we first check that the privilege
// is granted/denied/revoked before unconditionally
// granting/denying/revoking it?
if (self_.action_ == AuthQuery::Action::GRANT_PRIVILEGE) {
user->permissions().Grant(permission);
} else if (self_.action_ == AuthQuery::Action::DENY_PRIVILEGE) {
user->permissions().Deny(permission);
} else {
user->permissions().Revoke(permission);
}
}
auth.SaveUser(*user);
} else {
for (const auto &permission : permissions) {
// TODO (mferencevic): should we first check that the privilege
// is granted/denied/revoked before unconditionally
// granting/denying/revoking it?
if (self_.action_ == AuthQuery::Action::GRANT_PRIVILEGE) {
role->permissions().Grant(permission);
} else if (self_.action_ == AuthQuery::Action::DENY_PRIVILEGE) {
role->permissions().Deny(permission);
} else {
role->permissions().Revoke(permission);
}
}
auth.SaveRole(*role);
}
return false;
}
case AuthQuery::Action::SHOW_PRIVILEGES: {
if (!grants_) {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_or_role_);
auto role = auth.GetRole(self_.user_or_role_);
if (!user && !role) {
throw QueryRuntimeException("User or role '{}' doesn't exist.",
self_.user_or_role_);
}
if (user) {
grants_.emplace(GetGrantsForAuthUser(*user));
} else {
grants_.emplace(GetGrantsForAuthRole(*role));
}
grants_it_ = grants_->begin();
}
if (grants_it_ == grants_->end()) return false;
frame[self_.privilege_symbol_] = std::get<0>(*grants_it_);
frame[self_.effective_symbol_] = std::get<1>(*grants_it_);
frame[self_.details_symbol_] = std::get<2>(*grants_it_);
grants_it_++;
return true;
}
case AuthQuery::Action::SHOW_ROLE_FOR_USER: {
if (returned_role_for_user_) return false;
std::lock_guard<std::mutex> lock(auth.WithLock());
auto user = auth.GetUser(self_.user_);
if (!user) {
throw QueryRuntimeException("User '{}' doesn't exist.", self_.user_);
}
if (user->role()) {
frame[self_.role_symbol_] = user->role()->rolename();
} else {
frame[self_.role_symbol_] = TypedValue::Null;
}
returned_role_for_user_ = true;
return true;
}
case AuthQuery::Action::SHOW_USERS_FOR_ROLE: {
if (!users_) {
std::lock_guard<std::mutex> lock(auth.WithLock());
auto role = auth.GetRole(self_.role_);
if (!role) {
throw QueryRuntimeException("Role '{}' doesn't exist.",
self_.role_);
}
users_.emplace(auth.AllUsersForRole(self_.role_));
users_it_ = users_->begin();
}
if (users_it_ == users_->end()) return false;
frame[self_.user_symbol_] = users_it_->username();
users_it_++;
return true;
}
}
}
void Shutdown() override {}
void Reset() override {
LOG(FATAL) << "AuthHandler cursor should never be reset";
}
private:
const AuthHandler &self_;
std::experimental::optional<std::vector<auth::User>> users_;
std::vector<auth::User>::iterator users_it_;
std::experimental::optional<std::vector<auth::Role>> roles_;
std::vector<auth::Role>::iterator roles_it_;
std::experimental::optional<
std::vector<std::tuple<std::string, std::string, std::string>>>
grants_;
std::vector<std::tuple<std::string, std::string, std::string>>::iterator
grants_it_;
bool returned_role_for_user_{false};
};
std::unique_ptr<Cursor> AuthHandler::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<AuthHandlerCursor>(*this);
}
WITHOUT_SINGLE_INPUT(AuthHandler)
CreateStream::CreateStream(std::string stream_name, Expression *stream_uri,
Expression *stream_topic, Expression *transform_uri,
Expression *batch_interval_in_ms,
Expression *batch_size)
: stream_name_(std::move(stream_name)),
stream_uri_(stream_uri),
stream_topic_(stream_topic),
transform_uri_(transform_uri),
batch_interval_in_ms_(batch_interval_in_ms),
batch_size_(batch_size) {}
WITHOUT_SINGLE_INPUT(CreateStream)
class CreateStreamCursor : public Cursor {
using StreamInfo = integrations::kafka::StreamInfo;
public:
CreateStreamCursor(const CreateStream &self, database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
ExpressionEvaluator evaluator(&frame, ctx.symbol_table_,
ctx.evaluation_context_, &ctx.db_accessor_,
GraphView::OLD);
TypedValue stream_uri = self_.stream_uri_->Accept(evaluator);
TypedValue stream_topic = self_.stream_topic_->Accept(evaluator);
TypedValue transform_uri = self_.transform_uri_->Accept(evaluator);
std::experimental::optional<int64_t> batch_interval_in_ms, batch_size;
if (self_.batch_interval_in_ms_) {
batch_interval_in_ms =
self_.batch_interval_in_ms_->Accept(evaluator).Value<int64_t>();
}
if (self_.batch_size_) {
batch_size = self_.batch_size_->Accept(evaluator).Value<int64_t>();
}
try {
StreamInfo info;
info.stream_name = self_.stream_name_;
info.stream_uri = stream_uri.Value<std::string>();
info.stream_topic = stream_topic.Value<std::string>();
info.transform_uri = transform_uri.Value<std::string>();
info.batch_interval_in_ms = batch_interval_in_ms;
info.batch_size = batch_size;
ctx.kafka_streams_->Create(info);
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return false;
}
void Shutdown() override {}
void Reset() override { throw utils::NotYetImplemented("Create Stream"); }
private:
const CreateStream &self_;
};
std::unique_ptr<Cursor> CreateStream::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<CreateStreamCursor>(*this, db);
}
DropStream::DropStream(std::string stream_name)
: stream_name_(std::move(stream_name)) {}
WITHOUT_SINGLE_INPUT(DropStream)
class DropStreamCursor : public Cursor {
public:
DropStreamCursor(const DropStream &self, database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
try {
ctx.kafka_streams_->Drop(self_.stream_name_);
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return false;
}
void Shutdown() override {}
void Reset() override { throw utils::NotYetImplemented("Drop Stream"); }
private:
const DropStream &self_;
};
std::unique_ptr<Cursor> DropStream::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<DropStreamCursor>(*this, db);
}
ShowStreams::ShowStreams(Symbol name_symbol, Symbol uri_symbol,
Symbol topic_symbol, Symbol transform_symbol,
Symbol status_symbol)
: name_symbol_(name_symbol),
uri_symbol_(uri_symbol),
topic_symbol_(topic_symbol),
transform_symbol_(transform_symbol),
status_symbol_(status_symbol) {}
WITHOUT_SINGLE_INPUT(ShowStreams)
std::vector<Symbol> ShowStreams::OutputSymbols(const SymbolTable &) const {
return {name_symbol_, uri_symbol_, topic_symbol_, transform_symbol_,
status_symbol_};
}
class ShowStreamsCursor : public Cursor {
public:
ShowStreamsCursor(const ShowStreams &self, database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
if (!is_initialized_) {
streams_ = ctx.kafka_streams_->Show();
streams_it_ = streams_.begin();
is_initialized_ = true;
}
if (streams_it_ == streams_.end()) return false;
frame[self_.name_symbol_] = streams_it_->stream_name;
frame[self_.uri_symbol_] = streams_it_->stream_uri;
frame[self_.topic_symbol_] = streams_it_->stream_topic;
frame[self_.transform_symbol_] = streams_it_->transform_uri;
frame[self_.status_symbol_] = streams_it_->stream_status;
streams_it_++;
return true;
}
void Shutdown() override {}
void Reset() override { throw utils::NotYetImplemented("Show Streams"); }
private:
const ShowStreams &self_;
bool is_initialized_ = false;
using StreamStatus = integrations::kafka::StreamStatus;
std::vector<StreamStatus> streams_;
std::vector<StreamStatus>::iterator streams_it_ = streams_.begin();
};
std::unique_ptr<Cursor> ShowStreams::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<ShowStreamsCursor>(*this, db);
}
StartStopStream::StartStopStream(std::string stream_name, bool is_start,
Expression *limit_batches)
: stream_name_(stream_name),
is_start_(is_start),
limit_batches_(limit_batches) {}
WITHOUT_SINGLE_INPUT(StartStopStream)
class StartStopStreamCursor : public Cursor {
public:
StartStopStreamCursor(const StartStopStream &self,
database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
ExpressionEvaluator evaluator(&frame, ctx.symbol_table_,
ctx.evaluation_context_, &ctx.db_accessor_,
GraphView::OLD);
std::experimental::optional<int64_t> limit_batches;
if (self_.limit_batches_) {
limit_batches = self_.limit_batches_->Accept(evaluator).Value<int64_t>();
}
try {
if (self_.is_start_) {
ctx.kafka_streams_->Start(self_.stream_name_, limit_batches);
} else {
ctx.kafka_streams_->Stop(self_.stream_name_);
}
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return false;
}
void Shutdown() override {}
void Reset() override { throw utils::NotYetImplemented("Start/Stop Stream"); }
private:
const StartStopStream &self_;
};
std::unique_ptr<Cursor> StartStopStream::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<StartStopStreamCursor>(*this, db);
}
StartStopAllStreams::StartStopAllStreams(bool is_start) : is_start_(is_start) {}
WITHOUT_SINGLE_INPUT(StartStopAllStreams)
class StartStopAllStreamsCursor : public Cursor {
public:
StartStopAllStreamsCursor(const StartStopAllStreams &self,
database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
try {
if (self_.is_start_) {
ctx.kafka_streams_->StartAll();
} else {
ctx.kafka_streams_->StopAll();
}
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
return false;
}
void Shutdown() override {}
void Reset() override {
throw utils::NotYetImplemented("Start/Stop All Streams");
}
private:
const StartStopAllStreams &self_;
};
std::unique_ptr<Cursor> StartStopAllStreams::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<StartStopAllStreamsCursor>(*this, db);
}
TestStream::TestStream(std::string stream_name, Expression *limit_batches,
Symbol query_symbol, Symbol params_symbol)
: stream_name_(stream_name),
limit_batches_(limit_batches),
query_symbol_(query_symbol),
params_symbol_(params_symbol) {}
WITHOUT_SINGLE_INPUT(TestStream)
std::vector<Symbol> TestStream::OutputSymbols(const SymbolTable &) const {
return {query_symbol_, params_symbol_};
}
class TestStreamCursor : public Cursor {
public:
TestStreamCursor(const TestStream &self, database::GraphDbAccessor &)
: self_(self) {}
bool Pull(Frame &frame, Context &ctx) override {
if (ctx.in_explicit_transaction_) {
throw StreamClauseInMulticommandTxException();
}
if (!is_initialized_) {
ExpressionEvaluator evaluator(&frame, ctx.symbol_table_,
ctx.evaluation_context_, &ctx.db_accessor_,
GraphView::OLD);
std::experimental::optional<int64_t> limit_batches;
if (self_.limit_batches_) {
limit_batches =
self_.limit_batches_->Accept(evaluator).Value<int64_t>();
}
try {
auto results =
ctx.kafka_streams_->Test(self_.stream_name_, limit_batches);
for (const auto &result : results) {
std::map<std::string, query::TypedValue> params_tv;
for (const auto &kv : result.second) {
params_tv.emplace(kv.first, glue::ToTypedValue(kv.second));
}
results_.emplace_back(result.first, params_tv);
}
} catch (const integrations::kafka::KafkaStreamException &e) {
throw QueryRuntimeException(e.what());
}
results_it_ = results_.begin();
is_initialized_ = true;
}
if (results_it_ == results_.end()) return false;
frame[self_.query_symbol_] = results_it_->first;
frame[self_.params_symbol_] = results_it_->second;
results_it_++;
return true;
}
void Shutdown() override {}
void Reset() override { throw utils::NotYetImplemented("Test Stream"); }
private:
const TestStream &self_;
bool is_initialized_ = false;
std::vector<std::pair<std::string, TypedValue>> results_;
std::vector<std::pair<std::string, TypedValue>>::iterator results_it_ =
results_.begin();
};
std::unique_ptr<Cursor> TestStream::MakeCursor(
database::GraphDbAccessor &db) const {
return std::make_unique<TestStreamCursor>(*this, db);
}
Explain::Explain(
const std::shared_ptr<LogicalOperator> &input, const Symbol &output_symbol,
const std::function<void(const database::GraphDbAccessor &,
LogicalOperator *, std::ostream *)> &pretty_print)
: input_(input),
output_symbol_(output_symbol),
pretty_print_(pretty_print) {}
ACCEPT_WITH_INPUT(Explain);
std::vector<Symbol> Explain::OutputSymbols(const SymbolTable &) const {
return {output_symbol_};
}
std::vector<Symbol> Explain::ModifiedSymbols(const SymbolTable &table) const {
return OutputSymbols(table);
}
class ExplainCursor : public Cursor {
public:
ExplainCursor(const Explain &self, const database::GraphDbAccessor &dba,
const Symbol &output_symbol)
: printed_plan_rows_([&dba, &self]() {
std::stringstream stream;
self.pretty_print_(dba, self.input().get(), &stream);
return utils::Split(utils::RTrim(stream.str()), "\n");
}()),
print_it_(printed_plan_rows_.begin()),
output_symbol_(output_symbol) {}
bool Pull(Frame &frame, Context &ctx) override {
if (print_it_ != printed_plan_rows_.end()) {
frame[output_symbol_] = *print_it_;
print_it_++;
current_row_++;
return true;
}
return false;
}
void Reset() override {
pulled_ = false;
current_row_ = 0;
rows_.clear();
}
void Shutdown() override {}
void Reset() override { print_it_ = printed_plan_rows_.begin(); }
private:
std::vector<std::string> printed_plan_rows_;
std::vector<std::string>::iterator print_it_;
Symbol output_symbol_;
const OutputTable &self_;
size_t current_row_{0};
std::vector<std::vector<TypedValue>> rows_;
bool pulled_{false};
};
std::unique_ptr<Cursor> Explain::MakeCursor(
std::unique_ptr<Cursor> OutputTable::MakeCursor(
database::GraphDbAccessor &dba) const {
return std::make_unique<ExplainCursor>(*this, dba, output_symbol_);
return std::make_unique<OutputTableCursor>(*this);
}
} // namespace query::plan

View File

@ -94,17 +94,8 @@ class Merge;
class Optional;
class Unwind;
class Distinct;
class CreateIndex;
class Union;
class Cartesian;
class AuthHandler;
class CreateStream;
class DropStream;
class ShowStreams;
class StartStopStream;
class StartStopAllStreams;
class TestStream;
class Explain;
using LogicalOperatorCompositeVisitor = ::utils::CompositeVisitor<
Once, CreateNode, CreateExpand, ScanAll, ScanAllByLabel,
@ -113,12 +104,9 @@ using LogicalOperatorCompositeVisitor = ::utils::CompositeVisitor<
SetProperties, SetLabels, RemoveProperty, RemoveLabels,
ExpandUniquenessFilter<VertexAccessor>,
ExpandUniquenessFilter<EdgeAccessor>, Accumulate, Aggregate, Skip, Limit,
OrderBy, Merge, Optional, Unwind, Distinct, Union, Cartesian, Explain>;
OrderBy, Merge, Optional, Unwind, Distinct, Union, Cartesian>;
using LogicalOperatorLeafVisitor =
::utils::LeafVisitor<Once, CreateIndex, AuthHandler, CreateStream,
DropStream, ShowStreams, StartStopStream,
StartStopAllStreams, TestStream>;
using LogicalOperatorLeafVisitor = ::utils::LeafVisitor<Once>;
/**
* @brief Base class for hierarhical visitors of @c LogicalOperator class
@ -2003,137 +1991,6 @@ and returns true, once.")
cpp<#)
(:serialize :capnp))
(lcp:define-class auth-handler (logical-operator)
((action "AuthQuery::Action" :scope :public
:capnp-init nil
:capnp-type "Ast.AuthQuery.Action"
:capnp-save (lcp:capnp-save-enum "::query::capnp::AuthQuery::Action"
"AuthQuery::Action"
'(create-role drop-role show-roles
create-user set-password
drop-user show-users set-role
clear-role grant-privilege
deny-privilege revoke-privilege
show-privileges show-role-for-user
show-users-for-role))
:capnp-load (lcp:capnp-load-enum "::query::capnp::AuthQuery::Action"
"AuthQuery::Action"
'(create-role drop-role show-roles
create-user set-password
drop-user show-users set-role
clear-role grant-privilege
deny-privilege revoke-privilege
show-privileges show-role-for-user
show-users-for-role)))
(user "std::string" :scope :public)
(role "std::string" :scope :public)
(user-or-role "std::string" :scope :public)
(password "Expression *" :scope :public
:capnp-type "Ast.Tree" :capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(privileges "std::vector<AuthQuery::Privilege>" :scope :public
:capnp-type "List(Ast.AuthQuery.Privilege)"
:capnp-save
(lambda (builder member capnp-name)
(declare (ignore capnp-name))
#>cpp
for (size_t i = 0; i < ${member}.size(); ++i) {
switch (${member}[i]) {
case AuthQuery::Privilege::CREATE:
${builder}.set(i, query::capnp::AuthQuery::Privilege::CREATE);
break;
case AuthQuery::Privilege::DELETE:
${builder}.set(i, query::capnp::AuthQuery::Privilege::DELETE);
break;
case AuthQuery::Privilege::MATCH:
${builder}.set(i, query::capnp::AuthQuery::Privilege::MATCH);
break;
case AuthQuery::Privilege::MERGE:
${builder}.set(i, query::capnp::AuthQuery::Privilege::MERGE);
break;
case AuthQuery::Privilege::SET:
${builder}.set(i, query::capnp::AuthQuery::Privilege::SET);
break;
case AuthQuery::Privilege::REMOVE:
${builder}.set(i, query::capnp::AuthQuery::Privilege::REMOVE);
break;
case AuthQuery::Privilege::INDEX:
${builder}.set(i, query::capnp::AuthQuery::Privilege::INDEX);
break;
case AuthQuery::Privilege::AUTH:
${builder}.set(i, query::capnp::AuthQuery::Privilege::AUTH);
break;
case AuthQuery::Privilege::STREAM:
${builder}.set(i, query::capnp::AuthQuery::Privilege::STREAM);
break;
}
}
cpp<#)
:capnp-load
(lambda (reader member-name capnp-name)
(declare (ignore capnp-name))
#>cpp
for (auto privilege : ${reader}) {
switch (privilege) {
case query::capnp::AuthQuery::Privilege::CREATE:
${member-name}.push_back(AuthQuery::Privilege::CREATE);
break;
case query::capnp::AuthQuery::Privilege::DELETE:
${member-name}.push_back(AuthQuery::Privilege::DELETE);
break;
case query::capnp::AuthQuery::Privilege::MATCH:
${member-name}.push_back(AuthQuery::Privilege::MATCH);
break;
case query::capnp::AuthQuery::Privilege::MERGE:
${member-name}.push_back(AuthQuery::Privilege::MERGE);
break;
case query::capnp::AuthQuery::Privilege::SET:
${member-name}.push_back(AuthQuery::Privilege::SET);
break;
case query::capnp::AuthQuery::Privilege::REMOVE:
${member-name}.push_back(AuthQuery::Privilege::REMOVE);
break;
case query::capnp::AuthQuery::Privilege::INDEX:
${member-name}.push_back(AuthQuery::Privilege::INDEX);
break;
case query::capnp::AuthQuery::Privilege::AUTH:
${member-name}.push_back(AuthQuery::Privilege::AUTH);
break;
case query::capnp::AuthQuery::Privilege::STREAM:
${member-name}.push_back(AuthQuery::Privilege::STREAM);
break;
}
}
cpp<#))
(user-symbol "Symbol" :scope :public)
(role-symbol "Symbol" :scope :public)
(privilege-symbol "Symbol" :scope :public)
(effective-symbol "Symbol" :scope :public)
(details-symbol "Symbol" :scope :public))
(:public
#>cpp
AuthHandler() {}
AuthHandler(AuthQuery::Action action, std::string user, std::string role,
std::string user_or_role, Expression * password,
std::vector<AuthQuery::Privilege> privileges,
Symbol user_symbol, Symbol role_symbol,
Symbol privilege_symbol, Symbol effective_symbol,
Symbol details_symbol);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(database::GraphDbAccessor & db)
const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
virtual std::vector<Symbol> ModifiedSymbols(const SymbolTable &)
const override { return {}; }
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class unwind (logical-operator)
((input "std::shared_ptr<LogicalOperator>" :scope :public
:capnp-save #'save-operator-pointer
@ -2244,38 +2101,6 @@ This implementation maintains input ordering.")
cpp<#)
(:serialize :capnp))
(lcp:define-class create-index (logical-operator)
((label "storage::Label" :scope :public)
(properties "std::vector<storage::Property>" :scope :public
:capnp-save (lcp:capnp-save-vector "storage::capnp::Common" "storage::Property")
:capnp-load (lcp:capnp-load-vector "storage::capnp::Common" "storage::Property"))
(is-unique :bool :scope :public))
(:documentation
"Creates an index for a combination of label and a property.
This operator takes no input and it shouldn't serve as an input to any
operator. Pulling from the cursor of this operator will create an index in
the database for the vertices which have the given label and property. In
case the index already exists, nothing happens. When is_unique is set to true,
then a unique index will be created instead of a regular one.")
(:public
#>cpp
CreateIndex() {}
CreateIndex(storage::Label label, const std::vector<storage::Property> &properties,
bool is_unique);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class union (logical-operator)
((left-op "std::shared_ptr<LogicalOperator>" :scope :public
:capnp-save #'save-operator-pointer
@ -2374,225 +2199,37 @@ vectors of symbols used by each of the inputs.")
cpp<#)
(:serialize :capnp))
;;; KAFKA STREAM OPERATORS
(lcp:define-class create-stream (logical-operator)
((stream-name "std::string" :scope :public)
(stream-uri "Expression *"
(lcp:define-class output-table (logical-operator)
((output-symbols "std::vector<Symbol>" :scope :public
:capnp-save :dont-save)
(callback "std::function<std::vector<std::vector<TypedValue>>()>"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(stream-topic "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(transform-uri "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(batch-interval-in-ms "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(batch-size "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
(:documentation
"Creates a new stream with given parameters that imports data.")
:capnp-save :dont-save))
(:documentation "An operator that outputs a table, producing a single row on each pull")
(:public
#>cpp
CreateStream() {}
CreateStream(std::string stream_name, Expression *stream_uri,
Expression *stream_topic, Expression *transform_uri,
Expression *batch_interval_in_ms, Expression *batch_size);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
OutputTable() {}
OutputTable(std::vector<Symbol> output_symbols,
std::function<std::vector<std::vector<TypedValue>>()> callback);
OutputTable(std::vector<Symbol> output_symbols,
std::vector<std::vector<TypedValue>> rows);
bool Accept(HierarchicalLogicalOperatorVisitor &) override {
LOG(FATAL) << "OutputTable operator should not be visited!";
}
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
database::GraphDbAccessor &) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override {
return output_symbols_;
}
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
return output_symbols_;
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class drop-stream (logical-operator)
((stream-name "std::string" :scope :public))
(:documentation
"Stops and deletes a stream if it exists.")
(:public
#>cpp
DropStream() {}
explicit DropStream(std::string stream_name);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class show-streams (logical-operator)
((name-symbol "Symbol" :scope :public)
(uri-symbol "Symbol" :scope :public)
(topic-symbol "Symbol" :scope :public)
(transform-symbol "Symbol" :scope :public)
(status-symbol "Symbol" :scope :public))
(:documentation
"Shows all streams, started and stopped, that were configured.")
(:public
#>cpp
ShowStreams() {}
ShowStreams(Symbol name_symbol, Symbol endpoint_symbol, Symbol topic_symbol,
Symbol transform_symbol, Symbol status_symbol);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class start-stop-stream (logical-operator)
((stream-name "std::string" :scope :public)
(is-start :bool :scope :public)
(limit-batches "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *")))
(:documentation
"Starts or stops a stream.
When starting a stream import, it can be limited by number of batches
If a stream is already running, it wont limit the number of batches that this
stream is importing.")
(:public
#>cpp
StartStopStream() {}
StartStopStream(std::string stream_name, bool is_start,
Expression *limit_batches);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class start-stop-all-streams (logical-operator)
((stream-name "std::string" :scope :public)
(is-start :bool :scope :public))
(:documentation
"Starts or stops all stream.")
(:public
#>cpp
StartStopAllStreams() {}
explicit StartStopAllStreams(bool is_start);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
(lcp:define-class test-stream (logical-operator)
((stream-name "std::string" :scope :public)
(limit-batches "Expression *"
:scope :public
:capnp-type "Ast.Tree"
:capnp-init nil
:capnp-save #'save-ast-pointer
:capnp-load (load-ast-pointer "Expression *"))
(query-symbol "Symbol" :scope :public)
(params-symbol "Symbol" :scope :public))
(:documentation
"Test a stream. This will start consuming messages but wont insert anything
in the db.")
(:public
#>cpp
TestStream() {}
TestStream(std::string stream_name, Expression *limit_batches,
Symbol query_symbol, Symbol params_symbol);
DEFVISITABLE(HierarchicalLogicalOperatorVisitor);
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor &db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override {
return {};
}
bool HasSingleInput() const override;
std::shared_ptr<LogicalOperator> input() const override;
void set_input(std::shared_ptr<LogicalOperator>) override;
cpp<#)
(:serialize :capnp))
;;; END KAFKA STREAM OPERATORS
(lcp:define-class explain (logical-operator)
((input "std::shared_ptr<LogicalOperator>" :scope :public
:capnp-save #'save-operator-pointer
:capnp-load #'load-operator-pointer)
(output-symbol "Symbol" :scope :public)
(pretty-print
"std::function<void(const database::GraphDbAccessor &, LogicalOperator *, std::ostream *)>"
:scope :public :documentation "Pretty printer function"
:capnp-save :dont-save))
(:documentation "Pretty print a LogicalOperator plan")
(:public
#>cpp
Explain() {}
Explain(
const std::shared_ptr<LogicalOperator> &input, const Symbol &output_symbol,
const std::function<void(const database::GraphDbAccessor &,
LogicalOperator *, std::ostream *)> &pretty_print);
bool Accept(HierarchicalLogicalOperatorVisitor &visitor) override;
std::unique_ptr<Cursor> MakeCursor(
database::GraphDbAccessor & db) const override;
std::vector<Symbol> OutputSymbols(const SymbolTable &) const override;
std::vector<Symbol> ModifiedSymbols(const SymbolTable &) const override;
bool HasSingleInput() const override { return true; }
std::shared_ptr<LogicalOperator> input() const override { return input_; }
void set_input(std::shared_ptr<LogicalOperator> input) override {
input_ = input;
}
void set_input(std::shared_ptr<LogicalOperator> input) override;
cpp<#)
(:serialize :capnp))

View File

@ -100,15 +100,6 @@ auto MakeLogicalPlan(TPlanningContext &context, const Parameters &parameters,
prev_op, prev_op->OutputSymbols(context.symbol_table));
}
if (context.query->explain_) {
last_op = std::make_unique<Explain>(
std::move(last_op),
context.symbol_table.CreateSymbol("QUERY PLAN", false),
[](const auto &dba, auto *root, auto *stream) {
return PrettyPrint(dba, root, stream);
});
}
return std::make_pair(std::move(last_op), total_cost);
}

View File

@ -477,7 +477,7 @@ std::vector<SingleQueryPart> CollectSingleQueryParts(
}
QueryParts CollectQueryParts(SymbolTable &symbol_table, AstStorage &storage,
Query *query) {
CypherQuery *query) {
std::vector<QueryPart> query_parts;
auto *single_query = query->single_query_;

View File

@ -52,15 +52,9 @@ class UsedSymbolsCollector : public HierarchicalTreeVisitor {
bool Visit(PrimitiveLiteral &) override { return true; }
bool Visit(ParameterLookup &) override { return true; }
bool Visit(query::CreateIndex &) override { return true; }
bool Visit(query::CreateUniqueIndex &) override { return true; }
bool Visit(query::IndexQuery &) override { return true; }
bool Visit(query::AuthQuery &) override { return true; }
bool Visit(query::CreateStream &) override { return true; }
bool Visit(query::DropStream &) override { return true; }
bool Visit(query::ShowStreams &) override { return true; }
bool Visit(query::StartStopStream &) override { return true; }
bool Visit(query::StartStopAllStreams &) override { return true; }
bool Visit(query::TestStream &) override { return true; }
bool Visit(query::StreamQuery &) override { return true; }
std::unordered_set<Symbol> symbols_;
const SymbolTable &symbol_table_;
@ -290,6 +284,6 @@ struct QueryParts {
/// and do some other preprocessing in order to generate multiple @c QueryPart
/// structures. @c AstStorage and @c SymbolTable may be used to create new
/// AST nodes.
QueryParts CollectQueryParts(SymbolTable &, AstStorage &, Query *);
QueryParts CollectQueryParts(SymbolTable &, AstStorage &, CypherQuery *);
} // namespace query::plan

View File

@ -143,53 +143,6 @@ bool PlanPrinter::Visit(query::plan::Once &op) {
return true;
}
bool PlanPrinter::Visit(query::plan::CreateIndex &op) {
WithPrintLn([](auto &out) { out << "* CreateIndex"; });
return true;
}
bool PlanPrinter::Visit(query::plan::AuthHandler &op) {
WithPrintLn([](auto &out) { out << "* AuthHandler"; });
return true;
}
bool PlanPrinter::Visit(query::plan::CreateStream &op) {
WithPrintLn([](auto &out) { out << "* CreateStream"; });
return true;
}
bool PlanPrinter::Visit(query::plan::DropStream &op) {
WithPrintLn([](auto &out) { out << "* DropStream"; });
return true;
}
bool PlanPrinter::Visit(query::plan::ShowStreams &op) {
WithPrintLn([](auto &out) { out << "* ShowStreams"; });
return true;
}
bool PlanPrinter::Visit(query::plan::StartStopStream &op) {
WithPrintLn([](auto &out) { out << "* StartStopStream"; });
return true;
}
bool PlanPrinter::Visit(query::plan::StartStopAllStreams &op) {
WithPrintLn([](auto &out) { out << "* StartStopAllStreams"; });
return true;
}
bool PlanPrinter::Visit(query::plan::TestStream &op) {
WithPrintLn([](auto &out) { out << "* TestStream"; });
return true;
}
bool PlanPrinter::PreVisit(query::plan::Explain &explain) {
WithPrintLn([&explain](auto &out) {
out << "* Explain {" << explain.output_symbol_.name() << "}";
});
return true;
}
bool PlanPrinter::PreVisit(query::plan::Cartesian &op) {
WithPrintLn([&op](auto &out) {
out << "* Cartesian {";
@ -229,9 +182,10 @@ void PlanPrinter::PrintExpand(const query::plan::ExpandCommon &op) {
}
void PrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root, std::ostream *out) {
const LogicalOperator *plan_root, std::ostream *out) {
PlanPrinter printer(&dba, out);
plan_root->Accept(printer);
// FIXME(mtomic): We should make visitors that take const arguments.
const_cast<LogicalOperator *>(plan_root)->Accept(printer);
}
} // namespace query::plan

View File

@ -18,11 +18,11 @@ class LogicalOperator;
/// Note that `plan_root` isn't modified, but we can't take it as a const
/// because we don't have support for visiting a const LogicalOperator.
void PrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root, std::ostream *out);
const LogicalOperator *plan_root, std::ostream *out);
/// Overload of `PrettyPrint` which defaults the `std::ostream` to `std::cout`.
inline void PrettyPrint(const database::GraphDbAccessor &dba,
LogicalOperator *plan_root) {
const LogicalOperator *plan_root) {
PrettyPrint(dba, plan_root, &std::cout);
}
@ -74,19 +74,8 @@ class PlanPrinter : public virtual HierarchicalLogicalOperatorVisitor {
bool PreVisit(Distinct &) override;
bool PreVisit(Unwind &) override;
bool PreVisit(Explain &) override;
bool Visit(Once &) override;
bool Visit(CreateIndex &) override;
bool Visit(AuthHandler &) override;
bool Visit(CreateStream &) override;
bool Visit(DropStream &) override;
bool Visit(ShowStreams &) override;
bool Visit(StartStopStream &) override;
bool Visit(StartStopAllStreams &) override;
bool Visit(TestStream &) override;
/// Call fun with output stream. The stream is prefixed with amount of spaces
/// corresponding to the current depth_.

View File

@ -397,12 +397,7 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
return true;
}
bool Visit(query::CreateIndex &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::CreateUniqueIndex &) override {
bool Visit(query::IndexQuery &) override {
has_aggregation_.emplace_back(false);
return true;
}
@ -412,32 +407,7 @@ class ReturnBodyContext : public HierarchicalTreeVisitor {
return true;
}
bool Visit(query::CreateStream &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::DropStream &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::ShowStreams &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::StartStopStream &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::StartStopAllStreams &) override {
has_aggregation_.emplace_back(false);
return true;
}
bool Visit(query::TestStream &) override {
bool Visit(query::StreamQuery &) override {
has_aggregation_.emplace_back(false);
return true;
}

View File

@ -23,8 +23,8 @@ struct PlanningContext {
SymbolTable &symbol_table;
/// @brief The storage is used to create new AST nodes for use in operators.
AstStorage &ast_storage;
/// @brief Query to be planned
Query *query;
/// @brief Cypher query to be planned
CypherQuery *query;
/// @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.
@ -40,7 +40,7 @@ struct PlanningContext {
template <class TDbAccessor>
auto MakePlanningContext(AstStorage &ast_storage, SymbolTable &symbol_table,
Query *query, const TDbAccessor &db) {
CypherQuery *query, const TDbAccessor &db) {
return PlanningContext<TDbAccessor>{symbol_table, ast_storage, query, db};
}
@ -178,70 +178,6 @@ class RuleBasedPlanner {
input_op = std::make_unique<plan::Unwind>(
std::move(input_op), unwind->named_expression_->expression_,
symbol);
} else if (auto *create_index =
dynamic_cast<query::CreateIndex *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before CreateIndex";
input_op = std::make_unique<plan::CreateIndex>(
create_index->label_,
std::vector<storage::Property>{create_index->property_}, false);
} else if (auto *create_index =
dynamic_cast<query::CreateUniqueIndex *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before CreateIndex";
input_op = std::make_unique<plan::CreateIndex>(
create_index->label_, create_index->properties_, true);
} else if (auto *auth_query =
dynamic_cast<query::AuthQuery *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before AuthQuery";
auto &symbol_table = context.symbol_table;
input_op = std::make_unique<plan::AuthHandler>(
auth_query->action_, auth_query->user_, auth_query->role_,
auth_query->user_or_role_, auth_query->password_,
auth_query->privileges_, symbol_table.CreateSymbol("user", false),
symbol_table.CreateSymbol("role", false),
symbol_table.CreateSymbol("privilege", false),
symbol_table.CreateSymbol("effective", false),
symbol_table.CreateSymbol("details", false));
} else if (auto *create_stream =
dynamic_cast<query::CreateStream *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before CreateStream";
input_op = std::make_unique<plan::CreateStream>(
create_stream->stream_name_, create_stream->stream_uri_,
create_stream->stream_topic_, create_stream->transform_uri_,
create_stream->batch_interval_in_ms_, create_stream->batch_size_);
} else if (auto *drop_stream =
dynamic_cast<query::DropStream *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before DropStream";
input_op =
std::make_unique<plan::DropStream>(drop_stream->stream_name_);
} else if (dynamic_cast<query::ShowStreams *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before ShowStreams";
// Create symbols for ShowStreams results
auto &symbol_table = context.symbol_table;
input_op = std::make_unique<plan::ShowStreams>(
symbol_table.CreateSymbol("name", false),
symbol_table.CreateSymbol("uri", false),
symbol_table.CreateSymbol("topic", false),
symbol_table.CreateSymbol("transform", false),
symbol_table.CreateSymbol("status", false));
} else if (auto *start_stop_stream =
dynamic_cast<query::StartStopStream *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before StartStopStream";
input_op = std::make_unique<plan::StartStopStream>(
start_stop_stream->stream_name_, start_stop_stream->is_start_,
start_stop_stream->limit_batches_);
} else if (auto *start_stop_all_streams =
dynamic_cast<query::StartStopAllStreams *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before StartStopAllStreams";
input_op = std::make_unique<plan::StartStopAllStreams>(
start_stop_all_streams->is_start_);
} else if (auto *test_stream =
dynamic_cast<query::TestStream *>(clause)) {
DCHECK(!input_op) << "Unexpected operator before TestStream";
auto &symbol_table = context.symbol_table;
input_op = std::make_unique<plan::TestStream>(
test_stream->stream_name_, test_stream->limit_batches_,
symbol_table.CreateSymbol("query", false),
symbol_table.CreateSymbol("params", false));
} else {
throw utils::NotYetImplemented("clause conversion to operator(s)");
}

View File

@ -10,9 +10,9 @@
#include "query/plan/vertex_count_cache.hpp"
// Add chained MATCH (node1) -- (node2), MATCH (node2) -- (node3) ... clauses.
static query::Query *AddChainedMatches(int num_matches,
query::AstStorage &storage) {
auto *query = storage.Create<query::Query>();
static query::CypherQuery *AddChainedMatches(int num_matches,
query::AstStorage &storage) {
auto *query = storage.Create<query::CypherQuery>();
for (int i = 0; i < num_matches; ++i) {
auto *match = storage.Create<query::Match>();
auto *pattern = storage.Create<query::Pattern>();
@ -67,11 +67,11 @@ BENCHMARK(BM_PlanChainedMatches)
->Range(50, 400)
->Unit(benchmark::kMillisecond);
static query::Query *AddIndexedMatches(
static query::CypherQuery *AddIndexedMatches(
int num_matches, storage::Label label,
const std::pair<std::string, storage::Property> &property,
query::AstStorage &storage) {
auto *query = storage.Create<query::Query>();
auto *query = storage.Create<query::CypherQuery>();
for (int i = 0; i < num_matches; ++i) {
auto *match = storage.Create<query::Match>();
auto *pattern = storage.Create<query::Pattern>();

View File

@ -453,7 +453,7 @@ query::SymbolTable MakeSymbolTable(query::Query *query) {
// Returns a list of pairs (plan, estimated cost), sorted in the ascending
// order by cost.
auto MakeLogicalPlans(query::Query *query, query::AstStorage &ast,
auto MakeLogicalPlans(query::CypherQuery *query, query::AstStorage &ast,
query::SymbolTable &symbol_table,
InteractiveDbAccessor &dba) {
auto query_parts = query::plan::CollectQueryParts(symbol_table, ast, query);
@ -501,7 +501,13 @@ void RunInteractivePlanning(database::GraphDbAccessor *dba) {
if (line->empty()) continue;
try {
query::AstStorage ast;
auto *query = MakeAst(*line, &ast, *dba);
auto *query =
dynamic_cast<query::CypherQuery *>(MakeAst(*line, &ast, *dba));
if (!query) {
throw utils::BasicException(
"Interactive planning is only avaialable for regular openCypher "
"queries.");
}
auto symbol_table = MakeSymbolTable(query);
planning_timer.Start();
auto plans = MakeLogicalPlans(query, ast, symbol_table, interactive_db);

File diff suppressed because it is too large Load Diff

View File

@ -622,7 +622,7 @@ struct ExpectedDistributedPlan {
};
template <class TPlanner>
DistributedPlan MakeDistributedPlan(query::Query *query,
DistributedPlan MakeDistributedPlan(query::CypherQuery *query,
query::AstStorage &storage) {
auto symbol_table = MakeSymbolTable(*query);
FakeDbAccessor dba;
@ -662,7 +662,7 @@ void CheckDistributedPlan(const LogicalOperator &plan,
}
template <class TPlanner>
void CheckDistributedPlan(query::Query *query, AstStorage &storage,
void CheckDistributedPlan(query::CypherQuery *query, AstStorage &storage,
ExpectedDistributedPlan &expected_distributed_plan) {
auto distributed_plan = MakeDistributedPlan<TPlanner>(query, storage);
CheckDistributedPlan(distributed_plan, expected_distributed_plan);
@ -1346,18 +1346,6 @@ TYPED_TEST(TestPlanner, FunctionWithoutArguments) {
CheckDistributedPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, CreateIndex) {
// Test CREATE INDEX ON :Label(property)
FakeDbAccessor dba;
auto label = dba.Label("label");
auto property = dba.Property("property");
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(CREATE_INDEX_ON(label, property)));
auto expected =
ExpectDistributed(MakeCheckers(ExpectCreateIndex(label, property)));
CheckDistributedPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, MatchBfs) {
// Test MATCH (n) -[r:type *..10 (r, n|n)]-> (m) RETURN r
FakeDbAccessor dba;
@ -1383,190 +1371,6 @@ TYPED_TEST(TestPlanner, MatchBfs) {
CheckDistributedPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, AuthQuery) {
// Check if everything is properly forwarded from ast node to the operator
FakeDbAccessor dba;
AstStorage storage;
auto *query =
QUERY(SINGLE_QUERY(AUTH_QUERY(query::AuthQuery::Action::DROP_ROLE, "user",
"role", "user_or_role", LITERAL("password"),
std::vector<query::AuthQuery::Privilege>(
{query::AuthQuery::Privilege::MATCH,
query::AuthQuery::Privilege::AUTH}))));
auto expected = ExpectDistributed(MakeCheckers(
ExpectAuthHandler(query::AuthQuery::Action::DROP_ROLE, "user", "role",
"user_or_role", LITERAL("password"),
{query::AuthQuery::Privilege::MATCH,
query::AuthQuery::Privilege::AUTH})));
CheckDistributedPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, CreateStream) {
std::string stream_name("kafka"), stream_uri("localhost:1234"),
stream_topic("tropik"), transform_uri("localhost:1234/file.py");
int64_t batch_interval_in_ms = 100;
int64_t batch_size = 10;
{
FakeDbAccessor dba;
AstStorage storage;
auto *query =
QUERY(SINGLE_QUERY(CREATE_STREAM(stream_name, stream_uri, stream_topic,
transform_uri, nullptr, nullptr)));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, nullptr);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, nullptr)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
LITERAL(batch_interval_in_ms), nullptr)));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), LITERAL(batch_interval_in_ms), nullptr);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), LITERAL(batch_interval_in_ms), nullptr)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
nullptr, LITERAL(batch_size))));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, LITERAL(batch_size));
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, LITERAL(batch_size))));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
LITERAL(batch_interval_in_ms), LITERAL(batch_size))));
auto expected =
ExpectCreateStream(stream_name, LITERAL(stream_uri),
LITERAL(stream_topic), LITERAL(transform_uri),
LITERAL(batch_interval_in_ms), LITERAL(batch_size));
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), LITERAL(batch_interval_in_ms),
LITERAL(batch_size))));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
}
TYPED_TEST(TestPlanner, DropStream) {
std::string stream_name("kafka");
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(DROP_STREAM(stream_name)));
auto expected = ExpectDropStream(stream_name);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectDropStream(stream_name)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
TYPED_TEST(TestPlanner, ShowStreams) {
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(SHOW_STREAMS));
auto expected = ExpectShowStreams();
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectShowStreams()));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
TYPED_TEST(TestPlanner, StartStopStream) {
std::string stream_name("kafka");
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(START_STREAM(stream_name, nullptr)));
auto expected = ExpectStartStopStream(stream_name, true, nullptr);
auto expected_distributed = ExpectDistributed(
MakeCheckers(ExpectStartStopStream(stream_name, true, nullptr)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto limit_batches = LITERAL(10);
auto *query = QUERY(SINGLE_QUERY(START_STREAM(stream_name, limit_batches)));
auto expected = ExpectStartStopStream(stream_name, true, limit_batches);
auto expected_distributed = ExpectDistributed(
MakeCheckers(ExpectStartStopStream(stream_name, true, limit_batches)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(STOP_STREAM(stream_name)));
auto expected = ExpectStartStopStream(stream_name, false, nullptr);
auto expected_distributed = ExpectDistributed(
MakeCheckers(ExpectStartStopStream(stream_name, false, nullptr)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
}
TYPED_TEST(TestPlanner, StartStopAllStreams) {
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(START_ALL_STREAMS));
auto expected = ExpectStartStopAllStreams(true);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectStartStopAllStreams(true)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(STOP_ALL_STREAMS));
auto expected = ExpectStartStopAllStreams(false);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectStartStopAllStreams(false)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
}
TYPED_TEST(TestPlanner, TestStream) {
std::string stream_name("kafka");
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(TEST_STREAM(stream_name, nullptr)));
auto expected = ExpectTestStream(stream_name, nullptr);
auto expected_distributed =
ExpectDistributed(MakeCheckers(ExpectTestStream(stream_name, nullptr)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto limit_batches = LITERAL(10);
auto *query = QUERY(SINGLE_QUERY(TEST_STREAM(stream_name, limit_batches)));
auto expected = ExpectTestStream(stream_name, limit_batches);
auto expected_distributed = ExpectDistributed(
MakeCheckers(ExpectTestStream(stream_name, limit_batches)));
CheckDistributedPlan<TypeParam>(query, storage, expected_distributed);
}
}
TYPED_TEST(TestPlanner, DistributedAvg) {
// Test MATCH (n) RETURN AVG(n.prop) AS res
AstStorage storage;

View File

@ -269,7 +269,7 @@ auto GetCypherUnion(CypherUnion *cypher_union, SingleQuery *single_query) {
}
auto GetQuery(AstStorage &storage, SingleQuery *single_query) {
auto *query = storage.Create<Query>();
auto *query = storage.Create<CypherQuery>();
query->single_query_ = single_query;
return query;
}
@ -277,7 +277,7 @@ auto GetQuery(AstStorage &storage, SingleQuery *single_query) {
template <class... T>
auto GetQuery(AstStorage &storage, SingleQuery *single_query,
T *... cypher_unions) {
auto *query = storage.Create<Query>();
auto *query = storage.Create<CypherQuery>();
query->single_query_ = single_query;
query->cypher_unions_ = std::vector<CypherUnion *>{cypher_unions...};
return query;
@ -539,8 +539,10 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
query::test_common::OnCreate { \
std::vector<query::Clause *> { __VA_ARGS__ } \
}
#define CREATE_INDEX_ON(label, property) \
storage.Create<query::CreateIndex>((label), (property))
#define CREATE_INDEX_ON(label, property) \
storage.Create<query::IndexQuery>( \
query::IndexQuery::Action::CREATE, (label), \
std::vector<storage::Property>{(property)})
#define QUERY(...) query::test_common::GetQuery(storage, __VA_ARGS__)
#define SINGLE_QUERY(...) \
query::test_common::GetSingleQuery(storage.Create<SingleQuery>(), __VA_ARGS__)
@ -609,17 +611,35 @@ auto GetMerge(AstStorage &storage, Pattern *pattern, OnMatch on_match,
#define DROP_USER(usernames) storage.Create<query::DropUser>((usernames))
#define CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri, \
batch_interval, batch_size) \
storage.Create<query::CreateStream>( \
(stream_name), LITERAL(stream_uri), LITERAL(stream_topic), \
LITERAL(transform_uri), (batch_interval), (batch_size))
#define DROP_STREAM(stream_name) \
storage.Create<query::DropStream>((stream_name))
#define SHOW_STREAMS storage.Create<query::ShowStreams>()
#define START_STREAM(stream_name, limit_batches) \
storage.Create<query::StartStopStream>((stream_name), true, (limit_batches))
#define STOP_STREAM(stream_name) \
storage.Create<query::StartStopStream>((stream_name), false, nullptr)
#define START_ALL_STREAMS storage.Create<query::StartStopAllStreams>(true)
#define STOP_ALL_STREAMS storage.Create<query::StartStopAllStreams>(false)
#define TEST_STREAM(stream_name, limit_batches) \
storage.Create<query::TestStream>((stream_name), (limit_batches))
storage.Create<query::StreamQuery>( \
query::StreamQuery::Action::CREATE_STREAM, (stream_name), \
LITERAL(stream_uri), LITERAL(stream_topic), LITERAL(transform_uri), \
(batch_interval), (batch_size), nullptr)
#define DROP_STREAM(stream_name) \
storage.Create<query::StreamQuery>(query::StreamQuery::Action::DROP_STREAM, \
(stream_name), nullptr, nullptr, nullptr, \
nullptr, nullptr, nullptr)
#define SHOW_STREAMS \
storage.Create<query::StreamQuery>(query::StreamQuery::Action::SHOW_STREAMS, \
"", nullptr, nullptr, nullptr, nullptr, \
nullptr, nullptr)
#define START_STREAM(stream_name, limit_batches) \
storage.Create<query::StreamQuery>(query::StreamQuery::Action::START_STREAM, \
(stream_name), nullptr, nullptr, nullptr, \
nullptr, nullptr, (limit_batches))
#define STOP_STREAM(stream_name) \
storage.Create<query::StreamQuery>(query::StreamQuery::Action::STOP_STREAM, \
(stream_name), nullptr, nullptr, nullptr, \
nullptr, nullptr, nullptr)
#define START_ALL_STREAMS \
storage.Create<query::StreamQuery>( \
query::StreamQuery::Action::START_ALL_STREAMS, "", nullptr, nullptr, \
nullptr, nullptr, nullptr, nullptr)
#define STOP_ALL_STREAMS \
storage.Create<query::StreamQuery>( \
query::StreamQuery::Action::STOP_ALL_STREAMS, "", nullptr, nullptr, \
nullptr, nullptr, nullptr, nullptr)
#define TEST_STREAM(stream_name, limit_batches) \
storage.Create<query::TestStream>(query::StreamQuery::Action::TEST_STREAM, \
(stream_name), nullptr, nullptr, nullptr, \
nullptr, nullptr, (limit_batches))

View File

@ -64,7 +64,8 @@ auto CheckPlan(LogicalOperator &plan, const SymbolTable &symbol_table,
}
template <class TPlanner, class... TChecker>
auto CheckPlan(query::Query *query, AstStorage &storage, TChecker... checker) {
auto CheckPlan(query::CypherQuery *query, AstStorage &storage,
TChecker... checker) {
auto symbol_table = MakeSymbolTable(*query);
FakeDbAccessor dba;
auto planner = MakePlanner<TPlanner>(dba, storage, symbol_table, query);
@ -934,16 +935,6 @@ TYPED_TEST(TestPlanner, MapWithAggregationAndGroupBy) {
CheckPlan<TypeParam>(query, storage, aggr, ExpectProduce());
}
TYPED_TEST(TestPlanner, CreateIndex) {
// Test CREATE INDEX ON :Label(property)
FakeDbAccessor dba;
auto label = dba.Label("label");
auto property = dba.Property("property");
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(CREATE_INDEX_ON(label, property)));
CheckPlan<TypeParam>(query, storage, ExpectCreateIndex(label, property));
}
TYPED_TEST(TestPlanner, AtomIndexedLabelProperty) {
// Test MATCH (n :label {property: 42, not_indexed: 0}) RETURN n
AstStorage storage;
@ -1351,153 +1342,4 @@ TYPED_TEST(TestPlanner, ReturnAsteriskOmitsLambdaSymbols) {
}
}
TYPED_TEST(TestPlanner, AuthQuery) {
// Check if everything is properly forwarded from ast node to the operator
FakeDbAccessor dba;
AstStorage storage;
auto *query =
QUERY(SINGLE_QUERY(AUTH_QUERY(query::AuthQuery::Action::DROP_ROLE, "user",
"role", "user_or_role", LITERAL("password"),
std::vector<query::AuthQuery::Privilege>(
{query::AuthQuery::Privilege::MATCH,
query::AuthQuery::Privilege::AUTH}))));
CheckPlan<TypeParam>(
query, storage,
ExpectAuthHandler(query::AuthQuery::Action::DROP_ROLE, "user", "role",
"user_or_role", LITERAL("password"),
{query::AuthQuery::Privilege::MATCH,
query::AuthQuery::Privilege::AUTH}));
}
TYPED_TEST(TestPlanner, CreateStream) {
std::string stream_name("kafka"), stream_uri("localhost:1234"),
stream_topic("tropik"), transform_uri("localhost:1234/file.py");
int64_t batch_interval_in_ms = 100;
int64_t batch_size = 10;
{
FakeDbAccessor dba;
AstStorage storage;
auto *query =
QUERY(SINGLE_QUERY(CREATE_STREAM(stream_name, stream_uri, stream_topic,
transform_uri, nullptr, nullptr)));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, nullptr);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
LITERAL(batch_interval_in_ms), nullptr)));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), LITERAL(batch_interval_in_ms), nullptr);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
nullptr, LITERAL(batch_size))));
auto expected = ExpectCreateStream(
stream_name, LITERAL(stream_uri), LITERAL(stream_topic),
LITERAL(transform_uri), nullptr, LITERAL(batch_size));
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
LITERAL(batch_interval_in_ms), LITERAL(batch_size))));
auto expected =
ExpectCreateStream(stream_name, LITERAL(stream_uri),
LITERAL(stream_topic), LITERAL(transform_uri),
LITERAL(batch_interval_in_ms), LITERAL(batch_size));
CheckPlan<TypeParam>(query, storage, expected);
}
}
TYPED_TEST(TestPlanner, DropStream) {
std::string stream_name("kafka");
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(DROP_STREAM(stream_name)));
auto expected = ExpectDropStream(stream_name);
CheckPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, ShowStreams) {
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(SHOW_STREAMS));
auto expected = ExpectShowStreams();
CheckPlan<TypeParam>(query, storage, expected);
}
TYPED_TEST(TestPlanner, StartStopStream) {
std::string stream_name("kafka");
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(START_STREAM(stream_name, nullptr)));
auto expected = ExpectStartStopStream(stream_name, true, nullptr);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto limit_batches = LITERAL(10);
auto *query = QUERY(SINGLE_QUERY(START_STREAM(stream_name, limit_batches)));
auto expected = ExpectStartStopStream(stream_name, true, limit_batches);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(STOP_STREAM(stream_name)));
auto expected = ExpectStartStopStream(stream_name, false, nullptr);
CheckPlan<TypeParam>(query, storage, expected);
}
}
TYPED_TEST(TestPlanner, StartStopAllStreams) {
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(START_ALL_STREAMS));
auto expected = ExpectStartStopAllStreams(true);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(STOP_ALL_STREAMS));
auto expected = ExpectStartStopAllStreams(false);
CheckPlan<TypeParam>(query, storage, expected);
}
}
TYPED_TEST(TestPlanner, TestStream) {
std::string stream_name("kafka");
{
FakeDbAccessor dba;
AstStorage storage;
auto *query = QUERY(SINGLE_QUERY(TEST_STREAM(stream_name, nullptr)));
auto expected = ExpectTestStream(stream_name, nullptr);
CheckPlan<TypeParam>(query, storage, expected);
}
{
FakeDbAccessor dba;
AstStorage storage;
auto limit_batches = LITERAL(10);
auto *query = QUERY(SINGLE_QUERY(TEST_STREAM(stream_name, limit_batches)));
auto expected = ExpectTestStream(stream_name, limit_batches);
CheckPlan<TypeParam>(query, storage, expected);
}
}
} // namespace

View File

@ -86,24 +86,11 @@ class PlanChecker : public virtual HierarchicalLogicalOperatorVisitor {
return true;
}
VISIT(CreateIndex);
bool PreVisit(Cartesian &op) override {
CheckOp(op);
return false;
}
VISIT(AuthHandler);
VISIT(CreateStream);
VISIT(DropStream);
VISIT(ShowStreams);
VISIT(StartStopStream);
VISIT(StartStopAllStreams);
VISIT(TestStream);
PRE_VISIT(Explain);
#undef PRE_VISIT
#undef VISIT
@ -151,7 +138,6 @@ using ExpectLimit = OpChecker<Limit>;
using ExpectOrderBy = OpChecker<OrderBy>;
using ExpectUnwind = OpChecker<Unwind>;
using ExpectDistinct = OpChecker<Distinct>;
using ExpectShowStreams = OpChecker<ShowStreams>;
class ExpectExpandVariable : public OpChecker<ExpandVariable> {
public:
@ -323,56 +309,6 @@ class ExpectScanAllByLabelPropertyRange
std::experimental::optional<ScanAllByLabelPropertyRange::Bound> upper_bound_;
};
class ExpectAuthHandler : public OpChecker<AuthHandler> {
public:
ExpectAuthHandler(query::AuthQuery::Action action, std::string user,
std::string role, std::string user_or_role,
query::Expression *password,
std::vector<query::AuthQuery::Privilege> privileges)
: action_(action),
user_(user),
role_(role),
user_or_role_(user_or_role),
password_(password),
privileges_(privileges) {}
void ExpectOp(AuthHandler &auth_handler, const SymbolTable &) override {
EXPECT_EQ(auth_handler.action_, action_);
EXPECT_EQ(auth_handler.user_, user_);
EXPECT_EQ(auth_handler.role_, role_);
EXPECT_EQ(auth_handler.user_or_role_, user_or_role_);
// TODO(mtomic): We need to somehow test the password expression.
EXPECT_TRUE(password_);
EXPECT_TRUE(auth_handler.password_);
EXPECT_EQ(auth_handler.privileges_, privileges_);
}
private:
query::AuthQuery::Action action_;
std::string user_;
std::string role_;
std::string user_or_role_;
query::Expression *password_{nullptr};
std::vector<query::AuthQuery::Privilege> privileges_;
};
class ExpectCreateIndex : public OpChecker<CreateIndex> {
public:
ExpectCreateIndex(storage::Label label, storage::Property property)
: label_(label), property_(property) {}
void ExpectOp(CreateIndex &create_index, const SymbolTable &) override {
EXPECT_EQ(create_index.label_, label_);
EXPECT_EQ(create_index.properties_,
std::vector<storage::Property>{property_});
EXPECT_FALSE(create_index.is_unique_);
}
private:
storage::Label label_;
storage::Property property_;
};
class ExpectCartesian : public OpChecker<Cartesian> {
public:
ExpectCartesian(const std::list<std::unique_ptr<BaseOpChecker>> &left,
@ -393,130 +329,6 @@ class ExpectCartesian : public OpChecker<Cartesian> {
const std::list<std::unique_ptr<BaseOpChecker>> &right_;
};
class ExpectCreateStream : public OpChecker<CreateStream> {
public:
ExpectCreateStream(std::string stream_name, query::Expression *stream_uri,
query::Expression *stream_topic,
query::Expression *transform_uri,
query::Expression *batch_interval_in_ms,
query::Expression *batch_size)
: stream_name_(stream_name),
stream_uri_(stream_uri),
stream_topic_(stream_topic),
transform_uri_(transform_uri),
batch_interval_in_ms_(batch_interval_in_ms),
batch_size_(batch_size) {}
void ExpectOp(CreateStream &create_stream, const SymbolTable &) override {
EXPECT_EQ(create_stream.stream_name_, stream_name_);
// TODO: Proper expression equality
EXPECT_EQ(typeid(create_stream.stream_uri_).hash_code(),
typeid(stream_uri_).hash_code());
EXPECT_EQ(typeid(create_stream.stream_topic_).hash_code(),
typeid(stream_topic_).hash_code());
EXPECT_EQ(typeid(create_stream.transform_uri_).hash_code(),
typeid(transform_uri_).hash_code());
if (batch_interval_in_ms_ && create_stream.batch_interval_in_ms_) {
EXPECT_EQ(typeid(create_stream.batch_interval_in_ms_).hash_code(),
typeid(batch_interval_in_ms_).hash_code());
} else {
EXPECT_TRUE(batch_interval_in_ms_ == nullptr &&
create_stream.batch_interval_in_ms_ == nullptr);
}
if (batch_size_ && create_stream.batch_size_) {
EXPECT_EQ(typeid(create_stream.batch_size_).hash_code(),
typeid(batch_size_).hash_code());
} else {
EXPECT_TRUE(batch_size_ == nullptr &&
create_stream.batch_size_ == nullptr);
}
}
private:
std::string stream_name_;
query::Expression *stream_uri_;
query::Expression *stream_topic_;
query::Expression *transform_uri_;
query::Expression *batch_interval_in_ms_;
query::Expression *batch_size_;
};
class ExpectDropStream : public OpChecker<DropStream> {
public:
explicit ExpectDropStream(std::string stream_name)
: stream_name_(stream_name) {}
void ExpectOp(DropStream &drop_stream, const SymbolTable &) override {
EXPECT_EQ(drop_stream.stream_name_, stream_name_);
}
private:
std::string stream_name_;
};
class ExpectStartStopStream : public OpChecker<StartStopStream> {
public:
ExpectStartStopStream(std::string stream_name, bool is_start,
query::Expression *limit_batches)
: stream_name_(stream_name),
is_start_(is_start),
limit_batches_(limit_batches) {}
void ExpectOp(StartStopStream &start_stop_stream,
const SymbolTable &) override {
EXPECT_EQ(start_stop_stream.stream_name_, stream_name_);
EXPECT_EQ(start_stop_stream.is_start_, is_start_);
// TODO: Proper expression equality
if (limit_batches_ && start_stop_stream.limit_batches_) {
EXPECT_EQ(typeid(start_stop_stream.limit_batches_).hash_code(),
typeid(limit_batches_).hash_code());
} else {
EXPECT_TRUE(limit_batches_ == nullptr &&
start_stop_stream.limit_batches_ == nullptr);
}
}
private:
std::string stream_name_;
bool is_start_;
query::Expression *limit_batches_;
};
class ExpectStartStopAllStreams : public OpChecker<StartStopAllStreams> {
public:
explicit ExpectStartStopAllStreams(bool is_start) : is_start_(is_start) {}
void ExpectOp(StartStopAllStreams &start_stop_all_streams,
const SymbolTable &) override {
EXPECT_EQ(start_stop_all_streams.is_start_, is_start_);
}
private:
bool is_start_;
};
class ExpectTestStream : public OpChecker<TestStream> {
public:
ExpectTestStream(std::string stream_name, query::Expression *limit_batches)
: stream_name_(stream_name), limit_batches_(limit_batches) {}
void ExpectOp(TestStream &test_stream, const SymbolTable &) override {
EXPECT_EQ(test_stream.stream_name_, stream_name_);
// TODO: Proper expression equality
if (limit_batches_ && test_stream.limit_batches_) {
EXPECT_EQ(typeid(test_stream.limit_batches_).hash_code(),
typeid(limit_batches_).hash_code());
} else {
EXPECT_TRUE(limit_batches_ == nullptr &&
test_stream.limit_batches_ == nullptr);
}
}
private:
std::string stream_name_;
query::Expression *limit_batches_;
};
template <class T>
std::list<std::unique_ptr<BaseOpChecker>> MakeCheckers(T arg) {
std::list<std::unique_ptr<BaseOpChecker>> l;
@ -540,7 +352,7 @@ auto MakeSymbolTable(query::Query &query) {
template <class TPlanner, class TDbAccessor>
TPlanner MakePlanner(const TDbAccessor &dba, AstStorage &storage,
SymbolTable &symbol_table, Query *query) {
SymbolTable &symbol_table, CypherQuery *query) {
auto planning_context =
MakePlanningContext(storage, symbol_table, query, dba);
auto query_parts = CollectQueryParts(symbol_table, storage, query);

View File

@ -951,34 +951,6 @@ TEST(QueryPlan, RemoveLabelsOnNull) {
EXPECT_EQ(1, PullAll(remove_op, *dba, symbol_table));
}
TEST(QueryPlan, CreateIndex) {
// CREATE INDEX ON :Label(property)
database::GraphDb db;
auto dba = db.Access();
auto label = dba->Label("label");
auto property = dba->Property("property");
EXPECT_FALSE(dba->LabelPropertyIndexExists(label, property));
auto create_index = std::make_shared<plan::CreateIndex>(
label, std::vector<storage::Property>{property}, false);
SymbolTable symbol_table;
EXPECT_EQ(PullAll(create_index, *dba, symbol_table), 1);
EXPECT_TRUE(dba->LabelPropertyIndexExists(label, property));
}
TEST(QueryPlan, CreateUniqueIndex) {
// CREATE UNIQUE INDEX ON :Label(prop1)
database::GraphDb db;
auto dba = db.Access();
auto label = dba->Label("label");
auto prop1 = dba->Property("prop1");
std::vector<storage::Property> properties{prop1};
auto create_index =
std::make_shared<plan::CreateIndex>(label, properties, true);
SymbolTable symbol_table;
EXPECT_EQ(PullAll(create_index, *dba, symbol_table), 1);
EXPECT_TRUE(dba->LabelPropertyIndexExists(label, prop1));
}
TEST(QueryPlan, DeleteSetProperty) {
database::GraphDb db;
auto dba_ptr = db.Access();

View File

@ -97,15 +97,14 @@ TEST_F(TestPrivilegeExtractor, MatchNodeRemoveProperty) {
}
TEST_F(TestPrivilegeExtractor, CreateIndex) {
auto *query = QUERY(SINGLE_QUERY(CREATE_INDEX_ON(LABEL_0, PROP_0)));
auto *query = CREATE_INDEX_ON(LABEL_0, PROP_0);
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::INDEX));
}
TEST_F(TestPrivilegeExtractor, AuthQuery) {
auto *query = QUERY(
SINGLE_QUERY(AUTH_QUERY(AuthQuery::Action::CREATE_ROLE, "", "role", "",
nullptr, std::vector<AuthQuery::Privilege>{})));
auto *query = AUTH_QUERY(AuthQuery::Action::CREATE_ROLE, "", "role", "",
nullptr, std::vector<AuthQuery::Privilege>{});
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::AUTH));
}
@ -116,7 +115,7 @@ TEST_F(TestPrivilegeExtractor, StreamQuery) {
std::string stream_topic("tropik");
std::string transform_uri("localhost:1234/file.py");
std::vector<Clause *> stream_clauses = {
std::vector<StreamQuery *> stream_queries = {
CREATE_STREAM(stream_name, stream_uri, stream_topic, transform_uri,
nullptr, nullptr),
DROP_STREAM(stream_name),
@ -126,8 +125,7 @@ TEST_F(TestPrivilegeExtractor, StreamQuery) {
START_ALL_STREAMS,
STOP_ALL_STREAMS};
for (auto *stream_clause : stream_clauses) {
auto *query = QUERY(SINGLE_QUERY(stream_clause));
for (auto *query : stream_queries) {
EXPECT_THAT(GetRequiredPrivileges(query),
UnorderedElementsAre(AuthQuery::Privilege::STREAM));
}

View File

@ -64,7 +64,7 @@ void AssertRows(const std::vector<std::vector<TypedValue>> &datum,
};
void CheckPlansProduce(
size_t expected_plan_count, query::Query *query, AstStorage &storage,
size_t expected_plan_count, query::CypherQuery *query, AstStorage &storage,
database::GraphDbAccessor &dba,
std::function<void(const std::vector<std::vector<TypedValue>> &)> check) {
auto symbol_table = MakeSymbolTable(*query);