Replace dynamic_casts in query with utils::Downcast

Reviewers: mtomic, llugovic

Reviewed By: mtomic

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1830
This commit is contained in:
Teon Banek 2019-01-25 14:49:50 +01:00
parent f91428f23f
commit a871212675
10 changed files with 83 additions and 76 deletions

View File

@ -143,7 +143,7 @@ Interpreter::Results DistributedInterpreter::operator()(
&db_accessor, params);
ParsedQuery &parsed_query = queries.second;
if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {
if (utils::IsSubtype(*parsed_query.query, ProfileQuery::kType)) {
throw utils::NotYetImplemented("PROFILE in a distributed query");
}

View File

@ -34,7 +34,7 @@ TAst *LoadAstPointer(AstStorage *ast, slk::Reader *reader,
if (!has_ptr) {
return nullptr;
}
auto *ret = dynamic_cast<TAst *>(Load(ast, reader, loaded_uids));
auto *ret = utils::Downcast<TAst>(Load(ast, reader, loaded_uids));
if (!ret) {
throw slk::SlkDecodeException("Loading unknown Ast node type");
}

View File

@ -167,12 +167,13 @@ antlrcpp::Any CypherMainVisitor::visitSingleQuery(
bool has_optional_match = false;
for (Clause *clause : single_query->clauses_) {
if (dynamic_cast<Unwind *>(clause)) {
const auto &clause_type = clause->GetTypeInfo();
if (utils::IsSubtype(clause_type, Unwind::kType)) {
if (has_update || has_return) {
throw SemanticException(
"UNWIND can't be put after RETURN clause or after an update.");
}
} else if (auto *match = dynamic_cast<Match *>(clause)) {
} else if (auto *match = utils::Downcast<Match>(clause)) {
if (has_update || has_return) {
throw SemanticException(
"MATCH can't be put after RETURN clause or after an update.");
@ -182,24 +183,24 @@ antlrcpp::Any CypherMainVisitor::visitSingleQuery(
} else if (has_optional_match) {
throw SemanticException("MATCH can't be put after OPTIONAL MATCH.");
}
} else if (dynamic_cast<Create *>(clause) ||
dynamic_cast<Delete *>(clause) ||
dynamic_cast<SetProperty *>(clause) ||
dynamic_cast<SetProperties *>(clause) ||
dynamic_cast<SetLabels *>(clause) ||
dynamic_cast<RemoveProperty *>(clause) ||
dynamic_cast<RemoveLabels *>(clause) ||
dynamic_cast<Merge *>(clause)) {
} else if (utils::IsSubtype(clause_type, Create::kType) ||
utils::IsSubtype(clause_type, Delete::kType) ||
utils::IsSubtype(clause_type, SetProperty::kType) ||
utils::IsSubtype(clause_type, SetProperties::kType) ||
utils::IsSubtype(clause_type, SetLabels::kType) ||
utils::IsSubtype(clause_type, RemoveProperty::kType) ||
utils::IsSubtype(clause_type, RemoveLabels::kType) ||
utils::IsSubtype(clause_type, Merge::kType)) {
if (has_return) {
throw SemanticException("Update clause can't be used after RETURN.");
}
has_update = true;
} else if (dynamic_cast<Return *>(clause)) {
} else if (utils::IsSubtype(clause_type, Return::kType)) {
if (has_return) {
throw SemanticException("There can only be one RETURN in a clause.");
}
has_return = true;
} else if (dynamic_cast<With *>(clause)) {
} else if (utils::IsSubtype(clause_type, With::kType)) {
if (has_return) {
throw SemanticException("RETURN can't be put before WITH.");
}
@ -701,12 +702,14 @@ antlrcpp::Any CypherMainVisitor::visitReturnItem(
MemgraphCypher::ReturnItemContext *ctx) {
auto *named_expr = storage_->Create<NamedExpression>();
named_expr->expression_ = ctx->expression()->accept(this);
CHECK(named_expr->expression_);
if (ctx->variable()) {
named_expr->name_ =
std::string(ctx->variable()->accept(this).as<std::string>());
users_identifiers.insert(named_expr->name_);
} else {
if (in_with_ && !dynamic_cast<Identifier *>(named_expr->expression_)) {
if (in_with_ &&
!utils::IsSubtype(*named_expr->expression_, Identifier::kType)) {
throw SemanticException("Only variables can be non-aliased in WITH.");
}
named_expr->name_ = std::string(ctx->getText());
@ -1139,7 +1142,7 @@ antlrcpp::Any CypherMainVisitor::visitExpression8(
// First production is comparison operator.
for (auto *child : partial_comparison_expressions) {
operators.push_back(
dynamic_cast<antlr4::tree::TerminalNode *>(child->children[0])
static_cast<antlr4::tree::TerminalNode *>(child->children[0])
->getSymbol()
->getType());
}
@ -1627,12 +1630,11 @@ antlrcpp::Any CypherMainVisitor::visitPropertyExpression(
Expression *expression = ctx->atom()->accept(this);
for (auto *lookup : ctx->propertyLookup()) {
PropertyIx key = lookup->accept(this);
auto property_lookup =
storage_->Create<PropertyLookup>(expression, key);
auto property_lookup = storage_->Create<PropertyLookup>(expression, key);
expression = property_lookup;
}
// It is guaranteed by grammar that there is at least one propertyLookup.
return dynamic_cast<PropertyLookup *>(expression);
return static_cast<PropertyLookup *>(expression);
}
antlrcpp::Any CypherMainVisitor::visitCaseExpression(

View File

@ -356,7 +356,7 @@ bool SymbolGenerator::PreVisit(Extract &extract) {
bool SymbolGenerator::PreVisit(Pattern &pattern) {
scope_.in_pattern = true;
if ((scope_.in_create || scope_.in_merge) && pattern.atoms_.size() == 1U) {
DCHECK(dynamic_cast<NodeAtom *>(pattern.atoms_[0]))
CHECK(utils::IsSubtype(*pattern.atoms_[0], NodeAtom::kType))
<< "Expected a single NodeAtom in Pattern";
scope_.in_create_node = true;
}

View File

@ -615,7 +615,7 @@ Interpreter::Results Interpreter::operator()(
// we must ensure it lives during the whole interpretation.
std::shared_ptr<CachedPlan> plan{nullptr};
if (auto *cypher_query = dynamic_cast<CypherQuery *>(parsed_query.query)) {
if (auto *cypher_query = utils::Downcast<CypherQuery>(parsed_query.query)) {
plan = CypherQueryToPlan(stripped_query.hash(), cypher_query,
std::move(ast_storage), parameters, &db_accessor);
auto planning_time = planning_timer.Elapsed();
@ -638,7 +638,7 @@ Interpreter::Results Interpreter::operator()(
summary, parsed_query.required_privileges);
}
if (auto *explain_query = dynamic_cast<ExplainQuery *>(parsed_query.query)) {
if (utils::IsSubtype(*parsed_query.query, ExplainQuery::kType)) {
const std::string kExplainQueryStart = "explain ";
CHECK(utils::StartsWith(utils::ToLowerCase(stripped_query.query()),
kExplainQueryStart))
@ -671,10 +671,12 @@ Interpreter::Results Interpreter::operator()(
&parameters, &ast_storage, &db_accessor, params);
StrippedQuery &stripped_query = queries.first;
ParsedQuery &parsed_query = queries.second;
std::shared_ptr<CachedPlan> cypher_query_plan = CypherQueryToPlan(
stripped_query.hash(), dynamic_cast<CypherQuery *>(parsed_query.query),
std::move(ast_storage), parameters, &db_accessor);
auto *cypher_query = utils::Downcast<CypherQuery>(parsed_query.query);
CHECK(cypher_query)
<< "Cypher grammar should not allow other queries in EXPLAIN";
std::shared_ptr<CachedPlan> cypher_query_plan =
CypherQueryToPlan(stripped_query.hash(), cypher_query,
std::move(ast_storage), parameters, &db_accessor);
std::stringstream printed_plan;
PrettyPrintPlan(db_accessor, &cypher_query_plan->plan(), &printed_plan);
@ -704,7 +706,7 @@ Interpreter::Results Interpreter::operator()(
summary, parsed_query.required_privileges);
}
if (auto *profile_query = dynamic_cast<ProfileQuery *>(parsed_query.query)) {
if (utils::IsSubtype(*parsed_query.query, ProfileQuery::kType)) {
const std::string kProfileQueryStart = "profile ";
CHECK(utils::StartsWith(utils::ToLowerCase(stripped_query.query()),
kProfileQueryStart))
@ -726,10 +728,12 @@ Interpreter::Results Interpreter::operator()(
&parameters, &ast_storage, &db_accessor, params);
StrippedQuery &stripped_query = queries.first;
ParsedQuery &parsed_query = queries.second;
auto cypher_query_plan = CypherQueryToPlan(
stripped_query.hash(), dynamic_cast<CypherQuery *>(parsed_query.query),
std::move(ast_storage), parameters, &db_accessor);
auto *cypher_query = utils::Downcast<CypherQuery>(parsed_query.query);
CHECK(cypher_query)
<< "Cypher grammar should not allow other queries in PROFILE";
auto cypher_query_plan =
CypherQueryToPlan(stripped_query.hash(), cypher_query,
std::move(ast_storage), parameters, &db_accessor);
// Copy the symbol table and add our own symbols (used by the `OutputTable`
// operator below)
@ -777,7 +781,7 @@ Interpreter::Results Interpreter::operator()(
}
Callback callback;
if (auto *index_query = dynamic_cast<IndexQuery *>(parsed_query.query)) {
if (auto *index_query = utils::Downcast<IndexQuery>(parsed_query.query)) {
if (in_explicit_transaction) {
throw IndexInMulticommandTxException();
}
@ -790,13 +794,14 @@ Interpreter::Results Interpreter::operator()(
};
callback =
HandleIndexQuery(index_query, invalidate_plan_cache, &db_accessor);
} else if (auto *auth_query = dynamic_cast<AuthQuery *>(parsed_query.query)) {
} else if (auto *auth_query =
utils::Downcast<AuthQuery>(parsed_query.query)) {
if (in_explicit_transaction) {
throw UserModificationInMulticommandTxException();
}
callback = HandleAuthQuery(auth_query, auth_, parameters, &db_accessor);
} else if (auto *stream_query =
dynamic_cast<StreamQuery *>(parsed_query.query)) {
utils::Downcast<StreamQuery>(parsed_query.query)) {
if (in_explicit_transaction) {
throw StreamClauseInMulticommandTxException();
}

View File

@ -170,8 +170,8 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
// if the Unwind expression is a list literal, we can deduce cardinality
// exactly, otherwise we approximate
int unwind_value;
if (auto literal =
dynamic_cast<query::ListLiteral *>(unwind.input_expression_))
if (auto *literal =
utils::Downcast<query::ListLiteral>(unwind.input_expression_))
unwind_value = literal->elements_.size();
else
unwind_value = MiscParam::kUnwindNoLiteral;
@ -219,10 +219,10 @@ class CostEstimator : public HierarchicalLogicalOperatorVisitor {
// return nullopt.
std::experimental::optional<PropertyValue> ConstPropertyValue(
const Expression *expression) {
if (auto *literal = dynamic_cast<const PrimitiveLiteral *>(expression)) {
if (auto *literal = utils::Downcast<const PrimitiveLiteral>(expression)) {
return literal->value_;
} else if (auto *param_lookup =
dynamic_cast<const ParameterLookup *>(expression)) {
utils::Downcast<const ParameterLookup>(expression)) {
return parameters.AtTokenPosition(param_lookup->token_position_);
}
return std::experimental::nullopt;

View File

@ -1026,7 +1026,7 @@ class DistributedPlanner : public HierarchicalLogicalOperatorVisitor {
optional_planner.lhs_optional_symbols_ =
op.input()->ModifiedSymbols(optional_plan.symbol_table);
optional_plan.master_plan->Accept(optional_planner);
CHECK(dynamic_cast<Produce *>(optional_plan.master_plan.get()));
CHECK(utils::IsSubtype(*optional_plan.master_plan.get(), Produce::kType));
// Revert storage and symbol table
distributed_plan_.ast_storage = std::move(optional_plan.ast_storage);
distributed_plan_.symbol_table = std::move(optional_plan.symbol_table);

View File

@ -47,7 +47,7 @@ class PostProcessor final {
std::unique_ptr<LogicalOperator> curr_op,
std::unique_ptr<LogicalOperator> last_op, const Tree &combinator,
TPlanningContext *context) {
if (const auto *union_ = dynamic_cast<const CypherUnion *>(&combinator)) {
if (const auto *union_ = utils::Downcast<const CypherUnion>(&combinator)) {
return std::unique_ptr<LogicalOperator>(
impl::GenUnion(*union_, std::move(last_op), std::move(curr_op),
*context->symbol_table));

View File

@ -13,17 +13,17 @@ void ForEachPattern(
std::function<void(NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
auto atoms_it = pattern.atoms_.begin();
auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
auto current_node = utils::Downcast<NodeAtom>(*atoms_it++);
DCHECK(current_node) << "First pattern atom is not a node";
base(current_node);
// Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
while (atoms_it != pattern.atoms_.end()) {
auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
auto edge = utils::Downcast<EdgeAtom>(*atoms_it++);
DCHECK(edge) << "Expected an edge atom in pattern.";
DCHECK(atoms_it != pattern.atoms_.end())
<< "Edge atom should not end the pattern.";
auto prev_node = current_node;
current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
current_node = utils::Downcast<NodeAtom>(*atoms_it++);
DCHECK(current_node) << "Expected a node atom in pattern.";
collect(prev_node, edge, current_node);
}
@ -66,7 +66,7 @@ std::vector<Expansion> NormalizePatterns(
};
for (const auto &pattern : patterns) {
if (pattern->atoms_.size() == 1U) {
auto *node = dynamic_cast<NodeAtom *>(pattern->atoms_[0]);
auto *node = utils::Downcast<NodeAtom>(pattern->atoms_[0]);
DCHECK(node) << "First pattern atom is not a node";
expansions.emplace_back(Expansion{node});
} else {
@ -139,7 +139,7 @@ auto SplitExpressionOnAnd(Expression *expression) {
while (!pending_expressions.empty()) {
auto *current_expression = pending_expressions.top();
pending_expressions.pop();
if (auto *and_op = dynamic_cast<AndOperator *>(current_expression)) {
if (auto *and_op = utils::Downcast<AndOperator>(current_expression)) {
pending_expressions.push(and_op->expression1_);
pending_expressions.push(and_op->expression2_);
} else {
@ -332,8 +332,8 @@ void Filters::AnalyzeAndStoreFilter(Expression *expr,
};
auto get_property_lookup = [](auto *maybe_lookup, auto *&prop_lookup,
auto *&ident) -> bool {
return (prop_lookup = dynamic_cast<PropertyLookup *>(maybe_lookup)) &&
(ident = dynamic_cast<Identifier *>(prop_lookup->expression_));
return (prop_lookup = utils::Downcast<PropertyLookup>(maybe_lookup)) &&
(ident = utils::Downcast<Identifier>(prop_lookup->expression_));
};
// Checks if maybe_lookup is a property lookup, stores it as a
// PropertyFilter and returns true. If it isn't, returns false.
@ -379,19 +379,19 @@ void Filters::AnalyzeAndStoreFilter(Expression *expr,
};
// We are only interested to see the insides of And, because Or prevents
// indexing since any labels and properties found there may be optional.
DCHECK(!dynamic_cast<AndOperator *>(expr))
DCHECK(!utils::IsSubtype(*expr, AndOperator::kType))
<< "Expected AndOperators have been split.";
if (auto *labels_test = dynamic_cast<LabelsTest *>(expr)) {
if (auto *labels_test = utils::Downcast<LabelsTest>(expr)) {
// Since LabelsTest may contain any expression, we can only use the
// simplest test on an identifier.
if (dynamic_cast<Identifier *>(labels_test->expression_)) {
if (utils::Downcast<Identifier>(labels_test->expression_)) {
auto filter = make_filter(FilterInfo::Type::Label);
filter.labels = labels_test->labels_;
all_filters_.emplace_back(filter);
} else {
all_filters_.emplace_back(make_filter(FilterInfo::Type::Generic));
}
} else if (auto *eq = dynamic_cast<EqualOperator *>(expr)) {
} else if (auto *eq = utils::Downcast<EqualOperator>(expr)) {
// Try to get property equality test from the top expressions.
// Unfortunately, we cannot go deeper inside Equal, because chained equals
// need not correspond to And. For example, `(n.prop = value) = false)`:
@ -410,23 +410,23 @@ void Filters::AnalyzeAndStoreFilter(Expression *expr,
// No PropertyFilter was added, so just store a generic filter.
all_filters_.emplace_back(make_filter(FilterInfo::Type::Generic));
}
} else if (auto *gt = dynamic_cast<GreaterOperator *>(expr)) {
} else if (auto *gt = utils::Downcast<GreaterOperator>(expr)) {
if (!add_prop_greater(gt->expression1_, gt->expression2_,
Bound::Type::EXCLUSIVE)) {
all_filters_.emplace_back(make_filter(FilterInfo::Type::Generic));
}
} else if (auto *ge = dynamic_cast<GreaterEqualOperator *>(expr)) {
} else if (auto *ge = utils::Downcast<GreaterEqualOperator>(expr)) {
if (!add_prop_greater(ge->expression1_, ge->expression2_,
Bound::Type::INCLUSIVE)) {
all_filters_.emplace_back(make_filter(FilterInfo::Type::Generic));
}
} else if (auto *lt = dynamic_cast<LessOperator *>(expr)) {
} else if (auto *lt = utils::Downcast<LessOperator>(expr)) {
// Like greater, but in reverse.
if (!add_prop_greater(lt->expression2_, lt->expression1_,
Bound::Type::EXCLUSIVE)) {
all_filters_.emplace_back(make_filter(FilterInfo::Type::Generic));
}
} else if (auto *le = dynamic_cast<LessEqualOperator *>(expr)) {
} else if (auto *le = utils::Downcast<LessEqualOperator>(expr)) {
// Like greater equal, but in reverse.
if (!add_prop_greater(le->expression2_, le->expression1_,
Bound::Type::INCLUSIVE)) {
@ -448,7 +448,7 @@ std::vector<SingleQueryPart> CollectSingleQueryParts(
std::vector<SingleQueryPart> query_parts(1);
auto *query_part = &query_parts.back();
for (auto &clause : single_query->clauses_) {
if (auto *match = dynamic_cast<Match *>(clause)) {
if (auto *match = utils::Downcast<Match>(clause)) {
if (match->optional_) {
query_part->optional_matching.emplace_back(Matching{});
AddMatching(*match, symbol_table, storage,
@ -460,16 +460,16 @@ std::vector<SingleQueryPart> CollectSingleQueryParts(
}
} else {
query_part->remaining_clauses.push_back(clause);
if (auto *merge = dynamic_cast<query::Merge *>(clause)) {
if (auto *merge = utils::Downcast<query::Merge>(clause)) {
query_part->merge_matching.emplace_back(Matching{});
AddMatching({merge->pattern_}, nullptr, symbol_table, storage,
query_part->merge_matching.back());
} else if (dynamic_cast<With *>(clause) ||
dynamic_cast<query::Unwind *>(clause)) {
} else if (utils::IsSubtype(*clause, With::kType) ||
utils::IsSubtype(*clause, query::Unwind::kType)) {
// This query part is done, continue with a new one.
query_parts.emplace_back(SingleQueryPart{});
query_part = &query_parts.back();
} else if (dynamic_cast<Return *>(clause)) {
} else if (utils::IsSubtype(*clause, Return::kType)) {
return query_parts;
}
}

View File

@ -98,17 +98,17 @@ auto ReducePattern(
std::function<T(T, NodeAtom *, EdgeAtom *, NodeAtom *)> collect) {
DCHECK(!pattern.atoms_.empty()) << "Missing atoms in pattern";
auto atoms_it = pattern.atoms_.begin();
auto current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
auto current_node = utils::Downcast<NodeAtom>(*atoms_it++);
DCHECK(current_node) << "First pattern atom is not a node";
auto last_res = base(current_node);
// Remaining atoms need to follow sequentially as (EdgeAtom, NodeAtom)*
while (atoms_it != pattern.atoms_.end()) {
auto edge = dynamic_cast<EdgeAtom *>(*atoms_it++);
auto edge = utils::Downcast<EdgeAtom>(*atoms_it++);
DCHECK(edge) << "Expected an edge atom in pattern.";
DCHECK(atoms_it != pattern.atoms_.end())
<< "Edge atom should not end the pattern.";
auto prev_node = current_node;
current_node = dynamic_cast<NodeAtom *>(*atoms_it++);
current_node = utils::Downcast<NodeAtom>(*atoms_it++);
DCHECK(current_node) << "Expected a node atom in pattern.";
last_res = collect(std::move(last_res), prev_node, edge, current_node);
}
@ -180,20 +180,20 @@ class RuleBasedPlanner {
}
}
int merge_id = 0;
for (auto &clause : query_part.remaining_clauses) {
DCHECK(!dynamic_cast<Match *>(clause))
for (auto *clause : query_part.remaining_clauses) {
DCHECK(!utils::IsSubtype(*clause, Match::kType))
<< "Unexpected Match in remaining clauses";
if (auto *ret = dynamic_cast<Return *>(clause)) {
if (auto *ret = utils::Downcast<Return>(clause)) {
input_op = impl::GenReturn(
*ret, std::move(input_op), *context.symbol_table, is_write,
context.bound_symbols, *context.ast_storage);
} else if (auto *merge = dynamic_cast<query::Merge *>(clause)) {
} else if (auto *merge = utils::Downcast<query::Merge>(clause)) {
input_op = GenMerge(*merge, std::move(input_op),
query_part.merge_matching[merge_id++]);
// Treat MERGE clause as write, because we do not know if it will
// create anything.
is_write = true;
} else if (auto *with = dynamic_cast<query::With *>(clause)) {
} else if (auto *with = utils::Downcast<query::With>(clause)) {
input_op = impl::GenWith(*with, std::move(input_op),
*context.symbol_table, is_write,
context.bound_symbols, *context.ast_storage);
@ -204,7 +204,7 @@ class RuleBasedPlanner {
context.bound_symbols)) {
is_write = true;
input_op = std::move(op);
} else if (auto *unwind = dynamic_cast<query::Unwind *>(clause)) {
} else if (auto *unwind = utils::Downcast<query::Unwind>(clause)) {
const auto &symbol =
context.symbol_table->at(*unwind->named_expression_);
context.bound_symbols.insert(symbol);
@ -336,23 +336,23 @@ class RuleBasedPlanner {
Clause *clause, std::unique_ptr<LogicalOperator> &input_op,
const SymbolTable &symbol_table,
std::unordered_set<Symbol> &bound_symbols) {
if (auto *create = dynamic_cast<Create *>(clause)) {
if (auto *create = utils::Downcast<Create>(clause)) {
return GenCreate(*create, std::move(input_op), symbol_table,
bound_symbols);
} else if (auto *del = dynamic_cast<query::Delete *>(clause)) {
} else if (auto *del = utils::Downcast<query::Delete>(clause)) {
return std::make_unique<plan::Delete>(std::move(input_op),
del->expressions_, del->detach_);
} else if (auto *set = dynamic_cast<query::SetProperty *>(clause)) {
} else if (auto *set = utils::Downcast<query::SetProperty>(clause)) {
return std::make_unique<plan::SetProperty>(
std::move(input_op), GetProperty(set->property_lookup_->property_),
set->property_lookup_, set->expression_);
} else if (auto *set = dynamic_cast<query::SetProperties *>(clause)) {
} else if (auto *set = utils::Downcast<query::SetProperties>(clause)) {
auto op = set->update_ ? plan::SetProperties::Op::UPDATE
: plan::SetProperties::Op::REPLACE;
const auto &input_symbol = symbol_table.at(*set->identifier_);
return std::make_unique<plan::SetProperties>(
std::move(input_op), input_symbol, set->expression_, op);
} else if (auto *set = dynamic_cast<query::SetLabels *>(clause)) {
} else if (auto *set = utils::Downcast<query::SetLabels>(clause)) {
const auto &input_symbol = symbol_table.at(*set->identifier_);
std::vector<storage::Label> labels;
labels.reserve(set->labels_.size());
@ -361,11 +361,11 @@ class RuleBasedPlanner {
}
return std::make_unique<plan::SetLabels>(std::move(input_op),
input_symbol, labels);
} else if (auto *rem = dynamic_cast<query::RemoveProperty *>(clause)) {
} else if (auto *rem = utils::Downcast<query::RemoveProperty>(clause)) {
return std::make_unique<plan::RemoveProperty>(
std::move(input_op), GetProperty(rem->property_lookup_->property_),
rem->property_lookup_);
} else if (auto *rem = dynamic_cast<query::RemoveLabels *>(clause)) {
} else if (auto *rem = utils::Downcast<query::RemoveLabels>(clause)) {
const auto &input_symbol = symbol_table.at(*rem->identifier_);
std::vector<storage::Label> labels;
labels.reserve(rem->labels_.size());