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

137 lines
4.7 KiB
C
Raw Normal View History

2012-02-27 06:28:20 +00:00
#pragma once
#include <statdaemons/threadpool.hpp>
#include <DB/Interpreters/Aggregator.h>
#include <DB/Interpreters/Expression.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
using Poco::SharedPtr;
/** Агрегирует несколько источников параллельно.
* Запускает агрегацию отдельных источников в отдельных потоках, затем объединяет результаты.
* Агрегатные функции не финализируются, то есть, не заменяются на своё значение, а содержат промежуточное состояние вычислений.
* Это необходимо, чтобы можно было продолжить агрегацию (например, объединяя потоки частично агрегированных данных).
*/
class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream
{
public:
ParallelAggregatingBlockInputStream(BlockInputStreams inputs_, const ColumnNumbers & keys_, AggregateDescriptions & aggregates_,
unsigned max_threads_ = 1, size_t max_rows_to_group_by_ = 0, Limits::OverflowMode group_by_overflow_mode_ = Limits::THROW)
: inputs(inputs_), aggregator(new Aggregator(keys_, aggregates_, max_rows_to_group_by_, group_by_overflow_mode_)),
has_been_read(false), max_threads(max_threads_), pool(max_threads)
2012-02-27 06:28:20 +00:00
{
children.insert(children.end(), inputs_.begin(), inputs_.end());
}
/** keys берутся из GROUP BY части запроса
* Агрегатные функции ищутся везде в выражении.
* Столбцы, соответствующие keys и аргументам агрегатных функций, уже должны быть вычислены.
*/
ParallelAggregatingBlockInputStream(BlockInputStreams inputs_, ExpressionPtr expression,
unsigned max_threads_ = 1, size_t max_rows_to_group_by_ = 0, Limits::OverflowMode group_by_overflow_mode_ = Limits::THROW)
2012-02-27 06:28:20 +00:00
: inputs(inputs_), has_been_read(false), max_threads(max_threads_), pool(max_threads)
{
children.insert(children.end(), inputs_.begin(), inputs_.end());
Names key_names;
AggregateDescriptions aggregates;
expression->getAggregateInfo(key_names, aggregates);
aggregator = new Aggregator(key_names, aggregates, max_rows_to_group_by_, group_by_overflow_mode_);
2012-02-27 06:28:20 +00:00
}
2012-10-20 02:10:47 +00:00
String getName() const { return "ParallelAggregatingBlockInputStream"; }
BlockInputStreamPtr clone() { return new ParallelAggregatingBlockInputStream(*this); }
String getID() const
{
std::stringstream res;
res << "ParallelAggregating(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Порядок не имеет значения.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
res << ", " << aggregator->getID() << ")";
return res.str();
}
2012-10-20 02:10:47 +00:00
protected:
2012-02-27 06:28:20 +00:00
Block readImpl()
{
if (has_been_read)
return Block();
has_been_read = true;
ManyAggregatedDataVariants many_data(inputs.size());
Exceptions exceptions(inputs.size());
2012-10-20 02:10:47 +00:00
2012-02-27 06:28:20 +00:00
for (size_t i = 0, size = many_data.size(); i < size; ++i)
{
many_data[i] = new AggregatedDataVariants;
pool.schedule(boost::bind(&ParallelAggregatingBlockInputStream::calculate, this, boost::ref(inputs[i]), boost::ref(*many_data[i]), boost::ref(exceptions[i])));
}
pool.wait();
2012-09-12 18:18:48 +00:00
for (size_t i = 0, size = exceptions.size(); i < size; ++i)
if (exceptions[i])
exceptions[i]->rethrow();
2012-10-20 02:10:47 +00:00
if (isCancelled())
return Block();
2012-02-27 06:28:20 +00:00
AggregatedDataVariantsPtr res = aggregator->merge(many_data);
return aggregator->convertToBlock(*res);
}
private:
ParallelAggregatingBlockInputStream(const ParallelAggregatingBlockInputStream & src)
: inputs(src.inputs), aggregator(src.aggregator), has_been_read(src.has_been_read) {}
BlockInputStreams inputs;
SharedPtr<Aggregator> aggregator;
bool has_been_read;
size_t max_threads;
boost::threadpool::pool pool;
/// Вычисления, которые выполняться в отдельном потоке
void calculate(BlockInputStreamPtr & input, AggregatedDataVariants & data, ExceptionPtr & exception)
{
try
{
aggregator->execute(input, data);
}
catch (const Exception & e)
{
exception = e.clone();
2012-02-27 06:28:20 +00:00
}
catch (const Poco::Exception & e)
{
exception = e.clone();
2012-02-27 06:28:20 +00:00
}
catch (const std::exception & e)
{
exception = new Exception(e.what(), ErrorCodes::STD_EXCEPTION);
}
catch (...)
{
exception = new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
}
}
};
}