Merge pull request #39138 from nickitat/control_block_size_in_aggregator

Control block size in aggregator
This commit is contained in:
Alexey Milovidov 2022-09-04 04:51:00 +03:00 committed by GitHub
commit 193cd1b3b2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 482 additions and 325 deletions

View 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;
}
}

View 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);
}

View File

@ -34,6 +34,8 @@
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Interpreters/AggregationUtils.h>
namespace ProfileEvents namespace ProfileEvents
{ {
extern const Event ExternalAggregationWritePart; extern const Event ExternalAggregationWritePart;
@ -1587,16 +1589,10 @@ Block Aggregator::convertOneBucketToBlock(
bool final, bool final,
size_t bucket) const size_t bucket) const
{ {
Block block = prepareBlockAndFill(data_variants, final, method.data.impls[bucket].size(), // Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id).
[bucket, &method, arena, this] ( constexpr bool return_single_block = true;
MutableColumns & key_columns, Block block = convertToBlockImpl<return_single_block>(
AggregateColumnsData & aggregate_columns, method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size());
MutableColumns & final_aggregate_columns,
bool final_)
{
convertToBlockImpl(method, method.data.impls[bucket],
key_columns, aggregate_columns, final_aggregate_columns, arena, final_);
});
block.info.bucket_num = bucket; block.info.bucket_num = bucket;
return block; return block;
@ -1702,26 +1698,17 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
} }
template <typename Method, typename Table> template <bool return_single_block, typename Method, typename Table>
void Aggregator::convertToBlockImpl( Aggregator::ConvertToBlockRes<return_single_block>
Method & method, Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const
Table & data,
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
Arena * arena,
bool final) const
{ {
if (data.empty()) 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) ConvertToBlockRes<return_single_block> res;
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());
if (final) if (final)
{ {
@ -1729,20 +1716,23 @@ void Aggregator::convertToBlockImpl(
if (compiled_aggregate_functions_holder) if (compiled_aggregate_functions_holder)
{ {
static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; 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 else
#endif #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 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. /// In order to release memory early.
data.clearAndShrink(); data.clearAndShrink();
return res;
} }
@ -1811,38 +1801,9 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu
} }
template <typename Method, bool use_compiled_functions, typename Table> template <bool use_compiled_functions>
void NO_INLINE Aggregator::convertToBlockImplFinal( Block Aggregator::insertResultsIntoColumns(PaddedPODArray<AggregateDataPtr> & places, OutputBlockColumns && out_cols, Arena * arena) const
Method & method,
Table & data,
std::vector<IColumn *> key_columns,
MutableColumns & final_aggregate_columns,
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; std::exception_ptr exception;
size_t aggregate_functions_destroy_index = 0; size_t aggregate_functions_destroy_index = 0;
@ -1863,7 +1824,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
if (!is_aggregate_function_compiled[i]) if (!is_aggregate_function_compiled[i])
continue; 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()); final_aggregate_column = final_aggregate_column->cloneResized(places.size());
columns_data.emplace_back(getColumnData(final_aggregate_column.get())); 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]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index];
/** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch /** 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 is_state = aggregate_functions[destroy_index]->isState();
bool destroy_place_after_insert = !is_state; 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 (...) catch (...)
@ -1923,125 +1885,155 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
if (exception) if (exception)
std::rethrow_exception(exception); std::rethrow_exception(exception);
return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size());
} }
template <typename Method, typename Table> template <typename Method, bool use_compiled_functions, bool return_single_block, typename Table>
void NO_INLINE Aggregator::convertToBlockImplNotFinal( Aggregator::ConvertToBlockRes<return_single_block> NO_INLINE
Method & method, Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const
Table & data,
std::vector<IColumn *> key_columns,
AggregateColumnsData & aggregate_columns) const
{ {
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 = [&]()
{
out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size);
if constexpr (Method::low_cardinality_optimization) if constexpr (Method::low_cardinality_optimization)
{ {
if (data.hasNullKeyData()) if (data.hasNullKeyData())
{ {
key_columns[0]->insertDefault(); out_cols->key_columns[0]->insertDefault();
insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena);
data.hasNullKeyData() = false;
}
}
shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
places.reserve(max_block_size);
};
// should be invoked at least once, because null data might be the only content of the `data`
init_out_cols();
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)
{
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;
}
}
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) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]);
data.getNullKeyData() = nullptr; data.getNullKeyData() = nullptr;
data.hasNullKeyData() = false;
} }
} }
auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes);
const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; };
data.forEachValue([&](const auto & key, auto & mapped) // 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)
{ {
method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); 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 /// reserved, so push_back does not throw exceptions
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i]->push_back(mapped + offsets_of_aggregate_states[i]); out_cols->aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]);
mapped = nullptr; 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)
template <typename Filler>
Block Aggregator::prepareBlockAndFill(
AggregatedDataVariants & data_variants,
bool final,
size_t rows,
Filler && filler) const
{ {
MutableColumns key_columns(params.keys_size); return finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block);
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 else
{ {
final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); if (rows_in_current_block)
final_aggregate_columns[i]->reserve(rows); res.emplace_back(finalizeBlock(params, getHeader(final), std::move(out_cols).value(), final, rows_in_current_block));
return res;
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 : 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);
});
} }
}
}
filler(key_columns, aggregate_columns_data, final_aggregate_columns, final);
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; return res;
} }
@ -2105,13 +2097,10 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey(
Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const
{ {
size_t rows = 1; 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; AggregatedDataWithoutKey & data = data_variants.without_key;
@ -2119,10 +2108,10 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
if (!data) if (!data)
throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR); throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR);
if (!final_) if (!final)
{ {
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]); aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]);
data = nullptr; data = nullptr;
} }
else else
@ -2135,9 +2124,8 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
for (size_t i = 0; i < params.keys_size; ++i) for (size_t i = 0; i < params.keys_size; ++i)
key_columns[i]->insertDefault(); key_columns[i]->insertDefault();
} }
};
Block block = prepareBlockAndFill(data_variants, final, rows, filler); Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows);
if (is_overflows) if (is_overflows)
block.info.is_overflows = true; block.info.is_overflows = true;
@ -2148,29 +2136,22 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va
return block; return block;
} }
Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const template <bool return_single_block>
{ Aggregator::ConvertToBlockRes<return_single_block>
size_t rows = data_variants.sizeWithoutOverflowRow(); Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
auto filler = [&data_variants, this](
MutableColumns & key_columns,
AggregateColumnsData & aggregate_columns,
MutableColumns & final_aggregate_columns,
bool final_)
{ {
const size_t rows = data_variants.sizeWithoutOverflowRow();
#define M(NAME) \ #define M(NAME) \
else if (data_variants.type == AggregatedDataVariants::Type::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_); return convertToBlockImpl<return_single_block>( \
*data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \
}
if (false) {} // NOLINT if (false) {} // NOLINT
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
#undef M #undef M
else else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
};
return prepareBlockAndFill(data_variants, final, rows, filler);
} }
@ -2292,7 +2273,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
if (data_variants.type != AggregatedDataVariants::Type::without_key) if (data_variants.type != AggregatedDataVariants::Type::without_key)
{ {
if (!data_variants.isTwoLevel()) if (!data_variants.isTwoLevel())
blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final)); blocks.splice(blocks.end(), prepareBlockAndFillSingleLevel</* return_single_block */ false>(data_variants, final));
else else
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
} }
@ -3044,9 +3025,15 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
Block block; Block block;
if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) if (result.type == AggregatedDataVariants::Type::without_key || is_overflows)
{
block = prepareBlockAndFillWithoutKey(result, final, is_overflows); block = prepareBlockAndFillWithoutKey(result, final, is_overflows);
}
else 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. /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods.
if (!final) 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;
} }

