Using std::shared_ptr for functions and AST [#METR-21503].

This commit is contained in:
Alexey Milovidov 2016-05-28 18:46:03 +03:00
parent 8c4abcdd43
commit fc65afb6f0
10 changed files with 16 additions and 16 deletions

View File

@ -269,7 +269,7 @@ private:
auto & distributed_storage = static_cast<TStorageDistributed &>(*subquery_table_storage);
if (sub_select_query.database.isNull())
if (!sub_select_query.database)
{
sub_select_query.database = std::make_shared<ASTIdentifier>({}, distributed_storage.getRemoteDatabaseName(),
ASTIdentifier::Database);
@ -296,7 +296,7 @@ private:
StoragePtr getDistributedSubqueryStorage(const ASTSelectQuery & sub_select_query) const
{
if (sub_select_query.table.isNull())
if (!sub_select_query.table)
return {};
const auto identifier = typeid_cast<const ASTIdentifier *>(sub_select_query.table.get());

View File

@ -32,7 +32,7 @@ public:
/// Переписывает select_expression_list, чтобы вернуть только необходимые столбцы в правильном порядке.
void rewriteSelectExpressionList(const Names & column_names);
bool isUnionAllHead() const { return (prev_union_all == nullptr) && !next_union_all.isNull(); }
bool isUnionAllHead() const { return (prev_union_all == nullptr) && next_union_all != nullptr; }
ASTPtr clone() const override;

View File

@ -71,7 +71,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi
InterpreterSelectQuery * interpreter = this;
ASTPtr tail = query.next_union_all;
while (!tail.isNull())
while (tail)
{
ASTPtr head = tail;

View File

@ -358,11 +358,11 @@ void LogicalExpressionsOptimizer::fixBrokenOrExpressions()
/// Если узел OR был корнем выражения WHERE, PREWHERE или HAVING, то следует обновить этот корень.
/// Из-за того, что имеем дело с направленным ациклическим графом, надо проверить все случаи.
if (!select_query->where_expression.isNull() && (or_function == &*(select_query->where_expression)))
if (select_query->where_expression && (or_function == &*(select_query->where_expression)))
select_query->where_expression = operands[0];
if (!select_query->prewhere_expression.isNull() && (or_function == &*(select_query->prewhere_expression)))
if (select_query->prewhere_expression && (or_function == &*(select_query->prewhere_expression)))
select_query->prewhere_expression = operands[0];
if (!select_query->having_expression.isNull() && (or_function == &*(select_query->having_expression)))
if (select_query->having_expression && (or_function == &*(select_query->having_expression)))
select_query->having_expression = operands[0];
}
}

View File

@ -1233,7 +1233,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query)
auto begin = query.data();
auto end = begin + query.size();
ast = DB::tryParseQuery(parser, begin, end, message, false, "");
return !ast.isNull();
return ast != nullptr;
}
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)

View File

@ -240,7 +240,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query)
auto begin = query.data();
auto end = begin + query.size();
ast = DB::tryParseQuery(parser, begin, end, message, false, "");
return !ast.isNull();
return ast != nullptr;
}
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)

View File

@ -155,7 +155,7 @@ ASTPtr ASTSelectQuery::clone() const
ASTPtr current = ptr;
static_cast<ASTSelectQuery *>(&*current)->prev_union_all = nullptr;
ASTPtr next = static_cast<ASTSelectQuery *>(&*current)->next_union_all;
while (!next.isNull())
while (next != nullptr)
{
ASTSelectQuery * next_select_query = static_cast<ASTSelectQuery *>(&*next);
next_select_query->prev_union_all = current.get();
@ -226,7 +226,7 @@ ASTPtr ASTSelectQuery::cloneImpl(bool traverse_union_all) const
const IAST * ASTSelectQuery::getFormat() const
{
const ASTSelectQuery * query = this;
while (!query->next_union_all.isNull())
while (query->next_union_all != nullptr)
query = static_cast<const ASTSelectQuery *>(query->next_union_all.get());
return query->format.get();
}

View File

@ -305,7 +305,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
if (s_all.ignore(pos, end, max_parsed_pos, expected))
{
if (!select_query->format.isNull())
if (select_query->format)
{
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
expected = "FORMAT only in the last SELECT of the UNION ALL chain";

View File

@ -246,7 +246,7 @@ void StorageDistributed::reshardPartitions(ASTPtr query, const String & database
if (!resharding_worker.isStarted())
throw Exception{"Resharding background thread is not running", ErrorCodes::RESHARDING_NO_WORKER};
if (!coordinator.isNull())
if (coordinator)
throw Exception{"Use of COORDINATE WITH is forbidden in ALTER TABLE ... RESHARD"
" queries for distributed tables",
ErrorCodes::RESHARDING_INVALID_PARAMETERS};
@ -286,9 +286,9 @@ void StorageDistributed::reshardPartitions(ASTPtr query, const String & database
ASTAlterQuery::Parameters & parameters = alter_query.parameters.back();
parameters.type = ASTAlterQuery::RESHARD_PARTITION;
if (!first_partition.isNull())
if (first_partition)
parameters.partition = std::make_shared<ASTLiteral>({}, first_partition);
if (!last_partition.isNull())
if (last_partition)
parameters.last_partition = std::make_shared<ASTLiteral>({}, last_partition);
ASTPtr expr_list = std::make_shared<ASTExpressionList>();

View File

@ -3269,7 +3269,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
if (!resharding_worker.isStarted())
throw Exception{"Resharding background thread is not running", ErrorCodes::RESHARDING_NO_WORKER};
bool has_coordinator = !coordinator.isNull();
bool has_coordinator = coordinator != nullptr;
std::string coordinator_id;
UInt64 block_number = 0;