mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Using std::shared_ptr for functions and AST [#METR-21503].
This commit is contained in:
parent
8c4abcdd43
commit
fc65afb6f0
@ -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());
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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];
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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";
|
||||
|
@ -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>();
|
||||
|
@ -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;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user