View File

@ -1,8 +1,9 @@
#pragma once #pragma once
#include <mutex>
#include <memory>
#include <functional> #include <functional>
#include <memory>
#include <mutex>
#include <type_traits>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
@ -872,6 +873,7 @@ using ManyAggregatedDataVariantsPtr = std::shared_ptr<ManyAggregatedDataVariants
class CompiledAggregateFunctionsHolder; class CompiledAggregateFunctionsHolder;
class NativeWriter; class NativeWriter;
struct OutputBlockColumns;
/** How are "total" values calculated with WITH TOTALS? /** How are "total" values calculated with WITH TOTALS?
* (For more details, see TotalsHavingTransform.) * (For more details, see TotalsHavingTransform.)
@ -933,6 +935,8 @@ public:
bool compile_aggregate_expressions; bool compile_aggregate_expressions;
size_t min_count_to_compile_aggregate_expression; size_t min_count_to_compile_aggregate_expression;
size_t max_block_size;
bool only_merge; bool only_merge;
struct StatsCollectingParams struct StatsCollectingParams
@ -969,6 +973,7 @@ public:
size_t min_free_disk_space_, size_t min_free_disk_space_,
bool compile_aggregate_expressions_, bool compile_aggregate_expressions_,
size_t min_count_to_compile_aggregate_expression_, size_t min_count_to_compile_aggregate_expression_,
size_t max_block_size_,
bool only_merge_ = false, // true for projections bool only_merge_ = false, // true for projections
const StatsCollectingParams & stats_collecting_params_ = {}) const StatsCollectingParams & stats_collecting_params_ = {})
: keys(keys_) : keys(keys_)
@ -987,15 +992,16 @@ public:
, min_free_disk_space(min_free_disk_space_) , min_free_disk_space(min_free_disk_space_)
, compile_aggregate_expressions(compile_aggregate_expressions_) , compile_aggregate_expressions(compile_aggregate_expressions_)
, min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) , min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_)
, max_block_size(max_block_size_)
, only_merge(only_merge_) , only_merge(only_merge_)
, stats_collecting_params(stats_collecting_params_) , stats_collecting_params(stats_collecting_params_)
{ {
} }
/// Only parameters that matter during merge. /// 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( : 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( void mergeSingleLevelDataImpl(
ManyAggregatedDataVariants & non_empty_data) const; ManyAggregatedDataVariants & non_empty_data) const;
template <typename Method, typename Table> template <bool return_single_block>
void convertToBlockImpl( using ConvertToBlockRes = std::conditional_t<return_single_block, Block, BlocksList>;
Method & method,
Table & data, template <bool return_single_block, typename Method, typename Table>
MutableColumns & key_columns, ConvertToBlockRes<return_single_block>
AggregateColumnsData & aggregate_columns, convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const;
MutableColumns & final_aggregate_columns,
Arena * arena,
bool final) const;
template <typename Mapped> template <typename Mapped>
void insertAggregatesIntoColumns( void insertAggregatesIntoColumns(
@ -1293,27 +1296,16 @@ private:
MutableColumns & final_aggregate_columns, MutableColumns & final_aggregate_columns,
Arena * arena) const; Arena * arena) const;
template <typename Method, bool use_compiled_functions, typename Table> template <bool use_compiled_functions>
void convertToBlockImplFinal( Block insertResultsIntoColumns(PaddedPODArray<AggregateDataPtr> & places, OutputBlockColumns && out_cols, Arena * arena) const;
Method & method,
Table & data,
std::vector<IColumn *> key_columns,
MutableColumns & final_aggregate_columns,
Arena * arena) const;
template <typename Method, typename Table> template <typename Method, bool use_compiled_functions, bool return_single_block, typename Table>
void convertToBlockImplNotFinal( ConvertToBlockRes<return_single_block>
Method & method, convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const;
Table & data,
std::vector<IColumn *> key_columns,
AggregateColumnsData & aggregate_columns) const;
template <typename Filler> template <bool return_single_block, typename Method, typename Table>
Block prepareBlockAndFill( ConvertToBlockRes<return_single_block>
AggregatedDataVariants & data_variants, convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const;
bool final,
size_t rows,
Filler && filler) const;
template <typename Method> template <typename Method>
Block convertOneBucketToBlock( Block convertOneBucketToBlock(
@ -1331,9 +1323,11 @@ private:
std::atomic<bool> * is_cancelled = nullptr) const; std::atomic<bool> * is_cancelled = nullptr) const;
Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) 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; 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> template <typename Method>
BlocksList prepareBlocksAndFillTwoLevelImpl( BlocksList prepareBlocksAndFillTwoLevelImpl(
AggregatedDataVariants & data_variants, AggregatedDataVariants & data_variants,

View File

@ -1763,7 +1763,7 @@ static void executeMergeAggregatedImpl(
* but it can work more slowly. * 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>( auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
@ -2359,6 +2359,7 @@ static Aggregator::Params getAggregatorParams(
settings.min_free_disk_space_for_temporary_data, settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions, settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression, settings.min_count_to_compile_aggregate_expression,
settings.max_block_size,
/* only_merge */ false, /* only_merge */ false,
stats_collecting_params stats_collecting_params
}; };

