mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #10621 from azat/enable_optimize_predicate_expression-order
Fix predicates optimization for distributed queries with HAVING
This commit is contained in:
commit
a8f5b10f15
@ -128,11 +128,11 @@ Block Aggregator::getHeader(bool final) const
|
||||
|
||||
if (final)
|
||||
{
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
for (const auto & aggregate : params.aggregates)
|
||||
{
|
||||
auto & elem = res.getByPosition(params.keys_size + i);
|
||||
auto & elem = res.getByName(aggregate.column_name);
|
||||
|
||||
elem.type = params.aggregates[i].function->getReturnType();
|
||||
elem.type = aggregate.function->getReturnType();
|
||||
elem.column = elem.type->createColumn();
|
||||
}
|
||||
}
|
||||
@ -1060,7 +1060,8 @@ Block Aggregator::prepareBlockAndFill(
|
||||
{
|
||||
if (!final)
|
||||
{
|
||||
aggregate_columns[i] = header.safeGetByPosition(i + params.keys_size).type->createColumn();
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn();
|
||||
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states.
|
||||
ColumnAggregateFunction & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*aggregate_columns[i]);
|
||||
@ -1096,10 +1097,11 @@ Block Aggregator::prepareBlockAndFill(
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
if (final)
|
||||
res.getByPosition(i + params.keys_size).column = std::move(final_aggregate_columns[i]);
|
||||
res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]);
|
||||
else
|
||||
res.getByPosition(i + params.keys_size).column = std::move(aggregate_columns[i]);
|
||||
res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]);
|
||||
}
|
||||
|
||||
/// Change the size of the columns-constants in the block.
|
||||
@ -1824,7 +1826,10 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
key_columns[i] = block.safeGetByPosition(i).column.get();
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.safeGetByPosition(params.keys_size + i).column).getData();
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.getByName(aggregate_column_name).column).getData();
|
||||
}
|
||||
|
||||
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
|
||||
|
||||
@ -1900,7 +1905,10 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
||||
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.safeGetByPosition(params.keys_size + i).column).getData();
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.getByName(aggregate_column_name).column).getData();
|
||||
}
|
||||
|
||||
AggregatedDataWithoutKey & res = result.without_key;
|
||||
if (!res)
|
||||
|
@ -342,11 +342,7 @@ void ExpressionAction::execute(Block & block, bool dry_run, ExtraBlockPtr & not_
|
||||
{
|
||||
ColumnNumbers arguments(argument_names.size());
|
||||
for (size_t i = 0; i < argument_names.size(); ++i)
|
||||
{
|
||||
if (!block.has(argument_names[i]))
|
||||
throw Exception("Not found column: '" + argument_names[i] + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
arguments[i] = block.getPositionByName(argument_names[i]);
|
||||
}
|
||||
|
||||
size_t num_columns_without_result = block.columns();
|
||||
block.insert({ nullptr, result_type, result_name});
|
||||
|
@ -49,13 +49,6 @@ static ASTs splitConjunctionPredicate(const std::initializer_list<const ASTPtr>
|
||||
{
|
||||
std::vector<ASTPtr> res;
|
||||
|
||||
auto remove_expression_at_index = [&res] (const size_t index)
|
||||
{
|
||||
if (index < res.size() - 1)
|
||||
std::swap(res[index], res.back());
|
||||
res.pop_back();
|
||||
};
|
||||
|
||||
for (const auto & predicate : predicates)
|
||||
{
|
||||
if (!predicate)
|
||||
@ -65,14 +58,15 @@ static ASTs splitConjunctionPredicate(const std::initializer_list<const ASTPtr>
|
||||
|
||||
for (size_t idx = 0; idx < res.size();)
|
||||
{
|
||||
const auto & expression = res.at(idx);
|
||||
ASTPtr expression = res.at(idx);
|
||||
|
||||
if (const auto * function = expression->as<ASTFunction>(); function && function->name == "and")
|
||||
{
|
||||
res.erase(res.begin() + idx);
|
||||
|
||||
for (auto & child : function->arguments->children)
|
||||
res.emplace_back(child);
|
||||
|
||||
remove_expression_at_index(idx);
|
||||
continue;
|
||||
}
|
||||
++idx;
|
||||
|
@ -4,7 +4,7 @@ a 2 1 0
|
||||
a 3 1 0
|
||||
b 13 2 0
|
||||
b 15 2 0
|
||||
SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2)
|
||||
SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co2 != 2) AND (co != 0)\n)\nWHERE (co != 0) AND (co2 != 2)
|
||||
1 0 3 1
|
||||
1 0 0 1
|
||||
SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\'
|
||||
@ -26,3 +26,4 @@ SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMa
|
||||
0
|
||||
SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6]
|
||||
1 2 3
|
||||
SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0)
|
||||
|
@ -74,3 +74,13 @@ SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [
|
||||
|
||||
ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6];
|
||||
SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6];
|
||||
|
||||
-- check order is preserved
|
||||
ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0;
|
||||
|
||||
-- from #10613
|
||||
SELECT name, count() AS cnt
|
||||
FROM remote('127.{1,2}', system.settings)
|
||||
GROUP BY name
|
||||
HAVING (max(value) > '9') AND (min(changed) = 0)
|
||||
FORMAT Null;
|
||||
|
Loading…
Reference in New Issue
Block a user