mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Fix fuzzed query.
This commit is contained in:
parent
b327a93cf7
commit
8f4fb2aa3b
@ -245,6 +245,35 @@ std::optional<AggregateFunctionMatches> matchAggregateFunctions(
|
||||
return res;
|
||||
}
|
||||
|
||||
static void appendAggregateFunctions(
|
||||
ActionsDAG & proj_dag,
|
||||
const AggregateDescriptions & aggregates,
|
||||
const AggregateFunctionMatches & matched_aggregates)
|
||||
{
|
||||
std::unordered_map<const AggregateDescription *, const ActionsDAG::Node *> inputs;
|
||||
|
||||
/// Just add all the aggregates to dag inputs.
|
||||
auto & proj_dag_outputs = proj_dag.getOutputs();
|
||||
size_t num_aggregates = aggregates.size();
|
||||
for (size_t i = 0; i < num_aggregates; ++i)
|
||||
{
|
||||
const auto & aggregate = aggregates[i];
|
||||
const auto & match = matched_aggregates[i];
|
||||
auto type = std::make_shared<DataTypeAggregateFunction>(aggregate.function, match.argument_types, aggregate.parameters);
|
||||
|
||||
auto & input = inputs[match.description];
|
||||
if (!input)
|
||||
input = &proj_dag.addInput(match.description->column_name, std::move(type));
|
||||
|
||||
const auto * node = input;
|
||||
|
||||
if (node->result_name != aggregate.column_name)
|
||||
node = &proj_dag.addAlias(*node, aggregate.column_name);
|
||||
|
||||
proj_dag_outputs.push_back(node);
|
||||
}
|
||||
}
|
||||
|
||||
ActionsDAGPtr analyzeAggregateProjection(
|
||||
const AggregateProjectionInfo & info,
|
||||
const QueryDAG & query,
|
||||
@ -365,23 +394,7 @@ ActionsDAGPtr analyzeAggregateProjection(
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Folding actions by projection");
|
||||
|
||||
auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes);
|
||||
|
||||
/// Just add all the aggregates to dag inputs.
|
||||
auto & proj_dag_outputs = proj_dag->getOutputs();
|
||||
size_t num_aggregates = aggregates.size();
|
||||
for (size_t i = 0; i < num_aggregates; ++i)
|
||||
{
|
||||
const auto & aggregate = aggregates[i];
|
||||
const auto & match = (*matched_aggregates)[i];
|
||||
auto type = std::make_shared<DataTypeAggregateFunction>(aggregate.function, match.argument_types, aggregate.parameters);
|
||||
const auto * node = &proj_dag->addInput(match.description->column_name, std::move(type));
|
||||
|
||||
if (aggregate.column_name != match.description->column_name)
|
||||
node = &proj_dag->addAlias(*node, aggregate.column_name);
|
||||
|
||||
proj_dag_outputs.push_back(node);
|
||||
}
|
||||
|
||||
appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates);
|
||||
return proj_dag;
|
||||
}
|
||||
|
||||
|
@ -0,0 +1,8 @@
|
||||
drop table if exists projection_test__fuzz_0;
|
||||
set allow_suspicious_low_cardinality_types=1;
|
||||
|
||||
CREATE TABLE projection_test__fuzz_0 (`sum(block_count)` UInt64, `domain_alias` UInt64 ALIAS length(domain), `datetime` DateTime, `domain` LowCardinality(String), `x_id` String, `y_id` String, `block_count` Int64, `retry_count` Int64, `duration` Decimal(76, 13), `kbytes` LowCardinality(Int64), `buffer_time` Int64, `first_time` UInt256, `total_bytes` LowCardinality(Nullable(UInt64)), `valid_bytes` Nullable(UInt64), `completed_bytes` Nullable(UInt64), `fixed_bytes` LowCardinality(Nullable(UInt64)), `force_bytes` Int256, PROJECTION p (SELECT toStartOfMinute(datetime) AS dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) GROUP BY dt_m, domain)) ENGINE = MergeTree PARTITION BY toDate(datetime) ORDER BY (toStartOfTenMinutes(datetime), domain) SETTINGS index_granularity_bytes = 10000000;
|
||||
INSERT INTO projection_test__fuzz_0 SETTINGS max_threads = 1 WITH rowNumberInAllBlocks() AS id SELECT 1, toDateTime('2020-10-24 00:00:00') + (id / 20), toString(id % 100), * FROM generateRandom('x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64)', 10, 10, 1) LIMIT 1000 SETTINGS max_threads = 1;
|
||||
SELECT '-21474836.48', 10000000000., '', count(kbytes), '', 10.0001, toStartOfMinute(datetime) AS dt_m, 10, NULL FROM projection_test__fuzz_0 GROUP BY dt_m WITH ROLLUP WITH TOTALS ORDER BY count(retry_count / duration) ASC NULLS LAST, 100000000000000000000. ASC NULLS FIRST format Null;
|
||||
|
||||
drop table projection_test__fuzz_0;
|
Loading…
Reference in New Issue
Block a user