View File

@ -182,6 +182,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
transform_params->params.min_free_disk_space, transform_params->params.min_free_disk_space,
transform_params->params.compile_aggregate_expressions, transform_params->params.compile_aggregate_expressions,
transform_params->params.min_count_to_compile_aggregate_expression, transform_params->params.min_count_to_compile_aggregate_expression,
transform_params->params.max_block_size,
/* only_merge */ false, /* only_merge */ false,
transform_params->params.stats_collecting_params}; transform_params->params.stats_collecting_params};
auto transform_params_for_set = std::make_shared<AggregatingTransformParams>(src_header, std::move(params_for_set), final); 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. /// 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); aggregating = collector.detachProcessors(0);
} }
else else
{ {
pipeline.addSimpleTransform([&](const Block & header) pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared<AggregatingTransform>(header, transform_params); });
{
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); aggregating = collector.detachProcessors(0);
} }

View File

@ -38,7 +38,8 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm(
settings.max_threads, settings.max_threads,
settings.min_free_disk_space_for_temporary_data, settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions, 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); aggregator = std::make_unique<Aggregator>(header, params);

View File

@ -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 (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes)
{ {
if (group_by_key) 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; cur_block_bytes += current_memory_usage;
finalizeCurrentChunk(std::move(chunk), key_end); finalizeCurrentChunk(std::move(chunk), key_end);
return; return;
@ -293,7 +294,8 @@ void AggregatingInOrderTransform::generate()
if (cur_block_size && is_consume_finished) if (cur_block_size && is_consume_finished)
{ {
if (group_by_key) 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 else
params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns);
variants.invalidate(); variants.invalidate();

View File

@ -203,7 +203,7 @@ public:
{ {
auto & output = outputs.front(); auto & output = outputs.front();
if (finished && !has_input) if (finished && single_level_chunks.empty())
{ {
output.finish(); output.finish();
return Status::Finished; return Status::Finished;
@ -230,7 +230,7 @@ public:
if (!processors.empty()) if (!processors.empty())
return Status::ExpandPipeline; return Status::ExpandPipeline;
if (has_input) if (!single_level_chunks.empty())
return preparePushToOutput(); return preparePushToOutput();
/// Single level case. /// Single level case.
@ -244,11 +244,14 @@ public:
private: private:
IProcessor::Status preparePushToOutput() IProcessor::Status preparePushToOutput()
{ {
auto & output = outputs.front(); if (single_level_chunks.empty())
output.push(std::move(current_chunk)); throw Exception(ErrorCodes::LOGICAL_ERROR, "Some ready chunks expected");
has_input = false;
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(); output.finish();
return Status::Finished; return Status::Finished;
@ -268,17 +271,17 @@ private:
{ {
auto chunk = input.pull(); auto chunk = input.pull();
auto bucket = getInfoFromChunk(chunk)->bucket_num; 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]) if (!shared_data->is_bucket_processed[current_bucket_num])
return Status::NeedData; return Status::NeedData;
if (!chunks[current_bucket_num]) if (!two_level_chunks[current_bucket_num])
return Status::NeedData; return Status::NeedData;
output.push(std::move(chunks[current_bucket_num])); output.push(std::move(two_level_chunks[current_bucket_num]));
++current_bucket_num; ++current_bucket_num;
if (current_bucket_num == NUM_BUCKETS) if (current_bucket_num == NUM_BUCKETS)
@ -298,27 +301,16 @@ private:
size_t num_threads; size_t num_threads;
bool is_initialized = false; bool is_initialized = false;
bool has_input = false;
bool finished = false; bool finished = false;
Chunk current_chunk; Chunks single_level_chunks;
UInt32 current_bucket_num = 0; UInt32 current_bucket_num = 0;
static constexpr Int32 NUM_BUCKETS = 256; static constexpr Int32 NUM_BUCKETS = 256;
std::array<Chunk, NUM_BUCKETS> chunks; std::array<Chunk, NUM_BUCKETS> two_level_chunks;
Processors processors; 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() void initialize()
{ {
is_initialized = true; is_initialized = true;
@ -339,7 +331,7 @@ private:
auto block = params->aggregator.prepareBlockAndFillWithoutKey( auto block = params->aggregator.prepareBlockAndFillWithoutKey(
*first, params->final, first->type != AggregatedDataVariants::Type::without_key); *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 else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); 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; finished = true;
} }

View File

@ -313,6 +313,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
settings.min_free_disk_space_for_temporary_data, settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions, settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression, settings.min_count_to_compile_aggregate_expression,
settings.max_block_size,
only_merge); only_merge);
return std::make_pair(params, only_merge); return std::make_pair(params, only_merge);

View File

@ -1,4 +1,8 @@
<test> <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) 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) 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> <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>

View File

@ -1,4 +1,5 @@
SET joined_subquery_requires_alias = 0; SET joined_subquery_requires_alias = 0;
SET max_threads = 1;
-- incremental streaming usecase -- incremental streaming usecase
-- that has sense only if data filling order has guarantees of chronological order -- that has sense only if data filling order has guarantees of chronological order

View File

@ -28,7 +28,7 @@ WITH
ORDER BY event_time DESC ORDER BY event_time DESC
LIMIT 1 LIMIT 1
) AS id ) AS id
SELECT uniqExact(thread_id) SELECT uniqExact(thread_id) > 2
FROM system.query_thread_log FROM system.query_thread_log
WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id); WHERE (event_date >= (today() - 1)) AND (query_id = id) AND (thread_id != master_thread_id);

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash #!/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 # Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently
# Regression for MemoryTracker that had been incorrectly accounted # Regression for MemoryTracker that had been incorrectly accounted

View File

@ -6,4 +6,4 @@
2020-01-01 00:00:00 2 2020-01-01 00:00:00 2
1 1
499999 499999
5 18

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS select_final; DROP TABLE IF EXISTS select_final;
SET do_not_merge_across_partitions_select_final = 1; 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); CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t);

