From bca28ba9f8cb25cad2947da535a8fce006185383 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 8 Jul 2022 21:24:33 +0200 Subject: [PATCH 01/39] split prepareBlockAndFill --- src/Interpreters/Aggregator.cpp | 51 +++++++++++++++++++++++---------- src/Interpreters/Aggregator.h | 19 ++++++++++++ 2 files changed, 55 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 6e101005599..e59605890d3 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1951,13 +1951,7 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal( }); } - -template -Block Aggregator::prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const +Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const { MutableColumns key_columns(params.keys_size); MutableColumns aggregate_columns(params.aggregates_size); @@ -1982,7 +1976,7 @@ Block Aggregator::prepareBlockAndFill( /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); - for (auto & pool : data_variants.aggregates_pools) + for (auto & pool : aggregates_pools) column_aggregate_func.addArena(pool); aggregate_columns_data[i] = &column_aggregate_func.getData(); @@ -1997,22 +1991,49 @@ Block Aggregator::prepareBlockAndFill( { /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) - for (auto & pool : data_variants.aggregates_pools) + 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([&data_variants](auto & subcolumn) - { - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) - for (auto & pool : data_variants.aggregates_pools) - column_aggregate_func->addArena(pool); - }); + final_aggregate_columns[i]->forEachSubcolumn( + [&aggregates_pools](auto & subcolumn) + { + if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + }); } } } + return { + .key_columns = std::move(key_columns), + .aggregate_columns = std::move(aggregate_columns), + .final_aggregate_columns = std::move(final_aggregate_columns), + .aggregate_columns_data = std::move(aggregate_columns_data), + }; +} + +template +Block Aggregator::prepareBlockAndFill( + AggregatedDataVariants & data_variants, + bool final, + size_t rows, + Filler && filler) const +{ + auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); + return finalizeBlock(std::move(out_cols), final, rows); +} + +Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const +{ + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + Block res_header = getHeader(final); Block res = res_header.cloneEmpty(); for (size_t i = 0; i < params.keys_size; ++i) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3e8b25c1a8c..6e37c5a63d4 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1306,6 +1306,25 @@ private: std::vector key_columns, AggregateColumnsData & aggregate_columns) const; + struct OutputBlockColumns + { + /*OutputBlockColumns(size_t keys_size, size_t aggregates_size) + : key_columns(keys_size) + , aggregate_columns(aggregates_size) + , final_aggregate_columns(aggregates_size) + , aggregate_columns_data(aggregates_size) + { + }*/ + + MutableColumns key_columns; + MutableColumns aggregate_columns; + MutableColumns final_aggregate_columns; + AggregateColumnsData aggregate_columns_data; + }; + + OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; + Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; + template Block prepareBlockAndFill( AggregatedDataVariants & data_variants, From 4e974661d6033c5fd0a55b5741add4f39458021e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 9 Jul 2022 01:22:12 +0200 Subject: [PATCH 02/39] refactor convertToBlockImpl --- src/Interpreters/Aggregator.cpp | 158 ++++++++++++++++---------------- src/Interpreters/Aggregator.h | 22 +---- 2 files changed, 83 insertions(+), 97 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e59605890d3..4420bdb700c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1578,16 +1578,9 @@ 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_); - }); + Block block = convertToBlockImpl( + method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()) + .front(); block.info.bucket_num = bucket; return block; @@ -1694,25 +1687,16 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const template -void Aggregator::convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final) const +BlocksList +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(aggregates_pools, final, rows); + return {finalizeBlock(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 raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); + BlocksList res; if (final) { @@ -1720,20 +1704,23 @@ void Aggregator::convertToBlockImpl( if (compiled_aggregate_functions_holder) { static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } else #endif { - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } } else { - convertToBlockImplNotFinal(method, data, std::move(raw_key_columns), aggregate_columns); + res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); } + /// In order to release memory early. data.clearAndShrink(); + + return res; } @@ -1803,13 +1790,20 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template -void NO_INLINE Aggregator::convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, - Arena * arena) const +BlocksList NO_INLINE +Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const { + auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + if constexpr (Method::low_cardinality_optimization) { if (data.hasNullKeyData()) @@ -1819,20 +1813,21 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( } } - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); + auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; PaddedPODArray places; places.reserve(data.size()); - data.forEachValue([&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); - places.emplace_back(mapped); + data.forEachValue( + [&](const auto & key, auto & mapped) + { + method.insertKeyIntoColumns(key, 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; - }); + /// 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; @@ -1914,41 +1909,53 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( if (exception) std::rethrow_exception(exception); + + return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } template -void NO_INLINE Aggregator::convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const +BlocksList NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { + auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); + auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + if constexpr (Method::low_cardinality_optimization) { if (data.hasNullKeyData()) { - key_columns[0]->insertDefault(); + raw_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]); + aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); data.getNullKeyData() = nullptr; } } - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); + auto shuffled_key_sizes = method.shuffleKeyColumns(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) - { - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); + data.forEachValue( + [&](const auto & key, auto & mapped) + { + method.insertKeyIntoColumns(key, raw_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]); + /// 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; - }); + mapped = nullptr; + }); + + return {finalizeBlock(std::move(out_cols), /* final */ false, rows)}; } Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const @@ -2162,27 +2169,22 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { - size_t rows = data_variants.sizeWithoutOverflowRow(); + // clang-format off + const size_t rows = data_variants.sizeWithoutOverflowRow(); +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ + *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows).front(); \ + } - auto filler = [&data_variants, this]( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_) + if (false) { - #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); + } // NOLINT + APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) +#undef M + else + throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 6e37c5a63d4..9dd019da883 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1276,14 +1276,7 @@ private: ManyAggregatedDataVariants & non_empty_data) const; template - void convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final) const; + BlocksList convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; template void insertAggregatesIntoColumns( @@ -1292,19 +1285,10 @@ private: Arena * arena) const; template - void convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, - Arena * arena) const; + BlocksList convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; template - void convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const; + BlocksList convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; struct OutputBlockColumns { From e5e0a24ab3d5c843ba72920c9d1920d16ec49f76 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 11 Jul 2022 23:50:58 +0200 Subject: [PATCH 03/39] return chunks from prepareBlockAndFillWithoutKey --- src/Interpreters/Aggregator.cpp | 16 ++++---- src/Interpreters/Aggregator.h | 2 +- .../AggregatingInOrderTransform.cpp | 4 +- .../Transforms/AggregatingTransform.cpp | 39 ++++++++----------- 4 files changed, 27 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 4420bdb700c..063348f23a9 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2167,15 +2167,15 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } -Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const +BlocksList Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); -#define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - { \ - return convertToBlockImpl( \ - *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows).front(); \ +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ + *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } if (false) @@ -2306,7 +2306,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.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final).front()); else blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); } @@ -3075,7 +3075,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) block = prepareBlockAndFillWithoutKey(result, final, is_overflows); else - block = prepareBlockAndFillSingleLevel(result, final); + block = prepareBlockAndFillSingleLevel(result, final).front(); /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 9dd019da883..74cbcee433a 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1332,7 +1332,7 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; - Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; + BlocksList prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; template diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index ce50ae5eeee..6b8707113ac 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -182,7 +182,7 @@ 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(variants, /* final= */ false).front(); cur_block_bytes += current_memory_usage; finalizeCurrentChunk(std::move(chunk), key_end); return; @@ -293,7 +293,7 @@ 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(variants, /* final= */ false).front(); else params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); variants.invalidate(); diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7f5896f5e97..a2be680c6c2 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -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,9 +244,12 @@ public: private: IProcessor::Status preparePushToOutput() { + if (single_level_chunks.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Some ready chunks expected"); + auto & output = outputs.front(); - output.push(std::move(current_chunk)); - has_input = false; + output.push(std::move(single_level_chunks.back())); + single_level_chunks.pop_back(); if (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 chunks; + std::array 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(*first, params->final); + for (auto & block : blocks) + single_level_chunks.emplace_back(convertToChunk(block)); - setCurrentChunk(convertToChunk(block)); finished = true; } From db0110fd7a1a62312ac7b93f67e4651573905def Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 12 Jul 2022 17:06:20 +0200 Subject: [PATCH 04/39] more accurate crutch --- src/Interpreters/Aggregator.cpp | 36 ++++++++++--------- src/Interpreters/Aggregator.h | 27 +++++++++----- .../AggregatingInOrderTransform.cpp | 6 ++-- .../Transforms/AggregatingTransform.cpp | 2 +- 4 files changed, 42 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 063348f23a9..56f69c1f940 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1578,9 +1578,8 @@ Block Aggregator::convertOneBucketToBlock( bool final, size_t bucket) const { - Block block = convertToBlockImpl( - method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()) - .front(); + Block block = convertToBlockImpl( + method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()); block.info.bucket_num = bucket; return block; @@ -1686,8 +1685,8 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const } -template -BlocksList +template +Aggregator::ConvertToBlockRes Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const { if (data.empty()) @@ -1696,7 +1695,7 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are return {finalizeBlock(std::move(out_cols), final, rows)}; } - BlocksList res; + ConvertToBlockRes res; if (final) { @@ -1704,17 +1703,17 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are if (compiled_aggregate_functions_holder) { static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } else #endif { - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); } } else { - res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); + res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); } /// In order to release memory early. @@ -1789,8 +1788,8 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu } -template -BlocksList NO_INLINE +template +Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); @@ -1913,8 +1912,9 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } -template -BlocksList NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const +template +Aggregator::ConvertToBlockRes NO_INLINE +Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; @@ -2167,14 +2167,16 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } -BlocksList Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const +template +Aggregator::ConvertToBlockRes +Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); #define M(NAME) \ else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ { \ - return convertToBlockImpl( \ + return convertToBlockImpl( \ *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } @@ -2306,7 +2308,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).front()); + blocks.splice(blocks.end(), prepareBlockAndFillSingleLevel(data_variants, final)); else blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get())); } @@ -3075,7 +3077,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) block = prepareBlockAndFillWithoutKey(result, final, is_overflows); else - block = prepareBlockAndFillSingleLevel(result, final).front(); + block = prepareBlockAndFillSingleLevel(result, final); /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 74cbcee433a..4f06ee67624 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1,8 +1,9 @@ #pragma once -#include -#include #include +#include +#include +#include #include @@ -1275,8 +1276,12 @@ private: void mergeSingleLevelDataImpl( ManyAggregatedDataVariants & non_empty_data) const; - template - BlocksList convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; + template + using ConvertToBlockRes = std::conditional_t; + + template + ConvertToBlockRes + convertToBlockImpl(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows) const; template void insertAggregatesIntoColumns( @@ -1284,11 +1289,13 @@ private: MutableColumns & final_aggregate_columns, Arena * arena) const; - template - BlocksList convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; + template + ConvertToBlockRes + convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; - template - BlocksList convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; + template + ConvertToBlockRes + convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; struct OutputBlockColumns { @@ -1332,9 +1339,11 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; - BlocksList prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; + template + ConvertToBlockRes prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; + template BlocksList prepareBlocksAndFillTwoLevelImpl( AggregatedDataVariants & data_variants, diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 6b8707113ac..9137a50aba5 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -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).front(); + group_by_block + = params->aggregator.prepareBlockAndFillSingleLevel(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).front(); + group_by_block + = params->aggregator.prepareBlockAndFillSingleLevel(variants, /* final= */ false); else params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); variants.invalidate(); diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index a2be680c6c2..fb1dd45544d 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -356,7 +356,7 @@ private: else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - auto blocks = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); + auto blocks = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final); for (auto & block : blocks) single_level_chunks.emplace_back(convertToChunk(block)); From f650b23ee37febab19a53a6208e9485ded902ac1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 12 Jul 2022 22:33:42 +0200 Subject: [PATCH 05/39] generate many blocks --- src/Interpreters/Aggregator.cpp | 247 ++++++++++-------- src/Interpreters/Aggregator.h | 9 +- .../Transforms/AggregatingTransform.cpp | 2 +- 3 files changed, 141 insertions(+), 117 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 56f69c1f940..367da5d1352 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -1789,76 +1790,76 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template -Aggregator::ConvertToBlockRes NO_INLINE -Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const +Aggregator::ConvertToBlockRes + NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); - - if constexpr (Method::low_cardinality_optimization) + auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) { - if (data.hasNullKeyData()) + std::exception_ptr exception; + size_t aggregate_functions_destroy_index = 0; + + try { - key_columns[0]->insertDefault(); - insertAggregatesIntoColumns(data.getNullKeyData(), final_aggregate_columns, arena); - } - } - - auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - - PaddedPODArray places; - places.reserve(data.size()); - - data.forEachValue( - [&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, 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; - }); - - std::exception_ptr exception; - size_t aggregate_functions_destroy_index = 0; - - try - { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - /** For JIT compiled functions we need to resize columns before pass them into compiled code. + if constexpr (use_compiled_functions) + { + /** For JIT compiled functions we need to resize columns before pass them into compiled code. * insert_aggregates_into_columns_function function does not throw exception. */ - std::vector columns_data; + std::vector columns_data; - auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; + auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!is_aggregate_function_compiled[i]) - continue; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!is_aggregate_function_compiled[i]) + continue; - auto & final_aggregate_column = final_aggregate_columns[i]; - final_aggregate_column = final_aggregate_column->cloneResized(places.size()); - columns_data.emplace_back(getColumnData(final_aggregate_column.get())); + 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())); + } + + auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; + insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); } - - auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; - insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); - } #endif - for (; aggregate_functions_destroy_index < params.aggregates_size;) + for (; aggregate_functions_destroy_index < params.aggregates_size;) + { + if constexpr (use_compiled_functions) + { + if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + { + ++aggregate_functions_destroy_index; + continue; + } + } + + 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 + * throws exception, it also must destroy all necessary states. + * Then code need to continue to destroy other aggregate function states with next function index. + */ + size_t destroy_index = aggregate_functions_destroy_index; + ++aggregate_functions_destroy_index; + + /// For State AggregateFunction ownership of aggregate place is passed to result column after insert + 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); + } + } + catch (...) + { + exception = std::current_exception(); + } + + for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) { if constexpr (use_compiled_functions) { @@ -1869,47 +1870,76 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena } } - auto & final_aggregate_column = 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 - * throws exception, it also must destroy all necessary states. - * Then code need to continue to destroy other aggregate function states with next function index. - */ - size_t destroy_index = aggregate_functions_destroy_index; - ++aggregate_functions_destroy_index; - - /// For State AggregateFunction ownership of aggregate place is passed to result column after insert - 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[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); } - } - catch (...) - { - exception = std::current_exception(); - } - for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + if (exception) + std::rethrow_exception(exception); + + return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); + }; + + const size_t max_block_size = DEFAULT_BLOCK_SIZE; + ConvertToBlockRes res; + + std::optional out_cols; + std::optional shuffled_key_sizes; + const Sizes * key_sizes_ptr = nullptr; + PaddedPODArray places; + + auto init_out_cols = [&]() { - if constexpr (use_compiled_functions) + out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, max_block_size); + + if constexpr (Method::low_cardinality_optimization) { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + if (data.hasNullKeyData()) { - ++aggregate_functions_destroy_index; - continue; + out_cols->key_columns[0]->insertDefault(); + insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); } } - size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); + key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; + + places.reserve(max_block_size); + }; + + data.forEachValue( + [&](const auto & key, auto & mapped) + { + if (!out_cols.has_value()) + init_out_cols(); + + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + 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(insert_results_into_cols(places, out_cols.value())); + places.clear(); + out_cols.reset(); + } + } + }); + + if constexpr (return_single_block) + { + return insert_results_into_cols(places, out_cols.value()); + } + else + { + if (!places.empty()) + res.emplace_back(insert_results_into_cols(places, out_cols.value())); + return res; } - - if (exception) - std::rethrow_exception(exception); - - return {finalizeBlock(std::move(out_cols), /* final */ true, rows)}; } template @@ -1917,15 +1947,7 @@ Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const { auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; - - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; if constexpr (Method::low_cardinality_optimization) { @@ -1946,7 +1968,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a data.forEachValue( [&](const auto & key, auto & mapped) { - method.insertKeyIntoColumns(key, raw_key_columns, key_sizes_ref); + 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) @@ -2013,8 +2035,17 @@ Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & ag } } + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector 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), @@ -2029,7 +2060,7 @@ Block Aggregator::prepareBlockAndFill( Filler && filler) const { auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); @@ -2038,7 +2069,7 @@ Block Aggregator::prepareBlockAndFill( Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const { - auto && [key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; Block res_header = getHeader(final); Block res = res_header.cloneEmpty(); @@ -2171,12 +2202,11 @@ template Aggregator::ConvertToBlockRes Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const { - // clang-format off const size_t rows = data_variants.sizeWithoutOverflowRow(); -#define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - { \ - return convertToBlockImpl( \ +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + return convertToBlockImpl( \ *data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows); \ } @@ -2185,8 +2215,7 @@ Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variant } // NOLINT APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) #undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } @@ -3278,4 +3307,6 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons } +template Aggregator::ConvertToBlockRes +Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const; } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4f06ee67624..a3d7097a7cc 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1299,15 +1299,8 @@ private: struct OutputBlockColumns { - /*OutputBlockColumns(size_t keys_size, size_t aggregates_size) - : key_columns(keys_size) - , aggregate_columns(aggregates_size) - , final_aggregate_columns(aggregates_size) - , aggregate_columns_data(aggregates_size) - { - }*/ - MutableColumns key_columns; + std::vector raw_key_columns; MutableColumns aggregate_columns; MutableColumns final_aggregate_columns; AggregateColumnsData aggregate_columns_data; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index fb1dd45544d..4e55081ca48 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -251,7 +251,7 @@ private: output.push(std::move(single_level_chunks.back())); single_level_chunks.pop_back(); - if (finished) + if (finished && single_level_chunks.empty()) { output.finish(); return Status::Finished; From 63bc894a42ba2e3660dd7a112c7a201b51e28e75 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 13 Jul 2022 12:36:04 +0200 Subject: [PATCH 06/39] more parallelism --- src/Interpreters/Aggregator.cpp | 5 +++-- src/Processors/QueryPlan/AggregatingStep.cpp | 9 ++++----- src/QueryPipeline/QueryPipelineBuilder.h | 2 ++ tests/performance/queries_over_aggregation.xml | 4 ++++ .../02343_aggregation_pipeline.reference | 17 +++++++++++++++++ .../0_stateless/02343_aggregation_pipeline.sql | 2 ++ 6 files changed, 32 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 367da5d1352..5dc0a6e319a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -1579,7 +1578,9 @@ Block Aggregator::convertOneBucketToBlock( bool final, size_t bucket) const { - Block block = convertToBlockImpl( + // Used in ConvertingAggregatedToChunksSource -> ConvertingAggregatedToChunksTransform (expects single chunk for each bucket_id). + constexpr bool return_single_block = true; + Block block = convertToBlockImpl( method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size()); block.info.bucket_num = bucket; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f0374d2419b..3680b28f449 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -376,16 +376,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(header, transform_params); - }); + pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, transform_params); }); + + pipeline.resize(should_produce_results_in_order_of_bucket_number ? 1 : params.max_threads, false /* force */); aggregating = collector.detachProcessors(0); } diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 100a2e07341..850c4a66615 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -161,6 +161,8 @@ public: return std::max(1, num_threads); } + size_t getMaxThreads() const { return max_threads; } + /// Set upper limit for the recommend number of threads void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } diff --git a/tests/performance/queries_over_aggregation.xml b/tests/performance/queries_over_aggregation.xml index 2a92ea26819..ceaed61c5bb 100644 --- a/tests/performance/queries_over_aggregation.xml +++ b/tests/performance/queries_over_aggregation.xml @@ -1,4 +1,8 @@ + select sipHash64(number) from numbers(1e7) group by number format Null + select * from (select * from numbers(1e7) group by number) group by number format Null + select * from (select * from numbers(1e7) group by number) order by number format Null + select * from (select * from numbers_mt(1e7) group by number) group by number format Null select * from (select * from numbers_mt(1e7) group by number) order by number format Null select * from (select * from numbers_mt(1e7) group by number) group by number format Null settings max_bytes_before_external_group_by = 1 diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.reference b/tests/queries/0_stateless/02343_aggregation_pipeline.reference index 67bd9c414ba..ec9a394d05d 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.reference +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.reference @@ -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 diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.sql b/tests/queries/0_stateless/02343_aggregation_pipeline.sql index d259889b042..f25cef15733 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.sql +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.sql @@ -4,6 +4,8 @@ set optimize_aggregation_in_order = 0; -- { 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; From ae5baf290619c3c99e0a8ea9c560eb77c366f5da Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 13 Jul 2022 21:50:40 +0200 Subject: [PATCH 07/39] fix tests --- tests/queries/0_stateless/01091_num_threads.sql | 2 +- .../01524_do_not_merge_across_partitions_select_final.reference | 2 +- .../01524_do_not_merge_across_partitions_select_final.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01091_num_threads.sql b/tests/queries/0_stateless/01091_num_threads.sql index faeceb0e6d6..0d2a66a8c2e 100644 --- a/tests/queries/0_stateless/01091_num_threads.sql +++ b/tests/queries/0_stateless/01091_num_threads.sql @@ -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); diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index a3f2106cd5f..540137d4887 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -6,4 +6,4 @@ 2020-01-01 00:00:00 2 1 499999 -5 +18 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index a5423d1a3ff..23678c1abd9 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -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); From 433657e978c292e39bbd329f2a0bf18cf16fc84e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 01:46:18 +0200 Subject: [PATCH 08/39] rm prepareBlockAndFill --- src/Interpreters/Aggregator.cpp | 74 +++++++++++++-------------------- src/Interpreters/Aggregator.h | 7 ---- 2 files changed, 30 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 5dc0a6e319a..5a75cc2bacd 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2053,21 +2053,6 @@ Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & ag }; } -template -Block Aggregator::prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const -{ - auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); - - return finalizeBlock(std::move(out_cols), final, rows); -} - Block Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const { auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; @@ -2156,39 +2141,34 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const { size_t rows = 1; + auto && out_cols = prepareOutputBlockColumns(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(std::move(out_cols), final, rows); if (is_overflows) block.info.is_overflows = true; @@ -3105,9 +3085,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 in SortingAggregatedTransform (expects single chunk for each bucket_id). + constexpr bool return_single_block = true; + block = prepareBlockAndFillSingleLevel(result, final); + } /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. if (!final) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index a3d7097a7cc..7021a757df1 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1309,13 +1309,6 @@ private: OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; - template - Block prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - size_t rows, - Filler && filler) const; - template Block convertOneBucketToBlock( AggregatedDataVariants & data_variants, From 56c09bf8a90ffe03234ccc73494bc6d2e2badd35 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 01:46:41 +0200 Subject: [PATCH 09/39] generate many blocks in convertToBlockImplNotFinal --- src/Interpreters/Aggregator.cpp | 66 +++++++++++++++++++++++++-------- 1 file changed, 51 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 5a75cc2bacd..96119a06fa6 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1945,40 +1945,76 @@ Aggregator::ConvertToBlockRes template Aggregator::ConvertToBlockRes NO_INLINE -Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const +Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, rows); - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + const size_t max_block_size = DEFAULT_BLOCK_SIZE; + ConvertToBlockRes res; - if constexpr (Method::low_cardinality_optimization) + std::optional out_cols; + std::optional shuffled_key_sizes; + const Sizes * key_sizes_ptr = nullptr; + + auto init_out_cols = [&]() { - if (data.hasNullKeyData()) + out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, max_block_size); + + if constexpr (Method::low_cardinality_optimization) { - raw_key_columns[0]->insertDefault(); + if (data.hasNullKeyData()) + { + out_cols->raw_key_columns[0]->insertDefault(); - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); + 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.getNullKeyData() = nullptr; + } } - } - auto shuffled_key_sizes = method.shuffleKeyColumns(raw_key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; + shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); + key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; + }; + + size_t rows_in_current_block = 0; data.forEachValue( [&](const auto & key, auto & mapped) { - method.insertKeyIntoColumns(key, out_cols.raw_key_columns, key_sizes_ref); + if (!out_cols.has_value()) + init_out_cols(); + + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); /// 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]); + 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(std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + out_cols.reset(); + rows_in_current_block = 0; + } + } }); - return {finalizeBlock(std::move(out_cols), /* final */ false, rows)}; + if constexpr (return_single_block) + { + return finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block); + } + else + { + if (rows_in_current_block) + res.emplace_back(finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block)); + return res; + } + return res; } Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const From 66b3268c659c3d3594821e7b4ba74f0a20877a30 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 20:13:33 +0200 Subject: [PATCH 10/39] fix --- src/Interpreters/Aggregator.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 96119a06fa6..1f47684575b 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1908,6 +1908,9 @@ Aggregator::ConvertToBlockRes 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) { @@ -1975,6 +1978,9 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &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( From 4250a142a7300d82fdca48eaaff0161cc853bc7a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 14 Jul 2022 20:51:49 +0200 Subject: [PATCH 11/39] fix test --- ...egate_function_group_bitmap_data.reference | 68 +++++++++---------- ...1_aggregate_function_group_bitmap_data.sql | 3 +- 2 files changed, 36 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference index 161f4a6372f..8c3288df670 100644 --- a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference +++ b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.reference @@ -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 diff --git a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql index d70665655ca..a04f40058fd 100644 --- a/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql +++ b/tests/queries/0_stateless/01671_aggregate_function_group_bitmap_data.sql @@ -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; From 2b76abdacd539456280e8b85204736904bf50941 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 18 Jul 2022 15:19:26 +0200 Subject: [PATCH 12/39] fix tidy --- src/Interpreters/Aggregator.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 1f47684575b..9f075602ee0 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2233,9 +2233,7 @@ Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variant *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) #undef M else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); From 370c25cd2a387df1bb3b3bf67c3e1b07c1500501 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 29 Jul 2022 16:05:18 +0200 Subject: [PATCH 13/39] fix comment --- src/Interpreters/Aggregator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 9f075602ee0..984b4055ada 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -3130,7 +3130,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) } else { - // Used during memory efficient merging in SortingAggregatedTransform (expects single chunk for each bucket_id). + // Used during memory efficient merging (SortingAggregatedTransform expects single chunk for each bucket_id). constexpr bool return_single_block = true; block = prepareBlockAndFillSingleLevel(result, final); } From 6bdbaccc37379a73a975426b9cc6959f0ea1c0f7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 29 Jul 2022 16:10:53 +0200 Subject: [PATCH 14/39] use max_block_size from settings --- src/Interpreters/Aggregator.cpp | 6 +++--- src/Interpreters/Aggregator.h | 8 ++++++-- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- src/Processors/QueryPlan/AggregatingStep.cpp | 1 + src/Processors/TTL/TTLAggregationAlgorithm.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + tests/queries/0_stateless/02343_aggregation_pipeline.sql | 1 + .../02355_control_block_size_in_aggregator.reference | 1 + .../02355_control_block_size_in_aggregator.sql | 9 +++++++++ 9 files changed, 26 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference create mode 100644 tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 984b4055ada..49cf42ac3b4 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1792,7 +1792,7 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template Aggregator::ConvertToBlockRes - NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) { @@ -1881,7 +1881,7 @@ Aggregator::ConvertToBlockRes return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); }; - const size_t max_block_size = DEFAULT_BLOCK_SIZE; + const size_t max_block_size = params.max_block_size; ConvertToBlockRes res; std::optional out_cols; @@ -1950,7 +1950,7 @@ template Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = DEFAULT_BLOCK_SIZE; + const size_t max_block_size = params.max_block_size; ConvertToBlockRes res; std::optional out_cols; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 7021a757df1..3e9dbb7780a 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -933,6 +933,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 +971,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 +990,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, {}) { } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 205ec049975..fd676859c16 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1719,7 +1719,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( query_plan.getCurrentDataStream(), @@ -2315,6 +2315,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 }; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 3680b28f449..5a5326091e6 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -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(src_header, std::move(params_for_set), final); diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index 0d160b8d32d..6a813a770cf 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -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(header, params); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c5f546a9c36..b3d85e3c00c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -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); diff --git a/tests/queries/0_stateless/02343_aggregation_pipeline.sql b/tests/queries/0_stateless/02343_aggregation_pipeline.sql index f25cef15733..85e9fd1be1e 100644 --- a/tests/queries/0_stateless/02343_aggregation_pipeline.sql +++ b/tests/queries/0_stateless/02343_aggregation_pipeline.sql @@ -1,6 +1,7 @@ set max_threads = 16; set prefer_localhost_replica = 1; set optimize_aggregation_in_order = 0; +set max_block_size = 65505; -- { echoOn } diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql new file mode 100644 index 00000000000..b4754c6d6fe --- /dev/null +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql @@ -0,0 +1,9 @@ +SET max_block_size = 4213; + +SELECT DISTINCT (blockSize() <= 4213) +FROM +( + SELECT number + FROM numbers(100000) + GROUP BY number +); From 248011d7d9a55d6f58cbc53d6339a3e6fc6f169a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 00:05:14 +0200 Subject: [PATCH 15/39] move to utils --- src/Interpreters/AggregationUtils.h | 119 ++++++++++++++++++++++++++ src/Interpreters/Aggregator.cpp | 127 ++++------------------------ src/Interpreters/Aggregator.h | 12 --- 3 files changed, 136 insertions(+), 122 deletions(-) create mode 100644 src/Interpreters/AggregationUtils.h diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h new file mode 100644 index 00000000000..572757f65e9 --- /dev/null +++ b/src/Interpreters/AggregationUtils.h @@ -0,0 +1,119 @@ +#include + +namespace DB +{ + +struct OutputBlockColumns +{ + MutableColumns key_columns; + std::vector raw_key_columns; + MutableColumns aggregate_columns; + MutableColumns final_aggregate_columns; + Aggregator::AggregateColumnsData aggregate_columns_data; +}; + + +inline 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(*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(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(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 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), + }; +} + +inline 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; +} +} diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 49cf42ac3b4..aa04e1aec6e 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -32,6 +32,8 @@ #include +#include + namespace ProfileEvents { extern const Event ExternalAggregationWritePart; @@ -1693,8 +1695,8 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are { if (data.empty()) { - auto && out_cols = prepareOutputBlockColumns(aggregates_pools, final, rows); - return {finalizeBlock(std::move(out_cols), final, rows)}; + auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); + return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)}; } ConvertToBlockRes res; @@ -1878,7 +1880,7 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if (exception) std::rethrow_exception(exception); - return finalizeBlock(std::move(out_cols), /* final */ true, places.size()); + return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); }; const size_t max_block_size = params.max_block_size; @@ -1891,7 +1893,8 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ true, max_block_size); + out_cols = prepareOutputBlockColumns( + params, aggregate_functions, getHeader(/* final */ true), aggregates_pools, /* final */ true, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1959,7 +1962,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns(aggregates_pools, /* final */ false, max_block_size); + out_cols = prepareOutputBlockColumns( + params, aggregate_functions, getHeader(/* final */ false), aggregates_pools, /* final */ false, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -2003,7 +2007,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a { if (rows_in_current_block >= max_block_size) { - res.emplace_back(finalizeBlock(std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + res.emplace_back(finalizeBlock( + params, getHeader(/* final */ false), std::move(out_cols.value()), /* final */ false, rows_in_current_block)); out_cols.reset(); rows_in_current_block = 0; } @@ -2012,117 +2017,18 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if constexpr (return_single_block) { - return finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block); + return finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block); } else { if (rows_in_current_block) - res.emplace_back(finalizeBlock(std::move(out_cols).value(), /* final */ false, rows_in_current_block)); + res.emplace_back( + finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_block)); return res; } return res; } -Aggregator::OutputBlockColumns Aggregator::prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) 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(*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(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(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 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 Aggregator::finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const -{ - auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - - Block res_header = getHeader(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; -} - void Aggregator::addSingleKeyToAggregateColumns( AggregatedDataVariants & data_variants, MutableColumns & aggregate_columns) const @@ -2183,7 +2089,8 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const { size_t rows = 1; - auto && out_cols = prepareOutputBlockColumns(data_variants.aggregates_pools, final, rows); + 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; if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row) @@ -2210,7 +2117,7 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va key_columns[i]->insertDefault(); } - Block block = finalizeBlock(std::move(out_cols), final, rows); + Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); if (is_overflows) block.info.is_overflows = true; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3e9dbb7780a..2fd76a438a6 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1301,18 +1301,6 @@ private: ConvertToBlockRes convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; - struct OutputBlockColumns - { - MutableColumns key_columns; - std::vector raw_key_columns; - MutableColumns aggregate_columns; - MutableColumns final_aggregate_columns; - AggregateColumnsData aggregate_columns_data; - }; - - OutputBlockColumns prepareOutputBlockColumns(Arenas & aggregates_pools, bool final, size_t rows) const; - Block finalizeBlock(OutputBlockColumns && out_cols, bool final, size_t rows) const; - template Block convertOneBucketToBlock( AggregatedDataVariants & data_variants, From a6c4f9218a7f41b341b488482a240ea46391bfc3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 01:03:01 +0200 Subject: [PATCH 16/39] clean up --- src/Interpreters/AggregationUtils.cpp | 113 ++++++++++++++++ src/Interpreters/AggregationUtils.h | 100 +------------- src/Interpreters/Aggregator.cpp | 165 +++++++++++------------ src/Interpreters/Aggregator.h | 4 + src/QueryPipeline/QueryPipelineBuilder.h | 2 - 5 files changed, 201 insertions(+), 183 deletions(-) create mode 100644 src/Interpreters/AggregationUtils.cpp diff --git a/src/Interpreters/AggregationUtils.cpp b/src/Interpreters/AggregationUtils.cpp new file mode 100644 index 00000000000..43062546450 --- /dev/null +++ b/src/Interpreters/AggregationUtils.cpp @@ -0,0 +1,113 @@ +#include + +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(*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(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(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 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; +} +} diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h index 572757f65e9..d7769b0309d 100644 --- a/src/Interpreters/AggregationUtils.h +++ b/src/Interpreters/AggregationUtils.h @@ -13,107 +13,13 @@ struct OutputBlockColumns }; -inline OutputBlockColumns prepareOutputBlockColumns( +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); + size_t rows); - 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(*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(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(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 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), - }; -} - -inline 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; -} +Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows); } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index aa04e1aec6e..84708fc72ea 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1792,77 +1792,40 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu } -template -Aggregator::ConvertToBlockRes -NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +template +Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const { - auto insert_results_into_cols = [this, arena](PaddedPODArray & places, auto & out_cols) - { - std::exception_ptr exception; - size_t aggregate_functions_destroy_index = 0; + std::exception_ptr exception; + size_t aggregate_functions_destroy_index = 0; - try - { + try + { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - /** For JIT compiled functions we need to resize columns before pass them into compiled code. + if constexpr (use_compiled_functions) + { + /** For JIT compiled functions we need to resize columns before pass them into compiled code. * insert_aggregates_into_columns_function function does not throw exception. */ - std::vector columns_data; + std::vector columns_data; - auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; + auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!is_aggregate_function_compiled[i]) - continue; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!is_aggregate_function_compiled[i]) + continue; - 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())); - } - - auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; - insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); + 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())); } + + auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; + insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); + } #endif - for (; aggregate_functions_destroy_index < params.aggregates_size;) - { - if constexpr (use_compiled_functions) - { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } - } - - 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 - * throws exception, it also must destroy all necessary states. - * Then code need to continue to destroy other aggregate function states with next function index. - */ - size_t destroy_index = aggregate_functions_destroy_index; - ++aggregate_functions_destroy_index; - - /// For State AggregateFunction ownership of aggregate place is passed to result column after insert - 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); - } - } - catch (...) - { - exception = std::current_exception(); - } - - for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + for (; aggregate_functions_destroy_index < params.aggregates_size;) { if constexpr (use_compiled_functions) { @@ -1873,28 +1836,65 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are } } + auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; - aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + + /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch + * throws exception, it also must destroy all necessary states. + * Then code need to continue to destroy other aggregate function states with next function index. + */ + size_t destroy_index = aggregate_functions_destroy_index; + ++aggregate_functions_destroy_index; + + /// For State AggregateFunction ownership of aggregate place is passed to result column after insert + 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); + } + } + catch (...) + { + exception = std::current_exception(); + } + + for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) + { + if constexpr (use_compiled_functions) + { + if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) + { + ++aggregate_functions_destroy_index; + continue; + } } - if (exception) - std::rethrow_exception(exception); + size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; + aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); + } - return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); - }; + if (exception) + std::rethrow_exception(exception); + return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); +} + +template +Aggregator::ConvertToBlockRes NO_INLINE +Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +{ const size_t max_block_size = params.max_block_size; + const bool final = true; ConvertToBlockRes res; std::optional out_cols; std::optional shuffled_key_sizes; - const Sizes * key_sizes_ptr = nullptr; PaddedPODArray places; auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns( - params, aggregate_functions, getHeader(/* final */ true), aggregates_pools, /* final */ true, max_block_size); + out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1906,7 +1906,6 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are } shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); - key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; places.reserve(max_block_size); }; @@ -1920,7 +1919,8 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if (!out_cols.has_value()) init_out_cols(); - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + 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. @@ -1930,7 +1930,7 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are { if (places.size() >= max_block_size) { - res.emplace_back(insert_results_into_cols(places, out_cols.value())); + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); places.clear(); out_cols.reset(); } @@ -1939,12 +1939,12 @@ NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Are if constexpr (return_single_block) { - return insert_results_into_cols(places, out_cols.value()); + return insertResultsIntoColumns(places, std::move(out_cols.value()), arena); } else { - if (!places.empty()) - res.emplace_back(insert_results_into_cols(places, out_cols.value())); + if (out_cols.has_value()) + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); return res; } } @@ -1954,16 +1954,15 @@ Aggregator::ConvertToBlockRes 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 res; std::optional out_cols; std::optional shuffled_key_sizes; - const Sizes * key_sizes_ptr = nullptr; auto init_out_cols = [&]() { - out_cols = prepareOutputBlockColumns( - params, aggregate_functions, getHeader(/* final */ false), aggregates_pools, /* final */ false, max_block_size); + out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); if constexpr (Method::low_cardinality_optimization) { @@ -1979,7 +1978,6 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a } shuffled_key_sizes = method.shuffleKeyColumns(out_cols->raw_key_columns, key_sizes); - key_sizes_ptr = shuffled_key_sizes ? &*shuffled_key_sizes : &key_sizes; }; // should be invoked at least once, because null data might be the only content of the `data` @@ -1993,7 +1991,8 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if (!out_cols.has_value()) init_out_cols(); - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, *key_sizes_ptr); + 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) @@ -2007,8 +2006,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a { if (rows_in_current_block >= max_block_size) { - res.emplace_back(finalizeBlock( - params, getHeader(/* final */ false), std::move(out_cols.value()), /* final */ false, rows_in_current_block)); + 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; } @@ -2017,13 +2015,12 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a if constexpr (return_single_block) { - return finalizeBlock(params, getHeader(/* final */ false), std::move(out_cols).value(), /* final */ false, rows_in_current_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 */ false), std::move(out_cols).value(), /* final */ false, 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; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 2fd76a438a6..9c762318b2b 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -872,6 +872,7 @@ using ManyAggregatedDataVariantsPtr = std::shared_ptr + Block insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const; + template ConvertToBlockRes convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 850c4a66615..100a2e07341 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -161,8 +161,6 @@ public: return std::max(1, num_threads); } - size_t getMaxThreads() const { return max_threads; } - /// Set upper limit for the recommend number of threads void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } From 14d45f9d86c198006f1d162bf918e8df6fb54bcd Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 30 Jul 2022 19:49:43 +0200 Subject: [PATCH 17/39] fix style --- src/Interpreters/AggregationUtils.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/AggregationUtils.h b/src/Interpreters/AggregationUtils.h index d7769b0309d..cc37cec0a69 100644 --- a/src/Interpreters/AggregationUtils.h +++ b/src/Interpreters/AggregationUtils.h @@ -1,3 +1,5 @@ +#pragma once + #include namespace DB From 9af04ddf2186bab1137f3b2b69d127509da74955 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 1 Aug 2022 19:18:19 +0200 Subject: [PATCH 18/39] fix test --- .../01064_incremental_streaming_from_2_src_with_feedback.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index 0bc5fcd1db8..a63aa768051 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -23,6 +23,8 @@ AS FROM numbers(50000) GROUP BY id; +OPTIMIZE TABLE target_table FINAL; + -- source table #1 CREATE TABLE logins ( From 9242c2adb4af1cd930951156bec8c862ee5707f2 Mon Sep 17 00:00:00 2001 From: Stephan <79573800+cyber-moon@users.noreply.github.com> Date: Wed, 17 Aug 2022 11:56:48 +0200 Subject: [PATCH 19/39] Add description of {condition}-keyword Using the {condition}-keyword for Dictionaries is supported since a few weeks (see https://github.com/ClickHouse/ClickHouse/issues/33746 ). This PR adds a corresponding documentation. --- .../external-dicts-dict-lifetime.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md index ab83017f263..e2abdf21025 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md @@ -93,6 +93,21 @@ It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to onl - If the source is HTTP then `update_field` will be added as a query parameter with the last update time as the parameter value. - If the source is Executable then `update_field` will be added as an executable script argument with the last update time as the argument value. - If the source is ClickHouse, MySQL, PostgreSQL, ODBC there will be an additional part of `WHERE`, where `update_field` is compared as greater or equal with the last update time. + - Per default, this `WHERE`-condition is checked at the highest level of the SQL-Query. Alternatively, the condition can be checked in any other `WHERE`-clause within the query using the `{condition}`-keyword. Example: + ```sql + ... + SOURCE(CLICKHOUSE(... + update_field 'added_time' + QUERY ' + SELECT my_arr.1 AS x, my_arr.2 AS y, creation_time + FROM ( + SELECT arrayZip(x_arr, y_arr) AS my_arr, creation_time + FROM dictionary_source + WHERE {condition} + )' + )) + ... + ``` If `update_field` option is set, additional option `update_lag` can be set. Value of `update_lag` option is subtracted from previous update time before request updated data. From ec2e671d202cd86094bb294d12e2e8a4317ab134 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 17 Aug 2022 23:46:06 +0200 Subject: [PATCH 20/39] Remove ThreadStatus::untracked_memory_limit_increase It looks useless nowadays, because operator new cannot throw MEMORY_LIMIT_EXCEEDED today, and so any code that works on Exception is likely safe. Refs: #40249 Refs: #24483 Signed-off-by: Azat Khuzhin --- src/Common/CurrentMemoryTracker.cpp | 7 +------ src/Common/ThreadStatus.h | 2 -- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 921c244da21..720df07efb9 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -52,15 +52,10 @@ void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) if (current_thread) { Int64 will_be = current_thread->untracked_memory + size; - Int64 limit = current_thread->untracked_memory_limit + current_thread->untracked_memory_limit_increase; - if (will_be > limit) + if (will_be > current_thread->untracked_memory_limit) { - /// Increase limit before track. If tracker throws out-of-limit we would be able to alloc up to untracked_memory_limit bytes - /// more. It could be useful to enlarge Exception message in rethrow logic. - current_thread->untracked_memory_limit_increase = current_thread->untracked_memory_limit; memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); - current_thread->untracked_memory_limit_increase = 0; current_thread->untracked_memory = 0; } else diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 2a4ffd229f2..0b01f43a226 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -135,8 +135,6 @@ public: Int64 untracked_memory = 0; /// Each thread could new/delete memory in range of (-untracked_memory_limit, untracked_memory_limit) without access to common counters. Int64 untracked_memory_limit = 4 * 1024 * 1024; - /// Increase limit in case of exception. - Int64 untracked_memory_limit_increase = 0; /// Statistics of read and write rows/bytes Progress progress_in; From e4532da317b915e7003fc3d5b1bd1823eec078f1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Aug 2022 19:28:49 +0200 Subject: [PATCH 21/39] fix test --- .../01730_distributed_group_by_no_merge_order_by_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql index 62b578c21d6..3d6a25fe799 100644 --- a/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql +++ b/tests/queries/0_stateless/01730_distributed_group_by_no_merge_order_by_long.sql @@ -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, From b31342ec2cf24e0487466cde11143c8ba6063ffc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 22 Aug 2022 19:29:48 +0200 Subject: [PATCH 22/39] fix --- src/Interpreters/Aggregator.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 84708fc72ea..04fa2aaddae 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1902,6 +1902,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena { out_cols->key_columns[0]->insertDefault(); insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); + data.hasNullKeyData() = false; } } @@ -1974,6 +1975,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a out_cols->aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); data.getNullKeyData() = nullptr; + data.hasNullKeyData() = false; } } From 365a65219e2d3e83e63a2e326b86a91ae87f3339 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 23 Aug 2022 19:20:43 +0200 Subject: [PATCH 23/39] try fix tests --- .../01064_incremental_streaming_from_2_src_with_feedback.sql | 3 +-- .../0_stateless/01281_group_by_limit_memory_tracking.sh | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index a63aa768051..9a439180265 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -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 @@ -23,8 +24,6 @@ AS FROM numbers(50000) GROUP BY id; -OPTIMIZE TABLE target_table FINAL; - -- source table #1 CREATE TABLE logins ( diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index 2f4164ee0d1..7f353b84a80 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -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 From abf51bccfd22c66321a6df26c6b8ac11fb908fd3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 29 Aug 2022 01:18:44 +0800 Subject: [PATCH 24/39] Disable projection when grouping set is used. --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ ...1710_aggregate_projection_with_grouping_set.reference | 6 ++++++ .../01710_aggregate_projection_with_grouping_set.sql | 9 +++++++++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c19e4f5f559..2a9eae1a026 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5541,6 +5541,10 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (select_query->interpolate() && !select_query->interpolate()->children.empty()) return std::nullopt; + // Currently projections don't support GROUPING SET yet. + if (select_query->group_by_with_grouping_sets) + return std::nullopt; + auto query_options = SelectQueryOptions( QueryProcessingStage::WithMergeableState, /* depth */ 1, diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference new file mode 100644 index 00000000000..fd34e25e370 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference @@ -0,0 +1,6 @@ +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql new file mode 100644 index 00000000000..2b393482b43 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql @@ -0,0 +1,9 @@ +drop table if exists test; + +create table test(dim1 String, dim2 String, projection p1 (select dim1, dim2, count() group by dim1, dim2)) engine MergeTree order by dim1; + +insert into test values ('a', 'x') ('a', 'y') ('b', 'x') ('b', 'y'); + +select dim1, dim2, count() from test group by grouping sets ((dim1, dim2), dim1) order by dim1, dim2, count(); + +drop table test; From 6493fb4cbb46fe2e4d8a8a5225dce3f69b5b4054 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 29 Aug 2022 10:09:47 +0800 Subject: [PATCH 25/39] Add more test cases --- ...ate_projection_with_grouping_set.reference | 22 +++++++++++++++++++ ...aggregate_projection_with_grouping_set.sql | 6 +++++ 2 files changed, 28 insertions(+) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference index fd34e25e370..b233507ce6d 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.reference @@ -4,3 +4,25 @@ a y 1 b 2 b x 1 b y 1 + 4 +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 + 4 + x 2 + y 2 +a 2 +a x 1 +a y 1 +b 2 +b x 1 +b y 1 +a x 1 +a y 1 +b x 1 +b y 1 + + 4 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql index 2b393482b43..652ce786b5d 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_grouping_set.sql @@ -6,4 +6,10 @@ insert into test values ('a', 'x') ('a', 'y') ('b', 'x') ('b', 'y'); select dim1, dim2, count() from test group by grouping sets ((dim1, dim2), dim1) order by dim1, dim2, count(); +select dim1, dim2, count() from test group by dim1, dim2 with rollup order by dim1, dim2, count(); + +select dim1, dim2, count() from test group by dim1, dim2 with cube order by dim1, dim2, count(); + +select dim1, dim2, count() from test group by dim1, dim2 with totals order by dim1, dim2, count(); + drop table test; From dd8478422a4d6c76b421569588efda00bee2022a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Sep 2022 14:15:18 +0200 Subject: [PATCH 26/39] Fix usage of generator expressions for libcxx cmake Before, if you store clickhouse sources in /src, there was a typo and it produce the following error: CMake Error in contrib/libcxx-cmake/CMakeLists.txt: Target "cxx" INTERFACE_INCLUDE_DIRECTORIES property contains path: "/src" which is prefixed in the source directory. Also move "src" into PRIVATE, since it is required only for libcxx itself. Signed-off-by: Azat Khuzhin --- contrib/libcxx-cmake/CMakeLists.txt | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index a501c4df64f..6f42a479588 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -54,9 +54,8 @@ set(SRCS add_library(cxx ${SRCS}) set_target_properties(cxx PROPERTIES FOLDER "contrib/libcxx-cmake") -target_include_directories(cxx SYSTEM BEFORE PUBLIC - $ - $/src) +target_include_directories(cxx SYSTEM BEFORE PRIVATE $) +target_include_directories(cxx SYSTEM BEFORE PUBLIC $) target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) # Enable capturing stack traces for all exceptions. From 53836bbeebfef9fa990add7b08fa949ee1b88358 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Sep 2022 12:49:50 +0000 Subject: [PATCH 27/39] Fix typo The system view is called 'role*_*grants', documented on page 'role*-*grants.md'. --- docs/en/sql-reference/statements/show.md | 2 +- docs/ru/sql-reference/statements/show.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 0721f17e9e2..00347d9cb5b 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -303,7 +303,7 @@ SHOW USERS ## SHOW ROLES -Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). +Returns a list of [roles](../../operations/access-rights.md#role-management). To view another parameters, see system tables [system.roles](../../operations/system-tables/roles.md#system_tables-roles) and [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). ### Syntax diff --git a/docs/ru/sql-reference/statements/show.md b/docs/ru/sql-reference/statements/show.md index 1d072c9d5de..59f33c691ae 100644 --- a/docs/ru/sql-reference/statements/show.md +++ b/docs/ru/sql-reference/statements/show.md @@ -305,7 +305,7 @@ SHOW USERS ## SHOW ROLES {#show-roles-statement} -Выводит список [ролей](../../operations/access-rights.md#role-management). Для просмотра параметров ролей, см. системные таблицы [system.roles](../../operations/system-tables/roles.md#system_tables-roles) и [system.role-grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). +Выводит список [ролей](../../operations/access-rights.md#role-management). Для просмотра параметров ролей, см. системные таблицы [system.roles](../../operations/system-tables/roles.md#system_tables-roles) и [system.role_grants](../../operations/system-tables/role-grants.md#system_tables-role_grants). ### Синтаксис {#show-roles-syntax} From 396500aaa7c4e1d24f1c69f56bf244f75a6a0721 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 13:14:33 +0000 Subject: [PATCH 28/39] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index bdba298189e..1be805e7cb2 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit bdba298189e29995892de78dcecf64d127444e81 +Subproject commit 1be805e7cb2494aa8170015493474379b0362dfc From 0db38fcbc95483f577eee7f2ce40096f8fe456f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Sep 2022 15:32:46 +0200 Subject: [PATCH 29/39] Fix --- src/IO/WriteBufferFromS3.cpp | 16 ++++++++++++---- src/IO/WriteBufferFromS3.h | 1 + 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 2f2479766d5..eda62f05446 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -431,7 +431,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() { if (schedule) { - std::lock_guard lock(bg_tasks_mutex); + std::unique_lock lock(bg_tasks_mutex); { while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised) { @@ -442,7 +442,7 @@ void WriteBufferFromS3::waitForReadyBackGroundTasks() if (exception) { - waitForAllBackGroundTasks(); + waitForAllBackGroundTasksUnlocked(lock); std::rethrow_exception(exception); } @@ -457,7 +457,15 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (schedule) { std::unique_lock lock(bg_tasks_mutex); - bg_tasks_condvar.wait(lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); + waitForAllBackGroundTasksUnlocked(lock); + } +} + +void WriteBufferFromS3::waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock) +{ + if (schedule) + { + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; }); while (!upload_object_tasks.empty()) { @@ -472,7 +480,7 @@ void WriteBufferFromS3::waitForAllBackGroundTasks() if (put_object_task) { - bg_tasks_condvar.wait(lock, [this]() { return put_object_task->is_finised; }); + bg_tasks_condvar.wait(bg_tasks_lock, [this]() { return put_object_task->is_finised; }); if (put_object_task->exception) std::rethrow_exception(put_object_task->exception); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 99440654910..712044841d0 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -84,6 +84,7 @@ private: void waitForReadyBackGroundTasks(); void waitForAllBackGroundTasks(); + void waitForAllBackGroundTasksUnlocked(std::unique_lock & bg_tasks_lock); String bucket; String key; From 0f3003e37bd41829203585cef2af11125e5fa725 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 2 Sep 2022 17:13:36 +0800 Subject: [PATCH 30/39] Add ccache `brew install` list --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 97e4e4ddde1..12f74feb272 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -37,7 +37,7 @@ sudo xcode-select --install ``` bash brew update -brew install cmake ninja libtool gettext llvm gcc binutils grep findutils +brew install ccache cmake ninja libtool gettext llvm gcc binutils grep findutils ``` ## Checkout ClickHouse Sources {#checkout-clickhouse-sources} From 69b9d34b10c5188f3b6b13ce8ef2de9f28235da6 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Fri, 2 Sep 2022 17:25:10 +0200 Subject: [PATCH 31/39] docs. optional params for GenerateRandom table --- docs/en/engines/table-engines/special/generate.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/generate.md b/docs/en/engines/table-engines/special/generate.md index d03d6dc9d13..e42429a1b10 100644 --- a/docs/en/engines/table-engines/special/generate.md +++ b/docs/en/engines/table-engines/special/generate.md @@ -15,7 +15,7 @@ Usage examples: ## Usage in ClickHouse Server {#usage-in-clickhouse-server} ``` sql -ENGINE = GenerateRandom(random_seed, max_string_length, max_array_length) +ENGINE = GenerateRandom([random_seed] [,max_string_length] [,max_array_length]) ``` The `max_array_length` and `max_string_length` parameters specify maximum length of all From 90baf74f13cb4e6e1d89475bb8f3657403d27822 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 2 Sep 2022 19:30:35 +0200 Subject: [PATCH 32/39] Added mkdir command --- programs/disks/CommandMkDir.cpp | 67 +++++++++++++++++++++++++++++++++ programs/disks/DisksApp.cpp | 3 +- programs/disks/DisksApp.h | 1 + programs/disks/ICommand.h | 1 + 4 files changed, 71 insertions(+), 1 deletion(-) create mode 100644 programs/disks/CommandMkDir.cpp diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp new file mode 100644 index 00000000000..11a940028a3 --- /dev/null +++ b/programs/disks/CommandMkDir.cpp @@ -0,0 +1,67 @@ +#pragma once + +#include "ICommand.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +class CommandMkDir : public ICommand +{ +public: + CommandMkDir() + { + command_name = "mkdir"; + command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); + description = "Create directory or directories recursively"; + usage = "mkdir [OPTION]... "; + command_option_description->add_options() + ("recursive", "recursively create directories") + ; + } + + void processOptions( + Poco::Util::LayeredConfiguration & config, + po::variables_map & options) const override + { + if (options.count("recursive")) + config.setBool("recursive", true); + } + + void execute( + const std::vector & command_arguments, + DB::ContextMutablePtr & global_context, + Poco::Util::LayeredConfiguration & config) override + { + if (command_arguments.size() != 1) + { + printHelpMessage(); + throw DB::Exception("Bad Arguments", DB::ErrorCodes::BAD_ARGUMENTS); + } + + String disk_name = config.getString("disk", "default"); + + String path = command_arguments[0]; + + DiskPtr disk = global_context->getDisk(disk_name); + + String full_path = fullPathWithValidate(disk, path); + bool recursive = config.getBool("recursive", false); + + if (recursive) + disk->createDirectories(full_path); + else + disk->createDirectory(full_path); + } +}; +} + +std::unique_ptr makeCommandMkDir() +{ + return std::make_unique(); +} diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 08768386808..b662921a3b1 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -63,7 +63,7 @@ void DisksApp::addOptions( positional_options_description.add("command_name", 1); - supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read"}; + supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read", "mkdir"}; command_descriptions.emplace("list-disks", makeCommandListDisks()); command_descriptions.emplace("list", makeCommandList()); @@ -73,6 +73,7 @@ void DisksApp::addOptions( command_descriptions.emplace("copy", makeCommandCopy()); command_descriptions.emplace("write", makeCommandWrite()); command_descriptions.emplace("read", makeCommandRead()); + command_descriptions.emplace("mkdir", makeCommandMkDir()); } void DisksApp::processOptions() diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index cbb3a7dfcc9..24fa9c3d9eb 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -4,6 +4,7 @@ #include "CommandLink.cpp" #include "CommandList.cpp" #include "CommandListDisks.cpp" +#include "CommandMkDir.cpp" #include "CommandMove.cpp" #include "CommandRead.cpp" #include "CommandRemove.cpp" diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 9cde55dbb6b..f57f74a880e 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -65,3 +65,4 @@ std::unique_ptr makeCommandMove(); std::unique_ptr makeCommandRead(); std::unique_ptr makeCommandRemove(); std::unique_ptr makeCommandWrite(); +std::unique_ptr makeCommandMkDir(); From 1768a82a53fa9a132d497492b21d51e1a0cb74bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Sep 2022 06:45:18 +0200 Subject: [PATCH 33/39] git checkout c4b8137d31e8b6093dba1ad42355a328bfdf84df docs/en/getting-started/example-datasets/uk-price-paid.md --- .../example-datasets/uk-price-paid.md | 129 +++++++++++------- 1 file changed, 77 insertions(+), 52 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 5ba3dc0e38c..bb9046397a0 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,18 +6,28 @@ title: "UK Property Price Paid" --- The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. -Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads
+Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. +## Download the Dataset {#download-dataset} + +Run the command: + +```bash +wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv +``` + +Download will take about 2 minutes with good internet connection. + ## Create the Table {#create-table} ```sql CREATE TABLE uk_price_paid ( - uuid UUID, price UInt32, date Date, postcode1 LowCardinality(String), @@ -32,67 +42,66 @@ CREATE TABLE uk_price_paid town LowCardinality(String), district LowCardinality(String), county LowCardinality(String), - category UInt8, - category2 UInt8 -) ORDER BY (postcode1, postcode2, addr1, addr2); + category UInt8 +) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); ``` ## Preprocess and Import Data {#preprocess-import-data} -In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with either `clickhouse-client` or the web based Play UI. +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. The preprocessing is: -- splitting the postcode to two different columns `postcode1` and `postcode2` that are better for storage and queries; +- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; - coverting the `time` field to date as it only contains 00:00 time; - ignoring the [UUid](../../sql-reference/data-types/uuid.md) field because we don't need it for analysis; - transforming `type` and `duration` to more readable Enum fields with function [transform](../../sql-reference/functions/other-functions.md#transform); - transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. +Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. + ```bash -INSERT INTO uk_price_paid -WITH - splitByChar(' ', postcode) AS p -SELECT - replaceRegexpAll(uuid_string, '{|}','') AS uuid, - toUInt32(price_string) AS price, - parseDateTimeBestEffortUS(time) AS date, - p[1] AS postcode1, - p[2] AS postcode2, - transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, - b = 'Y' AS is_new, - transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, - addr1, - addr2, - street, - locality, - town, - district, - county, - d = 'B' AS category, - e = 'B' AS category2 -FROM url( - 'http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv', - 'CSV', - 'uuid_string String, - price_string String, - time String, - postcode String, - a String, - b String, - c String, - addr1 String, - addr2 String, - street String, - locality String, - town String, - district String, - county String, - d String, - e String' -) -SETTINGS max_http_get_redirects=1; +clickhouse-local --input-format CSV --structure ' + uuid String, + price UInt32, + time DateTime, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String +' --query " + WITH splitByChar(' ', postcode) AS p + SELECT + price, + toDate(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category + FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" ``` +It will take about 40 seconds. + ## Validate the Data {#validate-data} Query: @@ -103,12 +112,28 @@ SELECT count() FROM uk_price_paid; Result: -```response +```text ┌──count()─┐ -│ 27450499 │ +│ 26321785 │ └──────────┘ ``` +The size of dataset in ClickHouse is just 278 MiB, check it. + +Query: + +```sql +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; +``` + +Result: + +```text +┌─formatReadableSize(total_bytes)─┐ +│ 278.80 MiB │ +└─────────────────────────────────┘ +``` + ## Run Some Queries {#run-queries} ### Query 1. Average Price Per Year {#average-price} @@ -121,7 +146,7 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, Result: -```response +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ From 385427ade826e5d408ff82fd569b76a92b6cf239 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 3 Sep 2022 17:20:02 +0000 Subject: [PATCH 34/39] Fix formatting of notes box in documentation Follow-up to PR #38435 --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/ru/sql-reference/functions/date-time-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 52f9a06df72..3515e903adf 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -267,7 +267,7 @@ Result: └────────────────┘ ``` -:::Attention +:::note The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday` functions described below is `Date` or `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. In case argument is out of normal range: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 63386bb32b9..1c623cd1dab 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -267,7 +267,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; └────────────────┘ ``` -:::Attention +:::note Тип возвращаемого описанными далее функциями `toStartOf*`, `toMonday` значения - `Date` или `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. Возвращаемые значения для значений вне нормального диапазона: @@ -277,7 +277,7 @@ SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp; * `2149-05-31` будет результатом функции `toLastDayOfMonth` при обработке аргумента больше `2149-05-31`. ::: -:::Attention +:::note Тип возвращаемого описанными далее функциями `toStartOf*`, `toLastDayOfMonth`, `toMonday` значения - `Date` или `DateTime`. Хотя эти функции могут принимать значения типа `Date32` или `DateTime64` в качестве аргумента, при обработке аргумента вне нормального диапазона значений (`1970` - `2148` для `Date` и `1970-01-01 00:00:00`-`2106-02-07 08:28:15` для `DateTime`) будет получен некорректный результат. Возвращаемые значения для значений вне нормального диапазона: From f2de8ff8ff7768127c95a5796bc750034f5aedc0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 3 Sep 2022 19:42:47 -0300 Subject: [PATCH 35/39] Doc. commpressions http.md (#40959) --- docs/en/interfaces/http.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 036fcde6d7a..c980bc65152 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -175,6 +175,10 @@ You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP - `br` - `deflate` - `xz` +- `zstd` +- `lz4` +- `bz2` +- `snappy` To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. In order for ClickHouse to compress the response, enable compression with [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting and append `Accept-Encoding: compression_method` header to the request. You can configure the data compression level in the [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level) setting for all compression methods. From dd19b0856ea6e1d039035b1b7053256c334b3775 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 3 Sep 2022 19:43:39 -0300 Subject: [PATCH 36/39] Doc. mapApply, mapFilter, mapUpdate (#40961) * Update tuple-map-functions.md * Update tuple-map-functions.md --- .../functions/tuple-map-functions.md | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index c555a838927..52023df4d72 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -430,5 +430,119 @@ Result: └────────────────────────────┘ ``` +## mapApply + +**Syntax** + +```sql +mapApply(func, map) +``` + +**Parameters** + +- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `map` — [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map obtained from the original map by application of `func(map1[i], …, mapN[i])` for each element. + +**Example** + +Query: + +```sql +SELECT mapApply((k, v) -> (k, v * 10), _map) AS r +FROM +( + SELECT map('key1', number, 'key2', number * 2) AS _map + FROM numbers(3) +) +``` + +Result: + +```text +┌─r─────────────────────┐ +│ {'key1':0,'key2':0} │ +│ {'key1':10,'key2':20} │ +│ {'key1':20,'key2':40} │ +└───────────────────────┘ +``` + +## mapFilter + +**Syntax** + +```sql +mapFilter(func, map) +``` + +**Parameters** + +- `func` - [Lamda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `map` — [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map containing only the elements in `map` for which `func(map1[i], …, mapN[i])` returns something other than 0. + + +**Example** + +Query: + +```sql +SELECT mapFilter((k, v) -> ((v % 2) = 0), _map) AS r +FROM +( + SELECT map('key1', number, 'key2', number * 2) AS _map + FROM numbers(3) +) +``` + +Result: + +```text +┌─r───────────────────┐ +│ {'key1':0,'key2':0} │ +│ {'key2':2} │ +│ {'key1':2,'key2':4} │ +└─────────────────────┘ +``` + + +## mapUpdate + +**Syntax** + +```sql +mapUpdate(map1, map2) +``` + +**Parameters** + +- `map1` [Map](../../sql-reference/data-types/map.md). +- `map2` [Map](../../sql-reference/data-types/map.md). + +**Returned value** + +- Returns a map1 with values updated of values for the corresponding keys in map2. + +**Example** + +Query: + +```sql +SELECT mapUpdate(map('key1', 0, 'key3', 0), map('key1', 10, 'key2', 10)) AS map; +``` + +Result: + +```text +┌─map────────────────────────────┐ +│ {'key3':0,'key1':10,'key2':10} │ +└────────────────────────────────┘ +``` [Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) From 68bf3b70773b76b98eae3a35838031b09c5fd348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Sep 2022 02:53:14 +0200 Subject: [PATCH 37/39] Fix bad test --- .../0_stateless/02277_full_sort_join_misc.sql | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02277_full_sort_join_misc.sql b/tests/queries/0_stateless/02277_full_sort_join_misc.sql index b4e3882edaf..4297f532b98 100644 --- a/tests/queries/0_stateless/02277_full_sort_join_misc.sql +++ b/tests/queries/0_stateless/02277_full_sort_join_misc.sql @@ -1,24 +1,24 @@ SET join_algorithm = 'full_sorting_merge'; -SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 USING key; +SELECT * FROM (SELECT 1 as key) AS t1 JOIN (SELECT 1 as key) t2 USING key ORDER BY key; -SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT 1 :: Nullable(UInt32) as key) t2 USING (key) ORDER BY key; -SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key); +SELECT * FROM (SELECT 1 :: UInt32 as key) AS t1 FULL JOIN (SELECT NULL :: Nullable(UInt32) as key) t2 USING (key) ORDER BY key; -SELECT * FROM (SELECT 1 :: Int32 as key) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 1 :: Int32 as key) AS t1 JOIN (SELECT 1 :: UInt32 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT -1 :: Nullable(Int32) as key) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key; +SELECT * FROM (SELECT -1 :: Nullable(Int32) as key) AS t1 FULL JOIN (SELECT 4294967295 :: UInt32 as key) t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: String AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: Nullable(String) AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(Nullable(String)) AS key) AS t1 JOIN (SELECT 'a' :: Nullable(String) AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key; +SELECT * FROM (SELECT 'a' :: LowCardinality(String) AS key) AS t1 JOIN (SELECT 'a' :: LowCardinality(String) AS key) AS t2 ON t1.key = t2.key ORDER BY key; -SELECT 5 == count() FROM (SELECT number as a from numbers(5)) as t1 LEFT JOIN (SELECT number as b from numbers(5) WHERE number > 100) as t2 ON t1.a = t2.b; -SELECT 5 == count() FROM (SELECT number as a from numbers(5) WHERE number > 100) as t1 RIGHT JOIN (SELECT number as b from numbers(5)) as t2 ON t1.a = t2.b; +SELECT 5 == count() FROM (SELECT number as a from numbers(5)) as t1 LEFT JOIN (SELECT number as b from numbers(5) WHERE number > 100) as t2 ON t1.a = t2.b ORDER BY 1; +SELECT 5 == count() FROM (SELECT number as a from numbers(5) WHERE number > 100) as t1 RIGHT JOIN (SELECT number as b from numbers(5)) as t2 ON t1.a = t2.b ORDER BY 1; From 64f95710cd192d02811d94e94631caa8d449d09c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Sep 2022 14:02:46 +0200 Subject: [PATCH 38/39] ci: add symlink for llvm-symbolizer (to make symbolizer work) Simply installing llvm-symbolizer-$VER is not enough, since it does not contain proper symblink, while LLVM is looking only for "llvm-symbolizer" (without version) - [1]: [1]: https://github.com/llvm/llvm-project/blob/c444af1c20b35555f2fdb2c1ca38d3f23b2faebd/compiler-rt/lib/sanitizer_common/sanitizer_symbolizer_posix_libcdep.cpp#L454 Follow-up for: #40655 Signed-off-by: Azat Khuzhin --- docker/packager/binary/Dockerfile | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index ba0a27c9801..b9b0c5c2c6c 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -83,5 +83,8 @@ RUN export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ --yes --no-install-recommends \ && apt-get clean +# for external_symbolizer_path +RUN ln -s /usr/bin/llvm-symbolizer-15 /usr/bin/llvm-symbolizer + COPY build.sh / CMD ["bash", "-c", "/build.sh 2>&1"] From 16af4aebc815d5af439ff0da3b960d810401b790 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 4 Sep 2022 16:26:31 +0200 Subject: [PATCH 39/39] impl (#40952) --- .../ReplaceQueryParameterVisitor.cpp | 20 ++++++++++++------- ...d_protocol_with_query_parameters.reference | 7 +++++++ ...7_extend_protocol_with_query_parameters.sh | 16 ++++++++++++++- 3 files changed, 35 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 03de8aecc92..664cda74522 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -1,16 +1,17 @@ -#include -#include #include -#include #include +#include #include #include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include namespace DB @@ -30,7 +31,12 @@ void ReplaceQueryParameterVisitor::visit(ASTPtr & ast) else if (ast->as() || ast->as()) visitIdentifier(ast); else - visitChildren(ast); + { + if (auto * describe_query = dynamic_cast(ast.get()); describe_query && describe_query->table_expression) + visitChildren(describe_query->table_expression); + else + visitChildren(ast); + } } diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference index f46cdb6e5e3..1da5cd0b7b3 100644 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.reference @@ -7,3 +7,10 @@ UInt64 String DateTime Map(UUID, Array(Float32)) 13 str 2022-08-04 18:30:53 {'10':[11,12],'13':[14,15]} 1 1 +_CAST(42, \'Int64\') Int64 +_CAST([1, 2, 3], \'Array(UInt8)\') Array(UInt8) +_CAST(((\'abc\', 22), (\'def\', 33)), \'Map(String, UInt8)\') Map(String, UInt8) +_CAST([[4, 5, 6], [7], [8, 9]], \'Array(Array(UInt8))\') Array(Array(UInt8)) +_CAST(((10, [11, 12]), (13, [14, 15])), \'Map(UInt8, Array(UInt8))\') Map(UInt8, Array(UInt8)) +_CAST(((\'ghj\', ((\'klm\', [16, 17]))), (\'nop\', ((\'rst\', [18])))), \'Map(String, Map(String, Array(UInt8)))\') Map(String, Map(String, Array(UInt8))) +a Int8 diff --git a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh index 335af1bb6e6..e61dc337d2a 100755 --- a/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh +++ b/tests/queries/0_stateless/02377_extend_protocol_with_query_parameters.sh @@ -68,13 +68,27 @@ $CLICKHOUSE_CLIENT -n -q "select {n: UInt8} -- { serverError 456 }" $CLICKHOUSE_CLIENT -n -q "set param_n = 12; set param_n = 13; select {n: UInt8}" -# but multiple different parameters could be defined within each session +# multiple different parameters could be defined within each session $CLICKHOUSE_CLIENT -n -q " set param_a = 13, param_b = 'str'; set param_c = '2022-08-04 18:30:53'; set param_d = '{\'10\': [11, 12], \'13\': [14, 15]}'; select {a: UInt32}, {b: String}, {c: DateTime}, {d: Map(String, Array(UInt8))}" + # empty parameter name is not allowed $CLICKHOUSE_CLIENT --param_="" -q "select 1" 2>&1 | grep -c 'Code: 36' $CLICKHOUSE_CLIENT -q "set param_ = ''" 2>&1 | grep -c 'Code: 36' + + +# parameters are also supported for DESCRIBE TABLE queries +$CLICKHOUSE_CLIENT \ + --param_id="42" \ + --param_arr="[1, 2, 3]" \ + --param_map="{'abc': 22, 'def': 33}" \ + --param_mul_arr="[[4, 5, 6], [7], [8, 9]]" \ + --param_map_arr="{10: [11, 12], 13: [14, 15]}" \ + --param_map_map_arr="{'ghj': {'klm': [16, 17]}, 'nop': {'rst': [18]}}" \ + -q "describe table(select {id: Int64}, {arr: Array(UInt8)}, {map: Map(String, UInt8)}, {mul_arr: Array(Array(UInt8))}, {map_arr: Map(UInt8, Array(UInt8))}, {map_map_arr: Map(String, Map(String, Array(UInt8)))})" + +$CLICKHOUSE_CLIENT --param_p=42 -q "describe table (select * from (select {p:Int8} as a group by a) order by a)"