ClickHouse/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h

126 lines
4.0 KiB
C++
Raw Normal View History

2012-02-27 06:28:20 +00:00
#pragma once
#include <DB/Interpreters/Aggregator.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/CompressedReadBuffer.h>
2012-02-27 06:28:20 +00:00
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/DataStreams/ParallelInputsProcessor.h>
2012-02-27 06:28:20 +00:00
namespace DB
{
/** Агрегирует несколько источников параллельно.
* Производит агрегацию блоков из разных источников независимо в разных потоках, затем объединяет результаты.
2014-11-30 06:59:03 +00:00
* Если final == false, агрегатные функции не финализируются, то есть, не заменяются на своё значение, а содержат промежуточное состояние вычислений.
2012-02-27 06:28:20 +00:00
* Это необходимо, чтобы можно было продолжить агрегацию (например, объединяя потоки частично агрегированных данных).
*/
class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream
{
public:
/** Столбцы из key_names и аргументы агрегатных функций, уже должны быть вычислены.
2012-02-27 06:28:20 +00:00
*/
ParallelAggregatingBlockInputStream(
BlockInputStreams inputs, BlockInputStreamPtr additional_input_at_end,
const Aggregator::Params & params_, bool final_, size_t max_threads_, size_t temporary_data_merge_threads_);
2012-02-27 06:28:20 +00:00
String getName() const override { return "ParallelAggregating"; }
2012-10-20 02:10:47 +00:00
String getID() const override;
void cancel() override;
2012-10-20 02:10:47 +00:00
protected:
2016-05-20 20:43:07 +00:00
/// Ничего не делаем, чтобы подготовка к выполнению запроса делалась параллельно, в ParallelInputsProcessor.
void readPrefix() override
{
}
Block readImpl() override;
2012-02-27 06:28:20 +00:00
private:
Aggregator::Params params;
2015-01-08 18:52:48 +00:00
Aggregator aggregator;
bool final;
2012-02-27 06:28:20 +00:00
size_t max_threads;
size_t temporary_data_merge_threads;
2012-02-27 06:28:20 +00:00
size_t keys_size;
size_t aggregates_size;
/** Используется, если есть ограничение на максимальное количество строк при агрегации,
* и если group_by_overflow_mode == ANY.
* В этом случае, новые ключи не добавляются в набор, а производится агрегация только по
* ключам, которые уже успели попасть в набор.
*/
bool no_more_keys = false;
2015-12-16 20:21:52 +00:00
std::atomic<bool> executed {false};
/// Для чтения сброшенных во временный файл данных.
struct TemporaryFileStream
{
ReadBufferFromFile file_in;
CompressedReadBuffer compressed_in;
BlockInputStreamPtr block_in;
TemporaryFileStream(const std::string & path);
};
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;
Logger * log = &Logger::get("ParallelAggregatingBlockInputStream");
ManyAggregatedDataVariants many_data;
Exceptions exceptions;
struct ThreadData
{
size_t src_rows = 0;
size_t src_bytes = 0;
StringRefs key;
ConstColumnPlainPtrs key_columns;
Aggregator::AggregateColumns aggregate_columns;
Sizes key_sizes;
ThreadData(size_t keys_size, size_t aggregates_size)
{
key.resize(keys_size);
key_columns.resize(keys_size);
aggregate_columns.resize(aggregates_size);
key_sizes.resize(keys_size);
}
};
std::vector<ThreadData> threads_data;
struct Handler
2012-02-27 06:28:20 +00:00
{
Handler(ParallelAggregatingBlockInputStream & parent_)
: parent(parent_) {}
void onBlock(Block & block, size_t thread_num);
void onFinishThread(size_t thread_num);
void onFinish();
void onException(std::exception_ptr & exception, size_t thread_num);
ParallelAggregatingBlockInputStream & parent;
};
Handler handler;
ParallelInputsProcessor<Handler> processor;
void execute();
/** Отсюда будем доставать готовые блоки после агрегации.
*/
std::unique_ptr<IBlockInputStream> impl;
2012-02-27 06:28:20 +00:00
};
}