View File

@ -1,27 +1,12 @@
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 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 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 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 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
@ -29,32 +14,47 @@
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 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 51 0 1 51
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 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 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 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 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 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

View File

@ -52,6 +52,7 @@ ALL LEFT JOIN
FROM group_bitmap_data_test FROM group_bitmap_data_test
WHERE pickup_date = '2019-01-01' WHERE pickup_date = '2019-01-01'
GROUP BY city_id 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; DROP TABLE IF EXISTS group_bitmap_data_test;

View File

@ -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, -- 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, -- so the initiator will first receive all blocks from remotes and only after start merging,
-- and will hit the memory limit. -- 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, -- 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, -- since they don't need to wait until the aggregation will be finished,

View File

@ -1,5 +1,22 @@
-- { echoOn } -- { 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; explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number;
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16

View File

@ -1,9 +1,12 @@
set max_threads = 16; set max_threads = 16;
set prefer_localhost_replica = 1; set prefer_localhost_replica = 1;
set optimize_aggregation_in_order = 0; set optimize_aggregation_in_order = 0;
set max_block_size = 65505;
-- { echoOn } -- { 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) group by number;
explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number; explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number;

View File

@ -0,0 +1,9 @@
SET max_block_size = 4213;
SELECT DISTINCT (blockSize() <= 4213)
FROM
(
SELECT number
FROM numbers(100000)
GROUP BY number
);