Review fixes

This commit is contained in:
Igor Nikonov 2023-03-01 20:19:51 +00:00
parent 0a62f1f5c3
commit e17c0b6bf1
3 changed files with 16 additions and 24 deletions

View File

@ -1,6 +1,7 @@
#include <Planner/PlannerJoinTree.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Functions/FunctionFactory.h>
@ -19,6 +20,8 @@
#include <Analyzer/JoinNode.h>
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/Utils.h>
#include <Analyzer/AggregationUtils.h>
#include <Analyzer/FunctionNode.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/QueryPlan/SortingStep.h>
@ -27,6 +30,7 @@
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/ArrayJoinStep.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Interpreters/Context.h>
#include <Interpreters/IJoin.h>
@ -41,12 +45,8 @@
#include <Planner/Utils.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <Analyzer/AggregationUtils.h>
#include <Analyzer/FunctionNode.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Common/scope_guard_safe.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
namespace DB
{
@ -174,23 +174,17 @@ bool applyTrivialCountIfPossible(
if (!storage || storage->hasLightweightDeletedMask())
return false;
if (settings.max_parallel_replicas > 1 || //
settings.allow_experimental_query_deduplication || //
settings.empty_result_for_aggregation_by_empty_set)
if (settings.max_parallel_replicas > 1 || settings.allow_experimental_query_deduplication
|| settings.empty_result_for_aggregation_by_empty_set)
return false;
QueryTreeNodes aggregates = collectAggregateFunctionNodes(query_tree);
if (aggregates.size() != 1)
return false;
const auto * function_node = typeid_cast<const FunctionNode *>(aggregates.front().get());
if (!function_node)
return false;
if (!function_node->getAggregateFunction())
return false;
const auto * count_func = typeid_cast<const AggregateFunctionCount *>(function_node->getAggregateFunction().get());
const auto & function_node = aggregates.front().get()->as<const FunctionNode &>();
chassert(function_node.getAggregateFunction() != nullptr);
const auto * count_func = typeid_cast<const AggregateFunctionCount *>(function_node.getAggregateFunction().get());
if (!count_func)
return false;
@ -226,7 +220,7 @@ bool applyTrivialCountIfPossible(
SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place));
agg_count.set(place, num_rows.value());
auto column = ColumnAggregateFunction::create(function_node->getAggregateFunction());
auto column = ColumnAggregateFunction::create(function_node.getAggregateFunction());
column->insertFrom(place);
/// get count() argument type
@ -240,7 +234,7 @@ bool applyTrivialCountIfPossible(
Block block_with_count{
{std::move(column),
std::make_shared<DataTypeAggregateFunction>(function_node->getAggregateFunction(), argument_types, Array{}),
std::make_shared<DataTypeAggregateFunction>(function_node.getAggregateFunction(), argument_types, Array{}),
columns_names.front()}};
auto source = std::make_shared<SourceFromSingleChunk>(block_with_count);
@ -416,7 +410,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl
}
}
/// apply trivial_count optimization if possible
/// Apply trivial_count optimization if possible
bool is_trivial_count_applied = is_single_table_expression && table_node && select_query_info.has_aggregates
&& applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), columns_names);

View File

@ -18,8 +18,7 @@ select count(b) from m3;
2
select count() + 1 from m3;
3
-- drop table m3;
drop table m3;
-- checking queries with FINAL
create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b);
SYSTEM STOP MERGES replacing_m3;
@ -39,10 +38,10 @@ select count(a) from replacing_m3;
4
select count(b) from replacing_m3;
4
set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL
select count() from replacing_m3 FINAL;
3
select count(a) from replacing_m3 FINAL;
3
select count(b) from replacing_m3 FINAL;
3
drop table replacing_m3;

View File

@ -19,7 +19,7 @@ select count(a) from m3;
select count(b) from m3;
select count() + 1 from m3;
-- drop table m3;
drop table m3;
-- checking queries with FINAL
create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b);
@ -38,9 +38,8 @@ select count(*) from replacing_m3;
select count(a) from replacing_m3;
select count(b) from replacing_m3;
set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL
select count() from replacing_m3 FINAL;
select count(a) from replacing_m3 FINAL;
select count(b) from replacing_m3 FINAL;
-- drop table replacing_m3;
drop table replacing_m3;