mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #39138 from nickitat/control_block_size_in_aggregator
Control block size in aggregator
This commit is contained in:
commit
193cd1b3b2
113
src/Interpreters/AggregationUtils.cpp
Normal file
113
src/Interpreters/AggregationUtils.cpp
Normal file
@ -0,0 +1,113 @@
|
||||
#include <Interpreters/AggregationUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
OutputBlockColumns prepareOutputBlockColumns(
|
||||
const Aggregator::Params & params,
|
||||
const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions,
|
||||
const Block & res_header,
|
||||
Arenas & aggregates_pools,
|
||||
bool final,
|
||||
size_t rows)
|
||||
{
|
||||
MutableColumns key_columns(params.keys_size);
|
||||
MutableColumns aggregate_columns(params.aggregates_size);
|
||||
MutableColumns final_aggregate_columns(params.aggregates_size);
|
||||
Aggregator::AggregateColumnsData aggregate_columns_data(params.aggregates_size);
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = res_header.safeGetByPosition(i).type->createColumn();
|
||||
key_columns[i]->reserve(rows);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
{
|
||||
if (!final)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = res_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]);
|
||||
|
||||
for (auto & pool : aggregates_pools)
|
||||
column_aggregate_func.addArena(pool);
|
||||
|
||||
aggregate_columns_data[i] = &column_aggregate_func.getData();
|
||||
aggregate_columns_data[i]->reserve(rows);
|
||||
}
|
||||
else
|
||||
{
|
||||
final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn();
|
||||
final_aggregate_columns[i]->reserve(rows);
|
||||
|
||||
if (aggregate_functions[i]->isState())
|
||||
{
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
|
||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(final_aggregate_columns[i].get()))
|
||||
for (auto & pool : aggregates_pools)
|
||||
column_aggregate_func->addArena(pool);
|
||||
|
||||
/// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator.
|
||||
final_aggregate_columns[i]->forEachSubcolumn(
|
||||
[&aggregates_pools](auto & subcolumn)
|
||||
{
|
||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(subcolumn.get()))
|
||||
for (auto & pool : aggregates_pools)
|
||||
column_aggregate_func->addArena(pool);
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (key_columns.size() != params.keys_size)
|
||||
throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
std::vector<IColumn *> raw_key_columns;
|
||||
raw_key_columns.reserve(key_columns.size());
|
||||
for (auto & column : key_columns)
|
||||
raw_key_columns.push_back(column.get());
|
||||
|
||||
return {
|
||||
.key_columns = std::move(key_columns),
|
||||
.raw_key_columns = std::move(raw_key_columns),
|
||||
.aggregate_columns = std::move(aggregate_columns),
|
||||
.final_aggregate_columns = std::move(final_aggregate_columns),
|
||||
.aggregate_columns_data = std::move(aggregate_columns_data),
|
||||
};
|
||||
}
|
||||
|
||||
Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows)
|
||||
{
|
||||
auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols;
|
||||
|
||||
Block res = res_header.cloneEmpty();
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
res.getByPosition(i).column = std::move(key_columns[i]);
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
if (final)
|
||||
res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]);
|
||||
else
|
||||
res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]);
|
||||
}
|
||||
|
||||
/// Change the size of the columns-constants in the block.
|
||||
size_t columns = res_header.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
if (isColumnConst(*res.getByPosition(i).column))
|
||||
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
|
||||
|
||||
return res;
|
||||
}
|
||||
}
|
27
src/Interpreters/AggregationUtils.h
Normal file
27
src/Interpreters/AggregationUtils.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Aggregator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct OutputBlockColumns
|
||||
{
|
||||
MutableColumns key_columns;
|
||||
std::vector<IColumn *> raw_key_columns;
|
||||
MutableColumns aggregate_columns;
|
||||
MutableColumns final_aggregate_columns;
|
||||
Aggregator::AggregateColumnsData aggregate_columns_data;
|
||||
};
|
||||
|
||||
|
||||
OutputBlockColumns prepareOutputBlockColumns(
|
||||
const Aggregator::Params & params,
|
||||
const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions,
|
||||
const Block & res_header,
|
||||
Arenas & aggregates_pools,
|
||||
bool final,
|
||||
size_t rows);
|
||||
|
||||
Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows);
|
||||
}
|
@ -34,6 +34,8 @@
|
||||
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <Interpreters/AggregationUtils.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ExternalAggregationWritePart;
|
||||
@ -1587,16 +1589,10 @@ Block Aggregator::convertOneBucketToBlock(
|
||||
bool final,
|
||||
size_t bucket) const
|
||||
{
|
||||
Block block = prepareBlockAndFill(data_variants, final, method.data.impls[bucket].size(),
|
||||
[bucket, &method, arena, this] (
|
||||
MutableColumns & key_columns,
|
||||
AggregateColumnsData & aggregate_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
bool final_)
|
||||
{
|
||||
convertToBlockImpl(method, method.data.impls[bucket],
|
||||
key_columns, aggregate_columns, final_aggregate_columns, arena, final_);
|
||||
});
|
||||
// Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id).
|
||||
constexpr bool return_single_block = true;
|
||||
Block block = convertToBlockImpl<return_single_block>(
|
||||
method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size());
|
||||
|
||||
block.info.bucket_num = bucket;
|
||||
return block;
|
||||
@ -1702,26 +1698,17 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
|
||||
}
|
||||
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void Aggregator::convertToBlockImpl(
|
||||
Method & method,
|
||||
Table & data,
|
||||
MutableColumns & key_columns,
|
||||
AggregateColumnsData & aggregate_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena,
|
||||
bool final) const
|
||||
template <bool return_single_block, typename Method, typename Table>
|
||||
Aggregator::ConvertToBlockRes<return_single_block>
|
||||
Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const
|
||||
{
|
||||
if (data.empty())
|
||||
return;
|
||||
{
|
||||
auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows);
|
||||
return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)};
|
||||
}
|
||||
|
||||
if (key_columns.size() != params.keys_size)
|
||||
throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
std::vector<IColumn *> raw_key_columns;
|
||||
raw_key_columns.reserve(key_columns.size());
|
||||
for (auto & column : key_columns)
|
||||
raw_key_columns.push_back(column.get());
|
||||
ConvertToBlockRes<return_single_block> res;
|
||||
|
||||
if (final)
|
||||
{
|
||||
@ -1729,20 +1716,23 @@ void Aggregator::convertToBlockImpl(
|
||||
if (compiled_aggregate_functions_holder)
|
||||
{
|
||||
static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization;
|
||||
convertToBlockImplFinal<Method, use_compiled_functions>(method, data, std::move(raw_key_columns), final_aggregate_columns, arena);
|
||||
res = convertToBlockImplFinal<Method, use_compiled_functions, return_single_block>(method, data, arena, aggregates_pools, rows);
|
||||
}
|
||||
else
|
||||
#endif
|
||||
{
|
||||
convertToBlockImplFinal<Method, false>(method, data, std::move(raw_key_columns), final_aggregate_columns, arena);
|
||||
res = convertToBlockImplFinal<Method, false, return_single_block>(method, data, arena, aggregates_pools, rows);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
convertToBlockImplNotFinal(method, data, std::move(raw_key_columns), aggregate_columns);
|
||||
res = convertToBlockImplNotFinal<return_single_block>(method, data, aggregates_pools, rows);
|
||||
}
|
||||
|
||||
/// In order to release memory early.
|
||||
data.clearAndShrink();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
@ -1811,38 +1801,9 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu
|
||||
}
|
||||
|
||||
|
||||
template <typename Method, bool use_compiled_functions, typename Table>
|
||||
void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
Method & method,
|
||||
Table & data,
|
||||
std::vector<IColumn *> key_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena) const
|
||||
template <bool use_compiled_functions>
|
||||
Block Aggregator::insertResultsIntoColumns(PaddedPODArray<AggregateDataPtr> & places, OutputBlockColumns && out_cols, Arena * arena) const
|
||||
{
|
||||
if constexpr (Method::low_cardinality_optimization)
|
||||
{
|
||||
if (data.hasNullKeyData())
|
||||
{
|
||||
key_columns[0]->insertDefault();
|
||||
insertAggregatesIntoColumns(data.getNullKeyData(), final_aggregate_columns, arena);
|
||||
}
|
||||
}
|
||||
|
||||
auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes);
|
||||
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
|
||||
|
||||
PaddedPODArray<AggregateDataPtr> places;
|
||||
places.reserve(data.size());
|
||||
|
||||
data.forEachValue([&](const auto & key, auto & mapped)
|
||||
{
|
||||
method.insertKeyIntoColumns(key, key_columns, key_sizes_ref);
|
||||
places.emplace_back(mapped);
|
||||
|
||||
/// Mark the cell as destroyed so it will not be destroyed in destructor.
|
||||
mapped = nullptr;
|
||||
});
|
||||
|
||||
std::exception_ptr exception;
|
||||
size_t aggregate_functions_destroy_index = 0;
|
||||
|
||||
@ -1863,7 +1824,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
if (!is_aggregate_function_compiled[i])
|
||||
continue;
|
||||
|
||||
auto & final_aggregate_column = final_aggregate_columns[i];
|
||||
auto & final_aggregate_column = out_cols.final_aggregate_columns[i];
|
||||
final_aggregate_column = final_aggregate_column->cloneResized(places.size());
|
||||
columns_data.emplace_back(getColumnData(final_aggregate_column.get()));
|
||||
}
|
||||
@ -1884,7 +1845,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
}
|
||||
}
|
||||
|
||||
auto & final_aggregate_column = final_aggregate_columns[aggregate_functions_destroy_index];
|
||||
auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index];
|
||||
size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
|
||||
|
||||
/** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch
|
||||
@ -1898,7 +1859,8 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
bool is_state = aggregate_functions[destroy_index]->isState();
|
||||
bool destroy_place_after_insert = !is_state;
|
||||
|
||||
aggregate_functions[destroy_index]->insertResultIntoBatch(0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert);
|
||||
aggregate_functions[destroy_index]->insertResultIntoBatch(
|
||||
0, places.size(), places.data(), offset, *final_aggregate_column, arena, destroy_place_after_insert);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
@ -1923,125 +1885,155 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
|
||||
if (exception)
|
||||
std::rethrow_exception(exception);
|
||||
|
||||
return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size());
|
||||
}
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void NO_INLINE Aggregator::convertToBlockImplNotFinal(
|
||||
Method & method,
|
||||
Table & data,
|
||||
std::vector<IColumn *> key_columns,
|
||||
AggregateColumnsData & aggregate_columns) const
|
||||
template <typename Method, bool use_compiled_functions, bool return_single_block, typename Table>
|
||||
Aggregator::ConvertToBlockRes<return_single_block> NO_INLINE
|
||||
Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const
|
||||
{
|
||||
if constexpr (Method::low_cardinality_optimization)
|
||||
const size_t max_block_size = params.max_block_size;
|
||||
const bool final = true;
|
||||
ConvertToBlockRes<return_single_block> res;
|
||||
|
||||
std::optional<OutputBlockColumns> out_cols;
|
||||
std::optional<Sizes> shuffled_key_sizes;
|
||||
PaddedPODArray<AggregateDataPtr> places;
|
||||
|
||||
auto init_out_cols = [&]()
|
||||
{
|
||||
if (data.hasNullKeyData())
|
||||
out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size);
|
||||
|
||||
if constexpr (Method::low_cardinality_optimization)
|
||||
{
|
||||
key_columns[0]->insertDefault();
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]);
|
||||
|
||||
data.getNullKeyData() = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes);
|
||||
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
|
||||
|
||||
data.forEachValue([&](const auto & key, auto & mapped)
|
||||
{
|
||||
method.insertKeyIntoColumns(key, key_columns, key_sizes_ref);
|
||||
|
||||
/// reserved, so push_back does not throw exceptions
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i]->push_back(mapped + offsets_of_aggregate_states[i]);
|
||||
|
||||
mapped = nullptr;
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
template <typename Filler>
|
||||
Block Aggregator::prepareBlockAndFill(
|
||||
AggregatedDataVariants & data_variants,
|
||||
bool final,
|
||||
size_t rows,
|
||||
Filler && filler) const
|
||||
{
|
||||
MutableColumns key_columns(params.keys_size);
|
||||
MutableColumns aggregate_columns(params.aggregates_size);
|
||||
MutableColumns final_aggregate_columns(params.aggregates_size);
|
||||
AggregateColumnsData aggregate_columns_data(params.aggregates_size);
|
||||
|
||||
Block res_header = getHeader(final);
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = res_header.safeGetByPosition(i).type->createColumn();
|
||||
key_columns[i]->reserve(rows);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
{
|
||||
if (!final)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
aggregate_columns[i] = res_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]);
|
||||
|
||||
for (auto & pool : data_variants.aggregates_pools)
|
||||
column_aggregate_func.addArena(pool);
|
||||
|
||||
aggregate_columns_data[i] = &column_aggregate_func.getData();
|
||||
aggregate_columns_data[i]->reserve(rows);
|
||||
}
|
||||
else
|
||||
{
|
||||
final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn();
|
||||
final_aggregate_columns[i]->reserve(rows);
|
||||
|
||||
if (aggregate_functions[i]->isState())
|
||||
if (data.hasNullKeyData())
|
||||
{
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
|
||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(final_aggregate_columns[i].get()))
|
||||
for (auto & pool : data_variants.aggregates_pools)
|
||||
column_aggregate_func->addArena(pool);
|
||||
|
||||
/// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator.
|
||||
final_aggregate_columns[i]->forEachSubcolumn([&data_variants](auto & subcolumn)
|
||||
{
|
||||
if (auto * column_aggregate_func = typeid_cast<ColumnAggregateFunction *>(subcolumn.get()))
|
||||
for (auto & pool : data_variants.aggregates_pools)
|
||||
column_aggregate_func->addArena(pool);
|
||||
});
|
||||
out_cols->key_columns[0]->insertDefault();
|
||||
insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena);
|
||||
data.hasNullKeyData() = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
filler(key_columns, aggregate_columns_data, final_aggregate_columns, final);
|
||||
shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
|
||||
|
||||
Block res = res_header.cloneEmpty();
|
||||
places.reserve(max_block_size);
|
||||
};
|
||||
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
res.getByPosition(i).column = std::move(key_columns[i]);
|
||||
// should be invoked at least once, because null data might be the only content of the `data`
|
||||
init_out_cols();
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
data.forEachValue(
|
||||
[&](const auto & key, auto & mapped)
|
||||
{
|
||||
if (!out_cols.has_value())
|
||||
init_out_cols();
|
||||
|
||||
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
|
||||
method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref);
|
||||
places.emplace_back(mapped);
|
||||
|
||||
/// Mark the cell as destroyed so it will not be destroyed in destructor.
|
||||
mapped = nullptr;
|
||||
|
||||
if constexpr (!return_single_block)
|
||||
{
|
||||
if (places.size() >= max_block_size)
|
||||
{
|
||||
res.emplace_back(insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena));
|
||||
places.clear();
|
||||
out_cols.reset();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
if constexpr (return_single_block)
|
||||
{
|
||||
const auto & aggregate_column_name = params.aggregates[i].column_name;
|
||||
if (final)
|
||||
res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]);
|
||||
else
|
||||
res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]);
|
||||
return insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (out_cols.has_value())
|
||||
res.emplace_back(insertResultsIntoColumns<use_compiled_functions>(places, std::move(out_cols.value()), arena));
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
/// Change the size of the columns-constants in the block.
|
||||
size_t columns = res_header.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
if (isColumnConst(*res.getByPosition(i).column))
|
||||
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
|
||||
template <bool return_single_block, typename Method, typename Table>
|
||||
Aggregator::ConvertToBlockRes<return_single_block> NO_INLINE
|
||||
Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const
|
||||
{
|
||||
const size_t max_block_size = params.max_block_size;
|
||||
const bool final = false;
|
||||
ConvertToBlockRes<return_single_block> res;
|
||||
|
||||
std::optional<OutputBlockColumns> out_cols;
|
||||
std::optional<Sizes> shuffled_key_sizes;
|
||||
|
||||
auto init_out_cols = [&]()
|
||||
{
|
||||
out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size);
|
||||
|
||||
if constexpr (Method::low_cardinality_optimization)
|
||||
{
|
||||
if (data.hasNullKeyData())
|
||||
{
|
||||
out_cols->raw_key_columns[0]->insertDefault();
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]);
|
||||
|
||||
data.getNullKeyData() = nullptr;
|
||||
data.hasNullKeyData() = false;
|
||||
}
|
||||
}
|
||||
|
||||
shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
|
||||
};
|
||||
|
||||
// should be invoked at least once, because null data might be the only content of the `data`
|
||||
init_out_cols();
|
||||
|
||||
size_t rows_in_current_block = 0;
|
||||
|
||||
data.forEachValue(
|
||||
[&](const auto & key, auto & mapped)
|
||||
{
|
||||
if (!out_cols.has_value())
|
||||
init_out_cols();
|
||||
|
||||
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes;
|
||||
method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref);
|
||||
|
||||
/// reserved, so push_back does not throw exceptions
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
out_cols->aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]);
|
||||
|
||||
mapped = nullptr;
|
||||
|
||||
++rows_in_current_block;
|
||||
|
||||
if constexpr (!return_single_block)
|
||||
{
|
||||
if (rows_in_current_block >= max_block_size)
|
||||
{
|
||||
res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols.value()), final, rows_in_current_block));
|
||||
out_cols.reset();
|
||||
rows_in_current_block = 0;
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
if constexpr (return_single_block)
|
||||
{
|
||||
return finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (rows_in_current_block)
|
||||
res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block));
|
||||
return res;
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -2105,39 +2097,35 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey(
|
||||
Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const
|
||||
{
|
||||
size_t rows = 1;
|
||||
auto && out_cols
|
||||
= prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows);
|
||||
auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols;
|
||||
|
||||
auto filler = [&data_variants, this](
|
||||
MutableColumns & key_columns,
|
||||
AggregateColumnsData & aggregate_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
bool final_)
|
||||
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
||||
{
|
||||
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
||||
AggregatedDataWithoutKey & data = data_variants.without_key;
|
||||
|
||||
if (!data)
|
||||
throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!final)
|
||||
{
|
||||
AggregatedDataWithoutKey & data = data_variants.without_key;
|
||||
|
||||
if (!data)
|
||||
throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!final_)
|
||||
{
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]);
|
||||
data = nullptr;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
|
||||
insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool);
|
||||
}
|
||||
|
||||
if (params.overflow_row)
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i]->insertDefault();
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]);
|
||||
data = nullptr;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
|
||||
insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool);
|
||||
}
|
||||
};
|
||||
|
||||
Block block = prepareBlockAndFill(data_variants, final, rows, filler);
|
||||
if (params.overflow_row)
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i]->insertDefault();
|
||||
}
|
||||
|
||||
Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows);
|
||||
|
||||
if (is_overflows)
|
||||
block.info.is_overflows = true;
|
||||
@ -2148,29 +2136,22 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
|
||||
return block;
|
||||
}
|
||||
|
||||
Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
|
||||
template <bool return_single_block>
|
||||
Aggregator::ConvertToBlockRes<return_single_block>
|
||||
Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
|
||||
{
|
||||
size_t rows = data_variants.sizeWithoutOverflowRow();
|
||||
const size_t rows = data_variants.sizeWithoutOverflowRow();
|
||||
#define M(NAME) \
|
||||
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
||||
{ \
|
||||
return convertToBlockImpl<return_single_block>( \
|
||||
*data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \
|
||||
}
|
||||
|
||||
auto filler = [&data_variants, this](
|
||||
MutableColumns & key_columns,
|
||||
AggregateColumnsData & aggregate_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
bool final_)
|
||||
{
|
||||
#define M(NAME) \
|
||||
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
||||
convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
|
||||
key_columns, aggregate_columns, final_aggregate_columns, data_variants.aggregates_pool, final_);
|
||||
|
||||
if (false) {} // NOLINT
|
||||
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
||||
#undef M
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
};
|
||||
|
||||
return prepareBlockAndFill(data_variants, final, rows, filler);
|
||||
if (false) {} // NOLINT
|
||||
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
||||
#undef M
|
||||
else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
}
|
||||
|
||||
|
||||
@ -2292,7 +2273,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
|
||||
if (data_variants.type != AggregatedDataVariants::Type::without_key)
|
||||
{
|
||||
if (!data_variants.isTwoLevel())
|
||||
blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final));
|
||||
blocks.splice(blocks.end(), prepareBlockAndFillSingleLevel</* return_single_block */ false>(data_variants, final));
|
||||
else
|
||||
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
|
||||
}
|
||||
@ -3044,9 +3025,15 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
|
||||
Block block;
|
||||
if (result.type == AggregatedDataVariants::Type::without_key || is_overflows)
|
||||
{
|
||||
block = prepareBlockAndFillWithoutKey(result, final, is_overflows);
|
||||
}
|
||||
else
|
||||
block = prepareBlockAndFillSingleLevel(result, final);
|
||||
{
|
||||
// Used during memory efficient merging (SortingAggregatedTransform expects single chunk for each bucket_id).
|
||||
constexpr bool return_single_block = true;
|
||||
block = prepareBlockAndFillSingleLevel<return_single_block>(result, final);
|
||||
}
|
||||
/// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods.
|
||||
|
||||
if (!final)
|
||||
@ -3247,4 +3234,6 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons
|
||||
}
|
||||
|
||||
|
||||
template Aggregator::ConvertToBlockRes<false>
|
||||
Aggregator::prepareBlockAndFillSingleLevel<false>(AggregatedDataVariants & data_variants, bool final) const;
|
||||
}
|
||||
|
@ -1,8 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <memory>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <type_traits>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
@ -872,6 +873,7 @@ using ManyAggregatedDataVariantsPtr = std::shared_ptr<ManyAggregatedDataVariants
|
||||
|
||||
class CompiledAggregateFunctionsHolder;
|
||||
class NativeWriter;
|
||||
struct OutputBlockColumns;
|
||||
|
||||
/** How are "total" values calculated with WITH TOTALS?
|
||||
* (For more details, see TotalsHavingTransform.)
|
||||
@ -933,6 +935,8 @@ public:
|
||||
bool compile_aggregate_expressions;
|
||||
size_t min_count_to_compile_aggregate_expression;
|
||||
|
||||
size_t max_block_size;
|
||||
|
||||
bool only_merge;
|
||||
|
||||
struct StatsCollectingParams
|
||||
@ -969,6 +973,7 @@ public:
|
||||
size_t min_free_disk_space_,
|
||||
bool compile_aggregate_expressions_,
|
||||
size_t min_count_to_compile_aggregate_expression_,
|
||||
size_t max_block_size_,
|
||||
bool only_merge_ = false, // true for projections
|
||||
const StatsCollectingParams & stats_collecting_params_ = {})
|
||||
: keys(keys_)
|
||||
@ -987,15 +992,16 @@ public:
|
||||
, min_free_disk_space(min_free_disk_space_)
|
||||
, compile_aggregate_expressions(compile_aggregate_expressions_)
|
||||
, min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
|
||||
, max_block_size(max_block_size_)
|
||||
, only_merge(only_merge_)
|
||||
, stats_collecting_params(stats_collecting_params_)
|
||||
{
|
||||
}
|
||||
|
||||
/// Only parameters that matter during merge.
|
||||
Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_)
|
||||
Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_)
|
||||
: Params(
|
||||
keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, true, {})
|
||||
keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, max_block_size_, true, {})
|
||||
{
|
||||
}
|
||||
|
||||
@ -1277,15 +1283,12 @@ private:
|
||||
void mergeSingleLevelDataImpl(
|
||||
ManyAggregatedDataVariants & non_empty_data) const;
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void convertToBlockImpl(
|
||||
Method & method,
|
||||
Table & data,
|
||||
MutableColumns & key_columns,
|
||||
AggregateColumnsData & aggregate_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena,
|
||||
bool final) const;
|
||||
template <bool return_single_block>
|
||||
using ConvertToBlockRes = std::conditional_t<return_single_block, Block, BlocksList>;
|
||||
|
||||
template <bool return_single_block, typename Method, typename Table>
|
||||
ConvertToBlockRes<return_single_block>
|
||||
convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const;
|
||||
|
||||
template <typename Mapped>
|
||||
void insertAggregatesIntoColumns(
|
||||
@ -1293,27 +1296,16 @@ private:
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena) const;
|
||||
|
||||
template <typename Method, bool use_compiled_functions, typename Table>
|
||||
void convertToBlockImplFinal(
|
||||
Method & method,
|
||||
Table & data,
|
||||
std::vector<IColumn *> key_columns,
|
||||
MutableColumns & final_aggregate_columns,
|
||||
Arena * arena) const;
|
||||
template <bool use_compiled_functions>
|
||||
Block insertResultsIntoColumns(PaddedPODArray<AggregateDataPtr> & places, OutputBlockColumns && out_cols, Arena * arena) const;
|
||||
|
||||
template <typename Method, typename Table>
|
||||
void convertToBlockImplNotFinal(
|
||||
Method & method,
|
||||
Table & data,
|
||||
std::vector<IColumn *> key_columns,
|
||||
AggregateColumnsData & aggregate_columns) const;
|
||||
template <typename Method, bool use_compiled_functions, bool return_single_block, typename Table>
|
||||
ConvertToBlockRes<return_single_block>
|
||||
convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const;
|
||||
|
||||
template <typename Filler>
|
||||
Block prepareBlockAndFill(
|
||||
AggregatedDataVariants & data_variants,
|
||||
bool final,
|
||||
size_t rows,
|
||||
Filler && filler) const;
|
||||
template <bool return_single_block, typename Method, typename Table>
|
||||
ConvertToBlockRes<return_single_block>
|
||||
convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const;
|
||||
|
||||
template <typename Method>
|
||||
Block convertOneBucketToBlock(
|
||||
@ -1331,9 +1323,11 @@ private:
|
||||
std::atomic<bool> * is_cancelled = nullptr) const;
|
||||
|
||||
Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const;
|
||||
Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const;
|
||||
BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const;
|
||||
|
||||
template <bool return_single_block>
|
||||
ConvertToBlockRes<return_single_block> prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const;
|
||||
|
||||
template <typename Method>
|
||||
BlocksList prepareBlocksAndFillTwoLevelImpl(
|
||||
AggregatedDataVariants & data_variants,
|
||||
|
@ -1763,7 +1763,7 @@ static void executeMergeAggregatedImpl(
|
||||
* but it can work more slowly.
|
||||
*/
|
||||
|
||||
Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads);
|
||||
Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size);
|
||||
|
||||
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
@ -2359,6 +2359,7 @@ static Aggregator::Params getAggregatorParams(
|
||||
settings.min_free_disk_space_for_temporary_data,
|
||||
settings.compile_aggregate_expressions,
|
||||
settings.min_count_to_compile_aggregate_expression,
|
||||
settings.max_block_size,
|
||||
/* only_merge */ false,
|
||||
stats_collecting_params
|
||||
};
|
||||
|
@ -182,6 +182,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
|
||||
transform_params->params.min_free_disk_space,
|
||||
transform_params->params.compile_aggregate_expressions,
|
||||
transform_params->params.min_count_to_compile_aggregate_expression,
|
||||
transform_params->params.max_block_size,
|
||||
/* only_merge */ false,
|
||||
transform_params->params.stats_collecting_params};
|
||||
auto transform_params_for_set = std::make_shared<AggregatingTransformParams>(src_header, std::move(params_for_set), final);
|
||||
@ -376,16 +377,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
|
||||
});
|
||||
|
||||
/// We add the explicit resize here, but not in case of aggregating in order, since AIO don't use two-level hash tables and thus returns only buckets with bucket_number = -1.
|
||||
pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : pipeline.getNumStreams(), true /* force */);
|
||||
pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : params.max_threads, true /* force */);
|
||||
|
||||
aggregating = collector.detachProcessors(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<AggregatingTransform>(header, transform_params);
|
||||
});
|
||||
pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared<AggregatingTransform>(header, transform_params); });
|
||||
|
||||
pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : params.max_threads, false /* force */);
|
||||
|
||||
aggregating = collector.detachProcessors(0);
|
||||
}
|
||||
|
@ -38,7 +38,8 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm(
|
||||
settings.max_threads,
|
||||
settings.min_free_disk_space_for_temporary_data,
|
||||
settings.compile_aggregate_expressions,
|
||||
settings.min_count_to_compile_aggregate_expression);
|
||||
settings.min_count_to_compile_aggregate_expression,
|
||||
settings.max_block_size);
|
||||
|
||||
aggregator = std::make_unique<Aggregator>(header, params);
|
||||
|
||||
|
@ -182,7 +182,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
|
||||
if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes)
|
||||
{
|
||||
if (group_by_key)
|
||||
group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false);
|
||||
group_by_block
|
||||
= params->aggregator.prepareBlockAndFillSingleLevel</* return_single_block */ true>(variants, /* final= */ false);
|
||||
cur_block_bytes += current_memory_usage;
|
||||
finalizeCurrentChunk(std::move(chunk), key_end);
|
||||
return;
|
||||
@ -293,7 +294,8 @@ void AggregatingInOrderTransform::generate()
|
||||
if (cur_block_size && is_consume_finished)
|
||||
{
|
||||
if (group_by_key)
|
||||
group_by_block = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false);
|
||||
group_by_block
|
||||
= params->aggregator.prepareBlockAndFillSingleLevel</* return_single_block */ true>(variants, /* final= */ false);
|
||||
else
|
||||
params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns);
|
||||
variants.invalidate();
|
||||
|
@ -203,7 +203,7 @@ public:
|
||||
{
|
||||
auto & output = outputs.front();
|
||||
|
||||
if (finished && !has_input)
|
||||
if (finished && single_level_chunks.empty())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
@ -230,7 +230,7 @@ public:
|
||||
if (!processors.empty())
|
||||
return Status::ExpandPipeline;
|
||||
|
||||
if (has_input)
|
||||
if (!single_level_chunks.empty())
|
||||
return preparePushToOutput();
|
||||
|
||||
/// Single level case.
|
||||
@ -244,11 +244,14 @@ public:
|
||||
private:
|
||||
IProcessor::Status preparePushToOutput()
|
||||
{
|
||||
auto & output = outputs.front();
|
||||
output.push(std::move(current_chunk));
|
||||
has_input = false;
|
||||
if (single_level_chunks.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Some ready chunks expected");
|
||||
|
||||
if (finished)
|
||||
auto & output = outputs.front();
|
||||
output.push(std::move(single_level_chunks.back()));
|
||||
single_level_chunks.pop_back();
|
||||
|
||||
if (finished && single_level_chunks.empty())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
@ -268,17 +271,17 @@ private:
|
||||
{
|
||||
auto chunk = input.pull();
|
||||
auto bucket = getInfoFromChunk(chunk)->bucket_num;
|
||||
chunks[bucket] = std::move(chunk);
|
||||
two_level_chunks[bucket] = std::move(chunk);
|
||||
}
|
||||
}
|
||||
|
||||
if (!shared_data->is_bucket_processed[current_bucket_num])
|
||||
return Status::NeedData;
|
||||
|
||||
if (!chunks[current_bucket_num])
|
||||
if (!two_level_chunks[current_bucket_num])
|
||||
return Status::NeedData;
|
||||
|
||||
output.push(std::move(chunks[current_bucket_num]));
|
||||
output.push(std::move(two_level_chunks[current_bucket_num]));
|
||||
|
||||
++current_bucket_num;
|
||||
if (current_bucket_num == NUM_BUCKETS)
|
||||
@ -298,27 +301,16 @@ private:
|
||||
size_t num_threads;
|
||||
|
||||
bool is_initialized = false;
|
||||
bool has_input = false;
|
||||
bool finished = false;
|
||||
|
||||
Chunk current_chunk;
|
||||
Chunks single_level_chunks;
|
||||
|
||||
UInt32 current_bucket_num = 0;
|
||||
static constexpr Int32 NUM_BUCKETS = 256;
|
||||
std::array<Chunk, NUM_BUCKETS> chunks;
|
||||
std::array<Chunk, NUM_BUCKETS> two_level_chunks;
|
||||
|
||||
Processors processors;
|
||||
|
||||
void setCurrentChunk(Chunk chunk)
|
||||
{
|
||||
if (has_input)
|
||||
throw Exception("Current chunk was already set in "
|
||||
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
has_input = true;
|
||||
current_chunk = std::move(chunk);
|
||||
}
|
||||
|
||||
void initialize()
|
||||
{
|
||||
is_initialized = true;
|
||||
@ -339,7 +331,7 @@ private:
|
||||
auto block = params->aggregator.prepareBlockAndFillWithoutKey(
|
||||
*first, params->final, first->type != AggregatedDataVariants::Type::without_key);
|
||||
|
||||
setCurrentChunk(convertToChunk(block));
|
||||
single_level_chunks.emplace_back(convertToChunk(block));
|
||||
}
|
||||
}
|
||||
|
||||
@ -364,9 +356,10 @@ private:
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
auto block = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final);
|
||||
auto blocks = params->aggregator.prepareBlockAndFillSingleLevel</* return_single_block */ false>(*first, params->final);
|
||||
for (auto & block : blocks)
|
||||
single_level_chunks.emplace_back(convertToChunk(block));
|
||||
|
||||
setCurrentChunk(convertToChunk(block));
|
||||
finished = true;
|
||||
}
|
||||
|
||||
|
@ -313,6 +313,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
||||
settings.min_free_disk_space_for_temporary_data,
|
||||
settings.compile_aggregate_expressions,
|
||||
settings.min_count_to_compile_aggregate_expression,
|
||||
settings.max_block_size,
|
||||
only_merge);
|
||||
|
||||
return std::make_pair(params, only_merge);
|
||||
|
@ -1,4 +1,8 @@
|
||||
<test>
|
||||
<query>select sipHash64(number) from numbers(1e7) group by number format Null</query>
|
||||
<query>select * from (select * from numbers(1e7) group by number) group by number format Null</query>
|
||||
<query>select * from (select * from numbers(1e7) group by number) order by number format Null</query>
|
||||
|
||||
<query>select * from (select * from numbers_mt(1e7) group by number) group by number format Null</query>
|
||||
<query>select * from (select * from numbers_mt(1e7) group by number) order by number format Null</query>
|
||||
<query>select * from (select * from numbers_mt(1e7) group by number) group by number format Null settings max_bytes_before_external_group_by = 1</query>
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
SET max_threads = 1;
|
||||
|
||||
-- incremental streaming usecase
|
||||
-- that has sense only if data filling order has guarantees of chronological order
|
||||
|
@ -28,7 +28,7 @@ WITH
|
||||
ORDER BY event_time DESC
|
||||
LIMIT 1
|
||||
) AS id
|
||||
SELECT uniqExact(thread_id)
|
||||
SELECT uniqExact(thread_id) > 2
|
||||
FROM system.query_thread_log
|
||||
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings, no-s3-storage
|
||||
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings, no-s3-storage, no-msan
|
||||
# Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently
|
||||
|
||||
# Regression for MemoryTracker that had been incorrectly accounted
|
||||
|
@ -6,4 +6,4 @@
|
||||
2020-01-01 00:00:00 2
|
||||
1
|
||||
499999
|
||||
5
|
||||
18
|
||||
|
@ -1,7 +1,7 @@
|
||||
DROP TABLE IF EXISTS select_final;
|
||||
|
||||
SET do_not_merge_across_partitions_select_final = 1;
|
||||
SET max_threads = 0;
|
||||
SET max_threads = 16;
|
||||
|
||||
CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t);
|
||||
|
||||
|
@ -1,27 +1,12 @@
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
@ -29,32 +14,47 @@
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 50 1 0 49
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
1 50 51 0 1 51
|
||||
|
@ -52,6 +52,7 @@ ALL LEFT JOIN
|
||||
FROM group_bitmap_data_test
|
||||
WHERE pickup_date = '2019-01-01'
|
||||
GROUP BY city_id
|
||||
) AS js2 USING (city_id);
|
||||
) AS js2 USING (city_id)
|
||||
ORDER BY today_users, before_users, ll_users, old_users, new_users, diff_users;
|
||||
|
||||
DROP TABLE IF EXISTS group_bitmap_data_test;
|
||||
|
@ -12,7 +12,7 @@ select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by n
|
||||
-- and the query with GROUP BY on remote servers will first do GROUP BY and then send the block,
|
||||
-- so the initiator will first receive all blocks from remotes and only after start merging,
|
||||
-- and will hit the memory limit.
|
||||
select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='100Mi'; -- { serverError 241 }
|
||||
select * from remote('127.{2..11}', view(select * from numbers(1e6))) group by number order by number limit 1e6 settings distributed_group_by_no_merge=2, max_memory_usage='100Mi', max_block_size=1e12; -- { serverError 241 }
|
||||
|
||||
-- with optimize_aggregation_in_order=1 remote servers will produce blocks more frequently,
|
||||
-- since they don't need to wait until the aggregation will be finished,
|
||||
|
@ -1,5 +1,22 @@
|
||||
-- { echoOn }
|
||||
|
||||
explain pipeline select * from (select * from numbers(1e8) group by number) group by number;
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
(Aggregating)
|
||||
Resize 16 → 16
|
||||
AggregatingTransform × 16
|
||||
StrictResize 16 → 16
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
(Aggregating)
|
||||
Resize 1 → 16
|
||||
AggregatingTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromStorage)
|
||||
Limit
|
||||
Numbers 0 → 1
|
||||
explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number;
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
|
@ -1,9 +1,12 @@
|
||||
set max_threads = 16;
|
||||
set prefer_localhost_replica = 1;
|
||||
set optimize_aggregation_in_order = 0;
|
||||
set max_block_size = 65505;
|
||||
|
||||
-- { echoOn }
|
||||
|
||||
explain pipeline select * from (select * from numbers(1e8) group by number) group by number;
|
||||
|
||||
explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number;
|
||||
|
||||
explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number;
|
||||
|
@ -0,0 +1 @@
|
||||
1
|
@ -0,0 +1,9 @@
|
||||
SET max_block_size = 4213;
|
||||
|
||||
SELECT DISTINCT (blockSize() <= 4213)
|
||||
FROM
|
||||
(
|
||||
SELECT number
|
||||
FROM numbers(100000)
|
||||
GROUP BY number
|
||||
);
|
Loading…
Reference in New Issue
Block a user