2013-02-03 18:39:09 +00:00
|
|
|
|
#include <iomanip>
|
2014-12-30 12:58:02 +00:00
|
|
|
|
#include <thread>
|
|
|
|
|
#include <future>
|
2013-02-03 18:39:09 +00:00
|
|
|
|
|
2015-01-12 08:06:17 +00:00
|
|
|
|
#include <cxxabi.h>
|
|
|
|
|
|
2015-10-05 00:44:40 +00:00
|
|
|
|
#include <DB/Common/Stopwatch.h>
|
2015-12-06 16:22:01 +00:00
|
|
|
|
#include <DB/Common/setThreadName.h>
|
2013-02-03 18:39:09 +00:00
|
|
|
|
|
2011-09-19 03:34:23 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeAggregateFunction.h>
|
2011-09-26 07:25:22 +00:00
|
|
|
|
#include <DB/Columns/ColumnsNumber.h>
|
2015-10-04 06:10:48 +00:00
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
2012-07-15 23:13:08 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
|
2015-07-30 23:41:02 +00:00
|
|
|
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
2015-11-30 19:57:46 +00:00
|
|
|
|
#include <DB/DataStreams/NativeBlockOutputStream.h>
|
2015-12-06 16:22:01 +00:00
|
|
|
|
#include <DB/DataStreams/NullBlockInputStream.h>
|
2015-11-30 19:57:46 +00:00
|
|
|
|
#include <DB/IO/WriteBufferFromFile.h>
|
|
|
|
|
#include <DB/IO/CompressedWriteBuffer.h>
|
2011-09-19 03:34:23 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Interpreters/Aggregator.h>
|
2016-02-09 17:06:50 +00:00
|
|
|
|
#include <common/ClickHouseRevision.h>
|
2011-09-19 01:42:16 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int CANNOT_COMPILE_CODE;
|
|
|
|
|
extern const int TOO_MUCH_ROWS;
|
|
|
|
|
extern const int EMPTY_DATA_PASSED;
|
|
|
|
|
extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS;
|
|
|
|
|
}
|
|
|
|
|
|
2011-09-26 07:25:22 +00:00
|
|
|
|
|
2013-02-16 18:59:05 +00:00
|
|
|
|
AggregatedDataVariants::~AggregatedDataVariants()
|
|
|
|
|
{
|
2013-11-03 23:54:12 +00:00
|
|
|
|
if (aggregator && !aggregator->all_aggregates_has_trivial_destructor)
|
2014-05-10 00:31:22 +00:00
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-05-19 19:41:56 +00:00
|
|
|
|
aggregator->destroyAllAggregateStates(*this);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-02-16 18:59:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-12-30 12:58:02 +00:00
|
|
|
|
void AggregatedDataVariants::convertToTwoLevel()
|
|
|
|
|
{
|
|
|
|
|
if (aggregator)
|
|
|
|
|
LOG_TRACE(aggregator->log, "Converting aggregation data to two-level.");
|
|
|
|
|
|
|
|
|
|
switch (type)
|
|
|
|
|
{
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: \
|
|
|
|
|
NAME ## _two_level.reset(new decltype(NAME ## _two_level)::element_type(*NAME)); \
|
|
|
|
|
NAME.reset(); \
|
|
|
|
|
type = Type::NAME ## _two_level; \
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
|
|
|
|
|
|
|
|
|
#undef M
|
|
|
|
|
|
|
|
|
|
default:
|
|
|
|
|
throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
void Aggregator::Params::calculateColumnNumbers(const Block & block)
|
|
|
|
|
{
|
|
|
|
|
if (keys.empty() && !key_names.empty())
|
|
|
|
|
for (Names::const_iterator it = key_names.begin(); it != key_names.end(); ++it)
|
|
|
|
|
keys.push_back(block.getPositionByName(*it));
|
|
|
|
|
|
|
|
|
|
for (AggregateDescriptions::iterator it = aggregates.begin(); it != aggregates.end(); ++it)
|
|
|
|
|
if (it->arguments.empty() && !it->argument_names.empty())
|
|
|
|
|
for (Names::const_iterator jt = it->argument_names.begin(); jt != it->argument_names.end(); ++jt)
|
|
|
|
|
it->arguments.push_back(block.getPositionByName(*jt));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-09-07 07:40:14 +00:00
|
|
|
|
void Aggregator::initialize(const Block & block)
|
2012-03-05 07:58:34 +00:00
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-01-08 18:52:48 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2012-03-05 07:58:34 +00:00
|
|
|
|
|
|
|
|
|
if (initialized)
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
initialized = true;
|
2013-02-13 19:24:19 +00:00
|
|
|
|
|
2015-12-07 20:08:00 +00:00
|
|
|
|
if (current_memory_tracker)
|
|
|
|
|
memory_usage_before_aggregation = current_memory_tracker->get();
|
2015-11-30 19:57:46 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
aggregate_functions.resize(params.aggregates_size);
|
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
aggregate_functions[i] = params.aggregates[i].function.get();
|
2013-02-13 19:24:19 +00:00
|
|
|
|
|
|
|
|
|
/// Инициализируем размеры состояний и смещения для агрегатных функций.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
offsets_of_aggregate_states.resize(params.aggregates_size);
|
2013-02-13 19:24:19 +00:00
|
|
|
|
total_size_of_aggregate_states = 0;
|
2013-11-03 23:54:12 +00:00
|
|
|
|
all_aggregates_has_trivial_destructor = true;
|
2013-02-13 19:24:19 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2013-02-13 19:24:19 +00:00
|
|
|
|
{
|
|
|
|
|
offsets_of_aggregate_states[i] = total_size_of_aggregate_states;
|
2015-11-30 16:57:05 +00:00
|
|
|
|
total_size_of_aggregate_states += params.aggregates[i].function->sizeOfData();
|
2013-11-03 23:54:12 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (!params.aggregates[i].function->hasTrivialDestructor())
|
2013-11-03 23:54:12 +00:00
|
|
|
|
all_aggregates_has_trivial_destructor = false;
|
2013-02-13 19:24:19 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2013-02-13 19:24:19 +00:00
|
|
|
|
/** Всё остальное - только если передан непустой block.
|
|
|
|
|
* (всё остальное не нужно в методе merge блоков с готовыми состояниями агрегатных функций).
|
|
|
|
|
*/
|
|
|
|
|
if (!block)
|
|
|
|
|
return;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-03-05 07:58:34 +00:00
|
|
|
|
/// Преобразуем имена столбцов в номера, если номера не заданы
|
2015-11-30 16:57:05 +00:00
|
|
|
|
params.calculateColumnNumbers(block);
|
2012-03-05 07:58:34 +00:00
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2012-03-05 07:58:34 +00:00
|
|
|
|
/// Создадим пример блока, описывающего результат
|
|
|
|
|
if (!sample)
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2012-05-31 01:13:15 +00:00
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
sample.insert(block.getByPosition(params.keys[i]).cloneEmpty());
|
2015-10-28 21:21:18 +00:00
|
|
|
|
if (auto converted = sample.getByPosition(i).column->convertToFullColumnIfConst())
|
|
|
|
|
sample.getByPosition(i).column = converted;
|
2012-05-31 01:13:15 +00:00
|
|
|
|
}
|
2012-03-05 07:58:34 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2012-03-05 07:58:34 +00:00
|
|
|
|
{
|
2015-07-17 01:27:35 +00:00
|
|
|
|
ColumnWithTypeAndName col;
|
2015-11-30 16:57:05 +00:00
|
|
|
|
col.name = params.aggregates[i].column_name;
|
2012-10-24 18:14:36 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
size_t arguments_size = params.aggregates[i].arguments.size();
|
2012-10-24 18:14:36 +00:00
|
|
|
|
DataTypes argument_types(arguments_size);
|
|
|
|
|
for (size_t j = 0; j < arguments_size; ++j)
|
2015-11-30 16:57:05 +00:00
|
|
|
|
argument_types[j] = block.getByPosition(params.aggregates[i].arguments[j]).type;
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2016-05-28 07:48:40 +00:00
|
|
|
|
col.type = std::make_shared<DataTypeAggregateFunction>(params.aggregates[i].function, argument_types, params.aggregates[i].parameters);
|
2014-06-05 23:52:28 +00:00
|
|
|
|
col.column = col.type->createColumn();
|
2012-03-05 07:58:34 +00:00
|
|
|
|
|
2016-08-04 23:35:07 +00:00
|
|
|
|
sample.insert(std::move(col));
|
2012-03-05 07:58:34 +00:00
|
|
|
|
}
|
2014-05-10 00:31:22 +00:00
|
|
|
|
}
|
2012-03-05 07:58:34 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-09 02:56:18 +00:00
|
|
|
|
void Aggregator::setSampleBlock(const Block & block)
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
|
|
|
|
|
|
|
|
|
if (!sample)
|
|
|
|
|
sample = block.cloneEmpty();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-01-10 02:30:03 +00:00
|
|
|
|
void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
|
|
|
|
|
|
|
|
|
if (compiled_if_possible)
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
compiled_if_possible = true;
|
|
|
|
|
|
|
|
|
|
std::string method_typename;
|
2015-01-13 01:57:22 +00:00
|
|
|
|
std::string method_typename_two_level;
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
2015-01-13 01:57:22 +00:00
|
|
|
|
if (false) {}
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
{ \
|
|
|
|
|
method_typename = "decltype(AggregatedDataVariants::" #NAME ")::element_type"; \
|
|
|
|
|
method_typename_two_level = "decltype(AggregatedDataVariants::" #NAME "_two_level)::element_type"; \
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
2015-01-10 02:30:03 +00:00
|
|
|
|
else if (type == AggregatedDataVariants::Type::NAME) \
|
2015-01-10 02:34:38 +00:00
|
|
|
|
method_typename = "decltype(AggregatedDataVariants::" #NAME ")::element_type";
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
2015-01-13 01:57:22 +00:00
|
|
|
|
APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M)
|
2015-01-10 02:30:03 +00:00
|
|
|
|
#undef M
|
2015-01-13 03:03:45 +00:00
|
|
|
|
else if (type == AggregatedDataVariants::Type::without_key) {}
|
|
|
|
|
else
|
2015-01-10 02:30:03 +00:00
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
|
|
|
|
/// Список типов агрегатных функций.
|
|
|
|
|
std::stringstream aggregate_functions_typenames_str;
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-10 02:30:03 +00:00
|
|
|
|
{
|
2015-01-21 04:23:22 +00:00
|
|
|
|
IAggregateFunction & func = *aggregate_functions[i];
|
|
|
|
|
|
2015-01-10 02:30:03 +00:00
|
|
|
|
int status = 0;
|
2015-01-21 04:23:22 +00:00
|
|
|
|
char * type_name_ptr = abi::__cxa_demangle(typeid(func).name(), 0, 0, &status);
|
2015-01-10 02:30:03 +00:00
|
|
|
|
std::string type_name = type_name_ptr;
|
|
|
|
|
free(type_name_ptr);
|
|
|
|
|
|
|
|
|
|
if (status)
|
2015-01-21 04:23:22 +00:00
|
|
|
|
throw Exception("Cannot compile code: cannot demangle name " + String(typeid(func).name())
|
2015-01-13 01:57:22 +00:00
|
|
|
|
+ ", status: " + toString(status), ErrorCodes::CANNOT_COMPILE_CODE);
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
|
|
|
|
aggregate_functions_typenames_str << ((i != 0) ? ", " : "") << type_name;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
std::string aggregate_functions_typenames = aggregate_functions_typenames_str.str();
|
|
|
|
|
|
2015-01-13 03:03:45 +00:00
|
|
|
|
std::stringstream key_str;
|
|
|
|
|
key_str << "Aggregate: ";
|
|
|
|
|
if (!method_typename.empty())
|
|
|
|
|
key_str << method_typename + ", ";
|
|
|
|
|
key_str << aggregate_functions_typenames;
|
|
|
|
|
std::string key = key_str.str();
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
2015-01-13 01:57:22 +00:00
|
|
|
|
auto get_code = [method_typename, method_typename_two_level, aggregate_functions_typenames]
|
2015-01-10 02:30:03 +00:00
|
|
|
|
{
|
2015-01-11 02:00:26 +00:00
|
|
|
|
/// Короткий кусок кода, представляющий собой явное инстанцирование шаблона.
|
2015-01-10 02:30:03 +00:00
|
|
|
|
std::stringstream code;
|
2015-01-18 01:18:39 +00:00
|
|
|
|
code << /// Нет явного включения заголовочного файла. Он подключается с помощью опции компилятора -include.
|
2015-01-10 02:30:03 +00:00
|
|
|
|
"namespace DB\n"
|
|
|
|
|
"{\n"
|
2015-01-13 01:57:22 +00:00
|
|
|
|
"\n";
|
|
|
|
|
|
|
|
|
|
/// Может быть до двух инстанцирований шаблона - для обычного и two_level вариантов.
|
|
|
|
|
auto append_code_for_specialization =
|
|
|
|
|
[&code, &aggregate_functions_typenames] (const std::string & method_typename, const std::string & suffix)
|
|
|
|
|
{
|
|
|
|
|
code <<
|
|
|
|
|
"template void Aggregator::executeSpecialized<\n"
|
|
|
|
|
"\t" << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
|
|
|
|
|
"\t" << method_typename << " &, Arena *, size_t, ConstColumnPlainPtrs &,\n"
|
|
|
|
|
"\tAggregateColumns &, const Sizes &, StringRefs &, bool, AggregateDataPtr) const;\n"
|
|
|
|
|
"\n"
|
|
|
|
|
"static void wrapper" << suffix << "(\n"
|
|
|
|
|
"\tconst Aggregator & aggregator,\n"
|
|
|
|
|
"\t" << method_typename << " & method,\n"
|
|
|
|
|
"\tArena * arena,\n"
|
|
|
|
|
"\tsize_t rows,\n"
|
|
|
|
|
"\tConstColumnPlainPtrs & key_columns,\n"
|
|
|
|
|
"\tAggregator::AggregateColumns & aggregate_columns,\n"
|
|
|
|
|
"\tconst Sizes & key_sizes,\n"
|
|
|
|
|
"\tStringRefs & keys,\n"
|
|
|
|
|
"\tbool no_more_keys,\n"
|
|
|
|
|
"\tAggregateDataPtr overflow_row)\n"
|
|
|
|
|
"{\n"
|
|
|
|
|
"\taggregator.executeSpecialized<\n"
|
|
|
|
|
"\t\t" << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
|
|
|
|
|
"\t\tmethod, arena, rows, key_columns, aggregate_columns, key_sizes, keys, no_more_keys, overflow_row);\n"
|
|
|
|
|
"}\n"
|
|
|
|
|
"\n"
|
2015-01-21 04:23:22 +00:00
|
|
|
|
"void * getPtr" << suffix << "() __attribute__((__visibility__(\"default\")));\n"
|
|
|
|
|
"void * getPtr" << suffix << "()\n" /// Без этой обёртки непонятно, как достать нужный символ из скомпилированной библиотеки.
|
2015-01-13 01:57:22 +00:00
|
|
|
|
"{\n"
|
2015-01-21 04:23:22 +00:00
|
|
|
|
"\treturn reinterpret_cast<void *>(&wrapper" << suffix << ");\n"
|
2015-01-13 01:57:22 +00:00
|
|
|
|
"}\n";
|
|
|
|
|
};
|
|
|
|
|
|
2015-01-13 03:03:45 +00:00
|
|
|
|
if (!method_typename.empty())
|
|
|
|
|
append_code_for_specialization(method_typename, "");
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
/// Для метода without_key.
|
|
|
|
|
code <<
|
|
|
|
|
"template void Aggregator::executeSpecializedWithoutKey<\n"
|
|
|
|
|
"\t" << "TypeList<" << aggregate_functions_typenames << ">>(\n"
|
|
|
|
|
"\tAggregatedDataWithoutKey &, size_t, AggregateColumns &) const;\n"
|
|
|
|
|
"\n"
|
|
|
|
|
"static void wrapper(\n"
|
|
|
|
|
"\tconst Aggregator & aggregator,\n"
|
|
|
|
|
"\tAggregatedDataWithoutKey & method,\n"
|
|
|
|
|
"\tsize_t rows,\n"
|
|
|
|
|
"\tAggregator::AggregateColumns & aggregate_columns)\n"
|
|
|
|
|
"{\n"
|
|
|
|
|
"\taggregator.executeSpecializedWithoutKey<\n"
|
|
|
|
|
"\t\tTypeList<" << aggregate_functions_typenames << ">>(\n"
|
|
|
|
|
"\t\tmethod, rows, aggregate_columns);\n"
|
|
|
|
|
"}\n"
|
|
|
|
|
"\n"
|
2015-01-21 04:23:22 +00:00
|
|
|
|
"void * getPtr() __attribute__((__visibility__(\"default\")));\n"
|
|
|
|
|
"void * getPtr()\n"
|
2015-01-13 03:03:45 +00:00
|
|
|
|
"{\n"
|
2015-01-21 04:23:22 +00:00
|
|
|
|
"\treturn reinterpret_cast<void *>(&wrapper);\n"
|
2015-01-13 03:03:45 +00:00
|
|
|
|
"}\n";
|
|
|
|
|
}
|
2015-01-13 01:57:22 +00:00
|
|
|
|
|
|
|
|
|
if (!method_typename_two_level.empty())
|
|
|
|
|
append_code_for_specialization(method_typename_two_level, "TwoLevel");
|
|
|
|
|
else
|
|
|
|
|
{
|
2015-01-13 03:03:45 +00:00
|
|
|
|
/// Заглушка.
|
2015-01-13 01:57:22 +00:00
|
|
|
|
code <<
|
2015-01-21 04:23:22 +00:00
|
|
|
|
"void * getPtrTwoLevel() __attribute__((__visibility__(\"default\")));\n"
|
|
|
|
|
"void * getPtrTwoLevel()\n"
|
2015-01-13 01:57:22 +00:00
|
|
|
|
"{\n"
|
|
|
|
|
"\treturn nullptr;\n"
|
|
|
|
|
"}\n";
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
code <<
|
2015-01-10 02:30:03 +00:00
|
|
|
|
"}\n";
|
|
|
|
|
|
|
|
|
|
return code.str();
|
|
|
|
|
};
|
|
|
|
|
|
2015-01-11 02:00:26 +00:00
|
|
|
|
auto compiled_data_owned_by_callback = compiled_data;
|
|
|
|
|
auto on_ready = [compiled_data_owned_by_callback] (SharedLibraryPtr & lib)
|
|
|
|
|
{
|
|
|
|
|
if (compiled_data_owned_by_callback.unique()) /// Aggregator уже уничтожен.
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
compiled_data_owned_by_callback->compiled_aggregator = lib;
|
2015-01-21 04:23:22 +00:00
|
|
|
|
compiled_data_owned_by_callback->compiled_method_ptr = lib->get<void * (*) ()>("_ZN2DB6getPtrEv")();
|
|
|
|
|
compiled_data_owned_by_callback->compiled_two_level_method_ptr = lib->get<void * (*) ()>("_ZN2DB14getPtrTwoLevelEv")();
|
2015-01-11 02:00:26 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
/** Если библиотека уже была скомпилирована, то возвращается ненулевой SharedLibraryPtr.
|
|
|
|
|
* Если библиотека не была скомпилирована, то увеличивается счётчик, и возвращается nullptr.
|
|
|
|
|
* Если счётчик достигнул значения min_count_to_compile, то асинхронно (в отдельном потоке) запускается компиляция,
|
|
|
|
|
* по окончании которой вызывается колбэк on_ready.
|
|
|
|
|
*/
|
2015-11-30 16:57:05 +00:00
|
|
|
|
SharedLibraryPtr lib = params.compiler->getOrCount(key, params.min_count_to_compile,
|
2015-01-18 01:18:39 +00:00
|
|
|
|
"-include /usr/share/clickhouse/headers/dbms/include/DB/Interpreters/SpecializedAggregator.h",
|
|
|
|
|
get_code, on_ready);
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
2015-01-11 02:00:26 +00:00
|
|
|
|
/// Если результат уже готов.
|
|
|
|
|
if (lib)
|
|
|
|
|
on_ready(lib);
|
2015-01-10 02:30:03 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-06-30 16:56:00 +00:00
|
|
|
|
AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes)
|
2012-05-30 01:38:02 +00:00
|
|
|
|
{
|
2015-01-13 01:57:22 +00:00
|
|
|
|
/** Возвращает обычные (не two-level) методы, так как обработка начинается с них.
|
|
|
|
|
* Затем, в процессе работы, данные могут быть переконвертированы в two-level структуру, если их становится много.
|
|
|
|
|
*/
|
|
|
|
|
|
2015-02-22 05:53:16 +00:00
|
|
|
|
bool all_fixed = true;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
size_t keys_bytes = 0;
|
2015-10-04 06:10:48 +00:00
|
|
|
|
|
|
|
|
|
size_t num_array_keys = 0;
|
|
|
|
|
bool has_arrays_of_non_fixed_elems = false;
|
|
|
|
|
bool all_non_array_keys_are_fixed = true;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
key_sizes.resize(params.keys_size);
|
|
|
|
|
for (size_t j = 0; j < params.keys_size; ++j)
|
2012-05-30 01:38:02 +00:00
|
|
|
|
{
|
2015-10-04 06:10:48 +00:00
|
|
|
|
if (key_columns[j]->isFixed())
|
|
|
|
|
{
|
|
|
|
|
key_sizes[j] = key_columns[j]->sizeOfField();
|
|
|
|
|
keys_bytes += key_sizes[j];
|
|
|
|
|
}
|
|
|
|
|
else
|
2012-05-30 01:38:02 +00:00
|
|
|
|
{
|
2015-02-22 05:53:16 +00:00
|
|
|
|
all_fixed = false;
|
2015-10-04 06:10:48 +00:00
|
|
|
|
|
|
|
|
|
if (const ColumnArray * arr = typeid_cast<const ColumnArray *>(key_columns[j]))
|
|
|
|
|
{
|
|
|
|
|
++num_array_keys;
|
|
|
|
|
|
|
|
|
|
if (!arr->getData().isFixed())
|
|
|
|
|
has_arrays_of_non_fixed_elems = true;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
all_non_array_keys_are_fixed = false;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если ключей нет
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.keys_size == 0)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::without_key;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2013-07-19 20:12:02 +00:00
|
|
|
|
/// Если есть один числовой ключ, который помещается в 64 бита
|
2016-08-18 15:58:46 +00:00
|
|
|
|
if (params.keys_size == 1 && key_columns[0]->isNumericNotNullable())
|
2014-10-29 01:18:50 +00:00
|
|
|
|
{
|
|
|
|
|
size_t size_of_field = key_columns[0]->sizeOfField();
|
|
|
|
|
if (size_of_field == 1)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key8;
|
2014-10-29 01:18:50 +00:00
|
|
|
|
if (size_of_field == 2)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key16;
|
2014-10-29 01:18:50 +00:00
|
|
|
|
if (size_of_field == 4)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key32;
|
2014-10-29 01:18:50 +00:00
|
|
|
|
if (size_of_field == 8)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key64;
|
2014-10-29 01:18:50 +00:00
|
|
|
|
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
}
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-02-22 05:53:16 +00:00
|
|
|
|
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
|
|
|
|
|
if (all_fixed && keys_bytes <= 16)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::keys128;
|
2015-02-22 05:53:16 +00:00
|
|
|
|
if (all_fixed && keys_bytes <= 32)
|
|
|
|
|
return AggregatedDataVariants::Type::keys256;
|
2013-07-19 20:12:02 +00:00
|
|
|
|
|
2012-05-30 01:38:02 +00:00
|
|
|
|
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.keys_size == 1 && typeid_cast<const ColumnString *>(key_columns[0]))
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key_string;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
|
2014-12-30 10:16:23 +00:00
|
|
|
|
return AggregatedDataVariants::Type::key_fixed_string;
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2015-10-04 06:10:48 +00:00
|
|
|
|
/** Если есть массивы.
|
|
|
|
|
* Если есть не более одного массива из элементов фиксированной длины, и остальные ключи фиксированной длины,
|
|
|
|
|
* то всё ещё можно использовать метод concat. Иначе - serialized.
|
|
|
|
|
*/
|
|
|
|
|
if (num_array_keys > 1 || has_arrays_of_non_fixed_elems || (num_array_keys == 1 && !all_non_array_keys_are_fixed))
|
|
|
|
|
return AggregatedDataVariants::Type::serialized;
|
|
|
|
|
|
2015-02-22 16:09:16 +00:00
|
|
|
|
/// Иначе будем агрегировать по конкатенации ключей.
|
|
|
|
|
return AggregatedDataVariants::Type::concat;
|
|
|
|
|
|
|
|
|
|
/// NOTE AggregatedDataVariants::Type::hashed не используется.
|
2012-05-30 01:38:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-05-19 19:41:56 +00:00
|
|
|
|
void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t j = 0; j < params.aggregates_size; ++j)
|
2014-05-19 19:41:56 +00:00
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
/** Может возникнуть исключение при нехватке памяти.
|
|
|
|
|
* Для того, чтобы потом всё правильно уничтожилось, "откатываем" часть созданных состояний.
|
|
|
|
|
* Код не очень удобный.
|
|
|
|
|
*/
|
|
|
|
|
aggregate_functions[j]->create(aggregate_data + offsets_of_aggregate_states[j]);
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
for (size_t rollback_j = 0; rollback_j < j; ++rollback_j)
|
2014-05-20 19:30:39 +00:00
|
|
|
|
aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]);
|
2014-05-19 19:41:56 +00:00
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-10-29 01:18:50 +00:00
|
|
|
|
/** Интересно - если убрать noinline, то gcc зачем-то инлайнит эту функцию, и производительность уменьшается (~10%).
|
|
|
|
|
* (Возможно из-за того, что после инлайна этой функции, перестают инлайниться более внутренние функции.)
|
|
|
|
|
* Инлайнить не имеет смысла, так как внутренний цикл находится целиком внутри этой функции.
|
|
|
|
|
*/
|
2014-05-10 00:31:22 +00:00
|
|
|
|
template <typename Method>
|
2014-10-29 01:18:50 +00:00
|
|
|
|
void NO_INLINE Aggregator::executeImpl(
|
2014-05-10 00:31:22 +00:00
|
|
|
|
Method & method,
|
|
|
|
|
Arena * aggregates_pool,
|
|
|
|
|
size_t rows,
|
|
|
|
|
ConstColumnPlainPtrs & key_columns,
|
2015-11-21 19:46:27 +00:00
|
|
|
|
AggregateFunctionInstruction * aggregate_instructions,
|
2014-05-10 00:31:22 +00:00
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
StringRefs & keys,
|
|
|
|
|
bool no_more_keys,
|
|
|
|
|
AggregateDataPtr overflow_row) const
|
|
|
|
|
{
|
2015-01-03 06:49:32 +00:00
|
|
|
|
typename Method::State state;
|
|
|
|
|
state.init(key_columns);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2014-12-30 10:16:23 +00:00
|
|
|
|
if (!no_more_keys)
|
2015-11-21 19:46:27 +00:00
|
|
|
|
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
|
2014-12-30 10:16:23 +00:00
|
|
|
|
else
|
2015-11-21 19:46:27 +00:00
|
|
|
|
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
|
2014-12-30 10:16:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-01-21 04:23:22 +00:00
|
|
|
|
#ifndef __clang__
|
2014-12-30 12:58:02 +00:00
|
|
|
|
#pragma GCC diagnostic push
|
|
|
|
|
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
|
2015-01-21 04:23:22 +00:00
|
|
|
|
#endif
|
2014-12-30 10:16:23 +00:00
|
|
|
|
|
|
|
|
|
template <bool no_more_keys, typename Method>
|
|
|
|
|
void NO_INLINE Aggregator::executeImplCase(
|
|
|
|
|
Method & method,
|
2015-01-03 06:49:32 +00:00
|
|
|
|
typename Method::State & state,
|
2014-12-30 10:16:23 +00:00
|
|
|
|
Arena * aggregates_pool,
|
|
|
|
|
size_t rows,
|
|
|
|
|
ConstColumnPlainPtrs & key_columns,
|
2015-11-21 19:46:27 +00:00
|
|
|
|
AggregateFunctionInstruction * aggregate_instructions,
|
2014-12-30 10:16:23 +00:00
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
StringRefs & keys,
|
|
|
|
|
AggregateDataPtr overflow_row) const
|
|
|
|
|
{
|
2015-01-13 01:57:22 +00:00
|
|
|
|
/// NOTE При редактировании этого кода, обратите также внимание на SpecializedAggregator.h.
|
|
|
|
|
|
2014-05-10 00:31:22 +00:00
|
|
|
|
/// Для всех строчек.
|
2014-12-30 12:58:02 +00:00
|
|
|
|
typename Method::iterator it;
|
|
|
|
|
typename Method::Key prev_key;
|
2014-05-10 00:31:22 +00:00
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
|
{
|
|
|
|
|
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
|
|
|
|
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
|
|
|
|
|
|
|
|
|
/// Получаем ключ для вставки в хэш-таблицу.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2014-12-30 10:16:23 +00:00
|
|
|
|
if (!no_more_keys) /// Вставляем.
|
2014-12-30 12:58:02 +00:00
|
|
|
|
{
|
|
|
|
|
/// Оптимизация для часто повторяющихся ключей.
|
2015-02-22 18:02:54 +00:00
|
|
|
|
if (!Method::no_consecutive_keys_optimization)
|
2014-12-30 12:58:02 +00:00
|
|
|
|
{
|
2015-02-22 18:02:54 +00:00
|
|
|
|
if (i != 0 && key == prev_key)
|
|
|
|
|
{
|
|
|
|
|
/// Добавляем значения в агрегатные функции.
|
|
|
|
|
AggregateDataPtr value = Method::getAggregateData(it->second);
|
2015-11-21 19:46:27 +00:00
|
|
|
|
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
|
|
|
|
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i);
|
2015-02-22 18:02:54 +00:00
|
|
|
|
|
|
|
|
|
method.onExistingKey(key, keys, *aggregates_pool);
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
prev_key = key;
|
2014-12-30 12:58:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-05-10 00:31:22 +00:00
|
|
|
|
method.data.emplace(key, it, inserted);
|
2014-12-30 12:58:02 +00:00
|
|
|
|
}
|
2014-05-10 00:31:22 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
/// Будем добавлять только если ключ уже есть.
|
|
|
|
|
inserted = false;
|
|
|
|
|
it = method.data.find(key);
|
|
|
|
|
if (method.data.end() == it)
|
|
|
|
|
overflow = true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
2014-12-30 10:16:23 +00:00
|
|
|
|
if (no_more_keys && overflow && !overflow_row)
|
2015-02-22 16:09:16 +00:00
|
|
|
|
{
|
|
|
|
|
method.onExistingKey(key, keys, *aggregates_pool);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
continue;
|
2015-02-22 16:09:16 +00:00
|
|
|
|
}
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2014-08-12 09:35:15 +00:00
|
|
|
|
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
2014-05-10 00:31:22 +00:00
|
|
|
|
if (inserted)
|
|
|
|
|
{
|
|
|
|
|
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
2015-03-07 01:05:06 +00:00
|
|
|
|
|
|
|
|
|
/// exception-safety - если не удалось выделить память или создать состояния, то не будут вызываться деструкторы.
|
|
|
|
|
aggregate_data = nullptr;
|
2015-03-23 01:10:06 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
|
2015-03-23 01:10:06 +00:00
|
|
|
|
|
2015-03-07 01:05:06 +00:00
|
|
|
|
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
|
|
|
|
|
createAggregateStates(place);
|
|
|
|
|
aggregate_data = place;
|
2014-05-10 00:31:22 +00:00
|
|
|
|
}
|
2015-02-22 16:09:16 +00:00
|
|
|
|
else
|
|
|
|
|
method.onExistingKey(key, keys, *aggregates_pool);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
2014-12-30 10:16:23 +00:00
|
|
|
|
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
2014-05-10 00:31:22 +00:00
|
|
|
|
|
|
|
|
|
/// Добавляем значения в агрегатные функции.
|
2015-11-21 19:46:27 +00:00
|
|
|
|
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
|
|
|
|
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i);
|
2014-05-10 00:31:22 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-01-21 04:23:22 +00:00
|
|
|
|
#ifndef __clang__
|
2014-12-30 12:58:02 +00:00
|
|
|
|
#pragma GCC diagnostic pop
|
2015-01-21 04:23:22 +00:00
|
|
|
|
#endif
|
2014-12-30 11:27:58 +00:00
|
|
|
|
|
2015-01-13 03:03:45 +00:00
|
|
|
|
void NO_INLINE Aggregator::executeWithoutKeyImpl(
|
|
|
|
|
AggregatedDataWithoutKey & res,
|
|
|
|
|
size_t rows,
|
2015-11-21 19:46:27 +00:00
|
|
|
|
AggregateFunctionInstruction * aggregate_instructions) const
|
2015-01-13 03:03:45 +00:00
|
|
|
|
{
|
|
|
|
|
/// Оптимизация в случае единственной агрегатной функции count.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateFunctionCount * agg_count = params.aggregates_size == 1
|
2015-01-13 03:03:45 +00:00
|
|
|
|
? typeid_cast<AggregateFunctionCount *>(aggregate_functions[0])
|
|
|
|
|
: NULL;
|
|
|
|
|
|
|
|
|
|
if (agg_count)
|
|
|
|
|
agg_count->addDelta(res, rows);
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
|
{
|
|
|
|
|
/// Добавляем значения
|
2015-11-21 19:46:27 +00:00
|
|
|
|
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
|
|
|
|
(*inst->func)(inst->that, res + inst->state_offset, inst->arguments, i);
|
2015-01-13 03:03:45 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-05-10 05:16:23 +00:00
|
|
|
|
bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
|
|
|
|
ConstColumnPlainPtrs & key_columns, AggregateColumns & aggregate_columns,
|
|
|
|
|
Sizes & key_sizes, StringRefs & key,
|
|
|
|
|
bool & no_more_keys)
|
|
|
|
|
{
|
|
|
|
|
initialize(block);
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return true;
|
|
|
|
|
|
2014-05-10 05:16:23 +00:00
|
|
|
|
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
|
|
|
|
result.aggregator = this;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
aggregate_columns[i].resize(params.aggregates[i].arguments.size());
|
2014-05-10 05:16:23 +00:00
|
|
|
|
|
2015-06-30 19:05:30 +00:00
|
|
|
|
/** Константные столбцы не поддерживаются напрямую при агрегации.
|
|
|
|
|
* Чтобы они всё-равно работали, материализуем их.
|
|
|
|
|
*/
|
|
|
|
|
Columns materialized_columns;
|
|
|
|
|
|
2014-05-10 05:16:23 +00:00
|
|
|
|
/// Запоминаем столбцы, с которыми будем работать
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2014-10-29 01:18:50 +00:00
|
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = block.getByPosition(params.keys[i]).column.get();
|
2014-05-10 05:16:23 +00:00
|
|
|
|
|
2015-10-28 21:21:18 +00:00
|
|
|
|
if (auto converted = key_columns[i]->convertToFullColumnIfConst())
|
2015-06-30 19:05:30 +00:00
|
|
|
|
{
|
2015-10-28 21:21:18 +00:00
|
|
|
|
materialized_columns.push_back(converted);
|
2015-06-30 19:05:30 +00:00
|
|
|
|
key_columns[i] = materialized_columns.back().get();
|
|
|
|
|
}
|
2014-10-29 01:18:50 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateFunctionInstructions aggregate_functions_instructions(params.aggregates_size + 1);
|
|
|
|
|
aggregate_functions_instructions[params.aggregates_size].that = nullptr;
|
2015-11-21 19:46:27 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
{
|
|
|
|
|
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
|
|
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
|
aggregate_columns[i][j] = block.getByPosition(params.aggregates[i].arguments[j]).column.get();
|
2014-05-10 05:16:23 +00:00
|
|
|
|
|
2015-10-28 21:21:18 +00:00
|
|
|
|
if (auto converted = aggregate_columns[i][j]->convertToFullColumnIfConst())
|
2015-06-30 19:05:30 +00:00
|
|
|
|
{
|
2015-10-28 21:21:18 +00:00
|
|
|
|
materialized_columns.push_back(converted);
|
2015-06-30 19:05:30 +00:00
|
|
|
|
aggregate_columns[i][j] = materialized_columns.back().get();
|
|
|
|
|
}
|
2014-05-10 05:16:23 +00:00
|
|
|
|
}
|
2015-11-21 19:46:27 +00:00
|
|
|
|
|
|
|
|
|
aggregate_functions_instructions[i].that = aggregate_functions[i];
|
|
|
|
|
aggregate_functions_instructions[i].func = aggregate_functions[i]->getAddressOfAddFunction();
|
|
|
|
|
aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i];
|
2016-07-31 06:36:39 +00:00
|
|
|
|
aggregate_functions_instructions[i].arguments = aggregate_columns[i].data();
|
2014-05-10 05:16:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return true;
|
|
|
|
|
|
2014-05-10 05:16:23 +00:00
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
|
|
|
|
|
/// Каким способом выполнять агрегацию?
|
|
|
|
|
if (result.empty())
|
|
|
|
|
{
|
|
|
|
|
result.init(chooseAggregationMethod(key_columns, key_sizes));
|
2015-11-30 16:57:05 +00:00
|
|
|
|
result.keys_size = params.keys_size;
|
2014-05-10 05:16:23 +00:00
|
|
|
|
result.key_sizes = key_sizes;
|
|
|
|
|
LOG_TRACE(log, "Aggregation method: " << result.getMethodName());
|
2015-01-10 02:30:03 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.compiler)
|
2015-01-10 02:30:03 +00:00
|
|
|
|
compileIfPossible(result.type);
|
2014-05-10 05:16:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return true;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if ((params.overflow_row || result.type == AggregatedDataVariants::Type::without_key) && !result.without_key)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
{
|
2015-03-07 01:05:06 +00:00
|
|
|
|
AggregateDataPtr place = result.aggregates_pool->alloc(total_size_of_aggregate_states);
|
|
|
|
|
createAggregateStates(place);
|
|
|
|
|
result.without_key = place;
|
2014-05-10 05:16:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-01-13 03:03:45 +00:00
|
|
|
|
/// Выбираем один из методов агрегации и вызываем его.
|
|
|
|
|
|
|
|
|
|
/// Для случая, когда нет ключей (всё агегировать в одну строку).
|
2014-12-30 10:16:23 +00:00
|
|
|
|
if (result.type == AggregatedDataVariants::Type::without_key)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
{
|
2015-01-13 03:03:45 +00:00
|
|
|
|
/// Если есть динамически скомпилированный код.
|
|
|
|
|
if (compiled_data->compiled_method_ptr)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
{
|
2015-01-13 03:03:45 +00:00
|
|
|
|
reinterpret_cast<
|
|
|
|
|
void (*)(const Aggregator &, AggregatedDataWithoutKey &, size_t, AggregateColumns &)>
|
|
|
|
|
(compiled_data->compiled_method_ptr)(*this, result.without_key, rows, aggregate_columns);
|
2014-05-10 05:16:23 +00:00
|
|
|
|
}
|
2015-01-13 03:03:45 +00:00
|
|
|
|
else
|
2015-11-21 19:46:27 +00:00
|
|
|
|
executeWithoutKeyImpl(result.without_key, rows, &aggregate_functions_instructions[0]);
|
2015-01-10 02:30:03 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2015-01-13 03:03:45 +00:00
|
|
|
|
/// Сюда пишутся данные, не поместившиеся в max_rows_to_group_by при group_by_overflow_mode = any.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr;
|
2014-12-30 10:16:23 +00:00
|
|
|
|
|
2015-01-13 03:03:45 +00:00
|
|
|
|
bool is_two_level = result.isTwoLevel();
|
|
|
|
|
|
|
|
|
|
/// Скомпилированный код, для обычной структуры.
|
|
|
|
|
if (!is_two_level && compiled_data->compiled_method_ptr)
|
|
|
|
|
{
|
|
|
|
|
#define M(NAME, IS_TWO_LEVEL) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
reinterpret_cast<void (*)( \
|
|
|
|
|
const Aggregator &, decltype(result.NAME)::element_type &, \
|
|
|
|
|
Arena *, size_t, ConstColumnPlainPtrs &, AggregateColumns &, \
|
|
|
|
|
const Sizes &, StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_method_ptr) \
|
|
|
|
|
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
|
|
|
|
|
result.key_sizes, key, no_more_keys, overflow_row_ptr);
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
/// Скомпилированный код, для two-level структуры.
|
|
|
|
|
else if (is_two_level && compiled_data->compiled_two_level_method_ptr)
|
|
|
|
|
{
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
reinterpret_cast<void (*)( \
|
|
|
|
|
const Aggregator &, decltype(result.NAME)::element_type &, \
|
|
|
|
|
Arena *, size_t, ConstColumnPlainPtrs &, AggregateColumns &, \
|
|
|
|
|
const Sizes &, StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_two_level_method_ptr) \
|
|
|
|
|
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
|
|
|
|
|
result.key_sizes, key, no_more_keys, overflow_row_ptr);
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
/// Когда нет динамически скомпилированного кода.
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
#define M(NAME, IS_TWO_LEVEL) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
2015-11-21 19:46:27 +00:00
|
|
|
|
executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, &aggregate_functions_instructions[0], \
|
2015-01-13 03:03:45 +00:00
|
|
|
|
result.key_sizes, key, no_more_keys, overflow_row_ptr);
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
2015-01-10 02:30:03 +00:00
|
|
|
|
}
|
2014-05-10 05:16:23 +00:00
|
|
|
|
|
2015-01-15 19:29:37 +00:00
|
|
|
|
size_t result_size = result.sizeWithoutOverflowRow();
|
2015-12-07 20:08:00 +00:00
|
|
|
|
Int64 current_memory_usage = 0;
|
|
|
|
|
if (current_memory_tracker)
|
|
|
|
|
current_memory_usage = current_memory_tracker->get();
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Здесь учитываются все результаты в сумме, из разных потоков.
|
|
|
|
|
|
|
|
|
|
bool worth_convert_to_two_level
|
|
|
|
|
= (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold)
|
|
|
|
|
|| (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(params.group_by_two_level_threshold_bytes));
|
2014-12-30 12:58:02 +00:00
|
|
|
|
|
2015-11-30 19:57:46 +00:00
|
|
|
|
/** Преобразование в двухуровневую структуру данных.
|
|
|
|
|
* Она позволяет делать, в последующем, эффективный мердж - либо экономный по памяти, либо распараллеленный.
|
|
|
|
|
*/
|
2015-12-01 14:09:05 +00:00
|
|
|
|
if (result.isConvertibleToTwoLevel() && worth_convert_to_two_level)
|
2014-12-30 12:58:02 +00:00
|
|
|
|
result.convertToTwoLevel();
|
|
|
|
|
|
2014-05-10 05:16:23 +00:00
|
|
|
|
/// Проверка ограничений.
|
2015-10-22 01:44:33 +00:00
|
|
|
|
if (!checkLimits(result_size, no_more_keys))
|
|
|
|
|
return false;
|
|
|
|
|
|
2015-11-30 19:57:46 +00:00
|
|
|
|
/** Сброс данных на диск, если потребляется слишком много оперативки.
|
|
|
|
|
* Данные можно сбросить на диск только если используется двухуровневая структура агрегации.
|
|
|
|
|
*/
|
|
|
|
|
if (params.max_bytes_before_external_group_by
|
|
|
|
|
&& result.isTwoLevel()
|
2015-12-01 14:09:05 +00:00
|
|
|
|
&& current_memory_usage > static_cast<Int64>(params.max_bytes_before_external_group_by)
|
|
|
|
|
&& worth_convert_to_two_level)
|
|
|
|
|
{
|
|
|
|
|
writeToTemporaryFile(result, result_size);
|
|
|
|
|
}
|
2015-11-30 19:57:46 +00:00
|
|
|
|
|
2015-10-22 01:44:33 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, size_t rows)
|
2015-11-30 19:57:46 +00:00
|
|
|
|
{
|
2015-12-01 14:09:05 +00:00
|
|
|
|
Stopwatch watch;
|
|
|
|
|
|
2015-11-30 19:57:46 +00:00
|
|
|
|
auto file = std::make_unique<Poco::TemporaryFile>(params.tmp_path);
|
|
|
|
|
const std::string & path = file->path();
|
|
|
|
|
WriteBufferFromFile file_buf(path);
|
|
|
|
|
CompressedWriteBuffer compressed_buf(file_buf);
|
2016-02-09 17:06:50 +00:00
|
|
|
|
NativeBlockOutputStream block_out(compressed_buf, ClickHouseRevision::get());
|
2015-11-30 19:57:46 +00:00
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
LOG_DEBUG(log, "Writing part of aggregation data into temporary file " << path << ".");
|
2015-12-03 21:23:21 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart);
|
2015-12-01 14:09:05 +00:00
|
|
|
|
|
2015-11-30 19:57:46 +00:00
|
|
|
|
/// Сбрасываем только двухуровневые данные.
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
#define M(NAME) \
|
2015-11-30 19:57:46 +00:00
|
|
|
|
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
2015-12-01 14:09:05 +00:00
|
|
|
|
writeToTemporaryFileImpl(data_variants, *data_variants.NAME, block_out, path);
|
2015-11-30 19:57:46 +00:00
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
/// NOTE Вместо освобождения памяти и создания новых хэш-таблиц и арены, можно переиспользовать старые.
|
|
|
|
|
data_variants.init(data_variants.type);
|
2016-05-28 10:15:36 +00:00
|
|
|
|
data_variants.aggregates_pools = Arenas(1, std::make_shared<Arena>());
|
2015-12-01 14:09:05 +00:00
|
|
|
|
data_variants.aggregates_pool = data_variants.aggregates_pools.back().get();
|
|
|
|
|
|
|
|
|
|
block_out.flush();
|
|
|
|
|
compressed_buf.next();
|
|
|
|
|
file_buf.next();
|
|
|
|
|
|
|
|
|
|
double elapsed_seconds = watch.elapsedSeconds();
|
|
|
|
|
double compressed_bytes = file_buf.count();
|
|
|
|
|
double uncompressed_bytes = compressed_buf.count();
|
|
|
|
|
|
2015-12-01 16:58:15 +00:00
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(temporary_files.mutex);
|
|
|
|
|
temporary_files.files.emplace_back(std::move(file));
|
|
|
|
|
temporary_files.sum_size_uncompressed += uncompressed_bytes;
|
|
|
|
|
temporary_files.sum_size_compressed += compressed_bytes;
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-03 21:23:21 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ExternalAggregationCompressedBytes, compressed_bytes);
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ExternalAggregationUncompressedBytes, uncompressed_bytes);
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
|
|
|
|
<< "Written part in " << elapsed_seconds << " sec., "
|
|
|
|
|
<< rows << " rows, "
|
|
|
|
|
<< (uncompressed_bytes / 1048576.0) << " MiB uncompressed, "
|
|
|
|
|
<< (compressed_bytes / 1048576.0) << " MiB compressed, "
|
|
|
|
|
<< (uncompressed_bytes / rows) << " uncompressed bytes per row, "
|
|
|
|
|
<< (compressed_bytes / rows) << " compressed bytes per row, "
|
|
|
|
|
<< "compression rate: " << (uncompressed_bytes / compressed_bytes)
|
|
|
|
|
<< " (" << (rows / elapsed_seconds) << " rows/sec., "
|
|
|
|
|
<< (uncompressed_bytes / elapsed_seconds / 1048576.0) << " MiB/sec. uncompressed, "
|
|
|
|
|
<< (compressed_bytes / elapsed_seconds / 1048576.0) << " MiB/sec. compressed)");
|
2015-11-30 19:57:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-06 14:27:09 +00:00
|
|
|
|
template <typename Method>
|
|
|
|
|
Block Aggregator::convertOneBucketToBlock(
|
|
|
|
|
AggregatedDataVariants & data_variants,
|
|
|
|
|
Method & method,
|
|
|
|
|
bool final,
|
|
|
|
|
size_t bucket) const
|
|
|
|
|
{
|
|
|
|
|
Block block = prepareBlockAndFill(data_variants, final, method.data.impls[bucket].size(),
|
|
|
|
|
[bucket, &method, this] (
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
AggregateColumnsData & aggregate_columns,
|
|
|
|
|
ColumnPlainPtrs & final_aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
bool final)
|
|
|
|
|
{
|
|
|
|
|
convertToBlockImpl(method, method.data.impls[bucket],
|
|
|
|
|
key_columns, aggregate_columns, final_aggregate_columns, key_sizes, final);
|
|
|
|
|
});
|
|
|
|
|
|
|
|
|
|
block.info.bucket_num = bucket;
|
|
|
|
|
return block;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-11-30 19:57:46 +00:00
|
|
|
|
template <typename Method>
|
|
|
|
|
void Aggregator::writeToTemporaryFileImpl(
|
|
|
|
|
AggregatedDataVariants & data_variants,
|
|
|
|
|
Method & method,
|
|
|
|
|
IBlockOutputStream & out,
|
|
|
|
|
const String & path)
|
|
|
|
|
{
|
2015-12-01 14:09:05 +00:00
|
|
|
|
size_t max_temporary_block_size_rows = 0;
|
|
|
|
|
size_t max_temporary_block_size_bytes = 0;
|
2015-11-30 19:57:46 +00:00
|
|
|
|
|
|
|
|
|
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
|
|
|
|
|
{
|
2015-12-06 14:27:09 +00:00
|
|
|
|
Block block = convertOneBucketToBlock(data_variants, method, false, bucket);
|
2015-11-30 19:57:46 +00:00
|
|
|
|
out.write(block);
|
2015-12-01 14:09:05 +00:00
|
|
|
|
|
|
|
|
|
size_t block_size_rows = block.rowsInFirstColumn();
|
|
|
|
|
size_t block_size_bytes = block.bytes();
|
|
|
|
|
|
|
|
|
|
if (block_size_rows > max_temporary_block_size_rows)
|
|
|
|
|
max_temporary_block_size_rows = block.rowsInFirstColumn();
|
|
|
|
|
if (block_size_bytes > max_temporary_block_size_bytes)
|
|
|
|
|
max_temporary_block_size_bytes = block_size_bytes;
|
2015-11-30 19:57:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-12-01 16:58:15 +00:00
|
|
|
|
/// data_variants не будет уничтожать состояния агрегатных функций в деструкторе. Теперь состояниями владеют ColumnAggregateFunction.
|
|
|
|
|
data_variants.aggregator = nullptr;
|
|
|
|
|
|
2015-12-01 14:09:05 +00:00
|
|
|
|
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
|
|
|
|
<< "Max size of temporary block: " << max_temporary_block_size_rows << " rows, "
|
|
|
|
|
<< (max_temporary_block_size_bytes / 1048576.0) << " MiB.");
|
2015-11-30 19:57:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-10-22 01:44:33 +00:00
|
|
|
|
bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (!no_more_keys && params.max_rows_to_group_by && result_size > params.max_rows_to_group_by)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.group_by_overflow_mode == OverflowMode::THROW)
|
2014-12-30 18:04:53 +00:00
|
|
|
|
throw Exception("Limit for rows to GROUP BY exceeded: has " + toString(result_size)
|
2015-11-30 16:57:05 +00:00
|
|
|
|
+ " rows, maximum: " + toString(params.max_rows_to_group_by),
|
2014-05-10 05:16:23 +00:00
|
|
|
|
ErrorCodes::TOO_MUCH_ROWS);
|
2015-11-30 16:57:05 +00:00
|
|
|
|
else if (params.group_by_overflow_mode == OverflowMode::BREAK)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
return false;
|
2015-11-30 16:57:05 +00:00
|
|
|
|
else if (params.group_by_overflow_mode == OverflowMode::ANY)
|
2014-05-10 05:16:23 +00:00
|
|
|
|
no_more_keys = true;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2011-09-26 07:25:22 +00:00
|
|
|
|
void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & result)
|
2011-09-19 01:42:16 +00:00
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
StringRefs key(params.keys_size);
|
|
|
|
|
ConstColumnPlainPtrs key_columns(params.keys_size);
|
|
|
|
|
AggregateColumns aggregate_columns(params.aggregates_size);
|
2014-05-10 05:16:23 +00:00
|
|
|
|
Sizes key_sizes;
|
2011-09-19 01:42:16 +00:00
|
|
|
|
|
2012-12-25 19:28:59 +00:00
|
|
|
|
/** Используется, если есть ограничение на максимальное количество строк при агрегации,
|
|
|
|
|
* и если group_by_overflow_mode == ANY.
|
|
|
|
|
* В этом случае, новые ключи не добавляются в набор, а производится агрегация только по
|
|
|
|
|
* ключам, которые уже успели попасть в набор.
|
|
|
|
|
*/
|
|
|
|
|
bool no_more_keys = false;
|
|
|
|
|
|
2013-02-04 03:31:53 +00:00
|
|
|
|
LOG_TRACE(log, "Aggregating");
|
2013-02-08 20:34:30 +00:00
|
|
|
|
|
2013-02-04 03:31:53 +00:00
|
|
|
|
Stopwatch watch;
|
|
|
|
|
|
|
|
|
|
size_t src_rows = 0;
|
|
|
|
|
size_t src_bytes = 0;
|
2013-02-09 01:02:52 +00:00
|
|
|
|
|
|
|
|
|
/// Читаем все данные
|
2011-09-19 01:42:16 +00:00
|
|
|
|
while (Block block = stream->read())
|
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2013-02-04 03:31:53 +00:00
|
|
|
|
src_rows += block.rows();
|
|
|
|
|
src_bytes += block.bytes();
|
2011-09-25 05:07:47 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (!executeOnBlock(block, result,
|
|
|
|
|
key_columns, aggregate_columns, key_sizes, key,
|
|
|
|
|
no_more_keys))
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
double elapsed_seconds = watch.elapsedSeconds();
|
|
|
|
|
size_t rows = result.size();
|
|
|
|
|
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
|
|
|
|
<< "Aggregated. " << src_rows << " to " << rows << " rows (from " << src_bytes / 1048576.0 << " MiB)"
|
|
|
|
|
<< " in " << elapsed_seconds << " sec."
|
|
|
|
|
<< " (" << src_rows / elapsed_seconds << " rows/sec., " << src_bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void Aggregator::convertToBlockImpl(
|
|
|
|
|
Method & method,
|
|
|
|
|
Table & data,
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
AggregateColumnsData & aggregate_columns,
|
|
|
|
|
ColumnPlainPtrs & final_aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
bool final) const
|
|
|
|
|
{
|
2015-12-11 00:34:00 +00:00
|
|
|
|
if (data.empty())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (final)
|
|
|
|
|
convertToBlockImplFinal(method, data, key_columns, final_aggregate_columns, key_sizes);
|
|
|
|
|
else
|
|
|
|
|
convertToBlockImplNotFinal(method, data, key_columns, aggregate_columns, key_sizes);
|
2015-12-06 19:42:28 +00:00
|
|
|
|
|
|
|
|
|
/// Для того, чтобы пораньше освободить память.
|
|
|
|
|
data.clearAndShrink();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::convertToBlockImplFinal(
|
|
|
|
|
Method & method,
|
|
|
|
|
Table & data,
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
ColumnPlainPtrs & final_aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes) const
|
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
for (const auto & value : data)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
method.insertKeyIntoColumns(value, key_columns, params.keys_size, key_sizes);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->insertResultInto(
|
2015-12-23 07:29:20 +00:00
|
|
|
|
Method::getAggregateData(value.second) + offsets_of_aggregate_states[i],
|
2015-01-03 05:39:21 +00:00
|
|
|
|
*final_aggregate_columns[i]);
|
|
|
|
|
}
|
2015-12-06 19:42:28 +00:00
|
|
|
|
|
|
|
|
|
destroyImpl(method, data); /// NOTE Можно сделать лучше.
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::convertToBlockImplNotFinal(
|
|
|
|
|
Method & method,
|
|
|
|
|
Table & data,
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
AggregateColumnsData & aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes) const
|
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
for (auto & value : data)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
method.insertKeyIntoColumns(value, key_columns, params.keys_size, key_sizes);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
/// reserved, поэтому push_back не кидает исключений
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-12-23 07:29:20 +00:00
|
|
|
|
aggregate_columns[i]->push_back(Method::getAggregateData(value.second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
|
|
|
|
Method::getAggregateData(value.second) = nullptr;
|
2011-09-26 07:25:22 +00:00
|
|
|
|
}
|
2011-09-19 01:42:16 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
template <typename Filler>
|
|
|
|
|
Block Aggregator::prepareBlockAndFill(
|
|
|
|
|
AggregatedDataVariants & data_variants,
|
|
|
|
|
bool final,
|
|
|
|
|
size_t rows,
|
|
|
|
|
Filler && filler) const
|
2012-02-27 06:28:20 +00:00
|
|
|
|
{
|
2013-09-16 05:33:49 +00:00
|
|
|
|
Block res = sample.cloneEmpty();
|
2012-05-10 07:47:13 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
ColumnPlainPtrs key_columns(params.keys_size);
|
|
|
|
|
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
|
|
|
|
ColumnPlainPtrs final_aggregate_columns(params.aggregates_size);
|
2013-02-03 18:08:52 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2013-02-03 18:39:09 +00:00
|
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = res.getByPosition(i).column.get();
|
2013-02-03 18:39:09 +00:00
|
|
|
|
key_columns[i]->reserve(rows);
|
|
|
|
|
}
|
2013-02-03 18:08:52 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2013-02-03 18:39:09 +00:00
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (!final)
|
2014-05-26 16:11:20 +00:00
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
/// Столбец ColumnAggregateFunction захватывает разделяемое владение ареной с состояниями агрегатных функций.
|
|
|
|
|
ColumnAggregateFunction & column_aggregate_func = static_cast<ColumnAggregateFunction &>(
|
|
|
|
|
*res.getByPosition(i + params.keys_size).column);
|
|
|
|
|
|
|
|
|
|
for (size_t j = 0; j < data_variants.aggregates_pools.size(); ++j)
|
|
|
|
|
column_aggregate_func.addArena(data_variants.aggregates_pools[j]);
|
|
|
|
|
|
|
|
|
|
aggregate_columns[i] = &column_aggregate_func.getData();
|
|
|
|
|
aggregate_columns[i]->reserve(rows);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
ColumnWithTypeAndName & column = res.getByPosition(i + params.keys_size);
|
|
|
|
|
column.type = aggregate_functions[i]->getReturnType();
|
|
|
|
|
column.column = column.type->createColumn();
|
|
|
|
|
column.column->reserve(rows);
|
|
|
|
|
|
|
|
|
|
if (aggregate_functions[i]->isState())
|
2014-05-22 18:58:41 +00:00
|
|
|
|
{
|
|
|
|
|
/// Столбец ColumnAggregateFunction захватывает разделяемое владение ареной с состояниями агрегатных функций.
|
2015-12-23 07:29:20 +00:00
|
|
|
|
ColumnAggregateFunction & column_aggregate_func = static_cast<ColumnAggregateFunction &>(*column.column);
|
2014-05-22 18:58:41 +00:00
|
|
|
|
|
|
|
|
|
for (size_t j = 0; j < data_variants.aggregates_pools.size(); ++j)
|
|
|
|
|
column_aggregate_func.addArena(data_variants.aggregates_pools[j]);
|
|
|
|
|
}
|
2015-01-02 03:50:14 +00:00
|
|
|
|
|
2016-05-28 05:31:36 +00:00
|
|
|
|
final_aggregate_columns[i] = column.column.get();
|
2015-01-02 03:16:28 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-06-05 19:52:13 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
filler(key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final);
|
|
|
|
|
|
|
|
|
|
/// Изменяем размер столбцов-констант в блоке.
|
|
|
|
|
size_t columns = res.columns();
|
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
|
if (res.getByPosition(i).column->isConst())
|
|
|
|
|
res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows);
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
2014-05-28 14:54:42 +00:00
|
|
|
|
|
2014-05-22 18:58:41 +00:00
|
|
|
|
|
2015-11-09 18:45:55 +00:00
|
|
|
|
BlocksList Aggregator::prepareBlocksAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const
|
2015-01-02 03:16:28 +00:00
|
|
|
|
{
|
|
|
|
|
size_t rows = 1;
|
|
|
|
|
|
|
|
|
|
auto filler = [&data_variants, this](
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
AggregateColumnsData & aggregate_columns,
|
|
|
|
|
ColumnPlainPtrs & final_aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
bool final)
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
2013-11-03 23:35:18 +00:00
|
|
|
|
{
|
2014-05-22 18:58:41 +00:00
|
|
|
|
AggregatedDataWithoutKey & data = data_variants.without_key;
|
2013-02-08 20:34:30 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-12-23 07:29:20 +00:00
|
|
|
|
{
|
2014-05-28 14:54:42 +00:00
|
|
|
|
if (!final)
|
2015-12-23 07:29:20 +00:00
|
|
|
|
aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]);
|
2014-05-22 18:58:41 +00:00
|
|
|
|
else
|
|
|
|
|
aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]);
|
2015-12-23 07:29:20 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!final)
|
|
|
|
|
data = nullptr;
|
2014-05-22 18:58:41 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.overflow_row)
|
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2014-05-22 18:58:41 +00:00
|
|
|
|
key_columns[i]->insertDefault();
|
2013-11-03 23:35:18 +00:00
|
|
|
|
}
|
2015-01-02 03:16:28 +00:00
|
|
|
|
};
|
2013-02-03 18:08:52 +00:00
|
|
|
|
|
2015-01-03 03:18:49 +00:00
|
|
|
|
Block block = prepareBlockAndFill(data_variants, final, rows, filler);
|
2015-11-09 18:45:55 +00:00
|
|
|
|
|
|
|
|
|
if (is_overflows)
|
2015-01-03 03:18:49 +00:00
|
|
|
|
block.info.is_overflows = true;
|
|
|
|
|
|
2015-12-06 19:42:28 +00:00
|
|
|
|
if (final)
|
|
|
|
|
destroyWithoutKey(data_variants);
|
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
BlocksList blocks;
|
2015-01-03 03:18:49 +00:00
|
|
|
|
blocks.emplace_back(std::move(block));
|
2015-01-02 03:16:28 +00:00
|
|
|
|
return blocks;
|
|
|
|
|
}
|
2014-05-22 18:58:41 +00:00
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
BlocksList Aggregator::prepareBlocksAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const
|
|
|
|
|
{
|
2015-01-15 19:29:37 +00:00
|
|
|
|
size_t rows = data_variants.sizeWithoutOverflowRow();
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
|
|
|
|
auto filler = [&data_variants, this](
|
|
|
|
|
ColumnPlainPtrs & key_columns,
|
|
|
|
|
AggregateColumnsData & aggregate_columns,
|
|
|
|
|
ColumnPlainPtrs & final_aggregate_columns,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
bool final)
|
|
|
|
|
{
|
2015-12-06 19:42:28 +00:00
|
|
|
|
#define M(NAME) \
|
2014-12-30 10:16:23 +00:00
|
|
|
|
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
2015-01-02 03:16:28 +00:00
|
|
|
|
convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \
|
|
|
|
|
key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final);
|
2014-12-30 10:16:23 +00:00
|
|
|
|
|
|
|
|
|
if (false) {}
|
2015-12-06 19:42:28 +00:00
|
|
|
|
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
#undef M
|
2015-01-02 03:16:28 +00:00
|
|
|
|
else
|
2014-05-22 18:58:41 +00:00
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
2015-01-02 03:16:28 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
BlocksList blocks;
|
|
|
|
|
blocks.emplace_back(prepareBlockAndFill(data_variants, final, rows, filler));
|
|
|
|
|
return blocks;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2016-08-02 01:46:05 +00:00
|
|
|
|
BlocksList Aggregator::prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const
|
2015-01-02 03:16:28 +00:00
|
|
|
|
{
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
return prepareBlocksAndFillTwoLevelImpl(data_variants, *data_variants.NAME, final, thread_pool);
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method>
|
|
|
|
|
BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
|
|
|
|
|
AggregatedDataVariants & data_variants,
|
|
|
|
|
Method & method,
|
|
|
|
|
bool final,
|
2016-08-02 01:46:05 +00:00
|
|
|
|
ThreadPool * thread_pool) const
|
2015-01-02 03:16:28 +00:00
|
|
|
|
{
|
|
|
|
|
auto converter = [&](size_t bucket, MemoryTracker * memory_tracker)
|
|
|
|
|
{
|
|
|
|
|
current_memory_tracker = memory_tracker;
|
2015-12-06 14:27:09 +00:00
|
|
|
|
return convertOneBucketToBlock(data_variants, method, final, bucket);
|
2015-01-02 03:16:28 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
/// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
|
|
|
|
|
|
2015-03-19 05:15:38 +00:00
|
|
|
|
std::vector<std::packaged_task<Block()>> tasks(Method::Data::NUM_BUCKETS);
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
2015-03-16 19:24:57 +00:00
|
|
|
|
try
|
2015-01-02 03:16:28 +00:00
|
|
|
|
{
|
2015-03-16 19:24:57 +00:00
|
|
|
|
for (size_t bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
|
|
|
|
|
{
|
|
|
|
|
if (method.data.impls[bucket].empty())
|
|
|
|
|
continue;
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
2015-03-19 05:15:38 +00:00
|
|
|
|
tasks[bucket] = std::packaged_task<Block()>(std::bind(converter, bucket, current_memory_tracker));
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
2015-03-16 19:24:57 +00:00
|
|
|
|
if (thread_pool)
|
|
|
|
|
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
|
|
|
|
|
else
|
|
|
|
|
tasks[bucket]();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
/// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо.
|
2015-01-02 03:16:28 +00:00
|
|
|
|
if (thread_pool)
|
2015-03-16 19:24:57 +00:00
|
|
|
|
thread_pool->wait();
|
|
|
|
|
|
|
|
|
|
throw;
|
2015-01-02 03:16:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (thread_pool)
|
|
|
|
|
thread_pool->wait();
|
|
|
|
|
|
|
|
|
|
BlocksList blocks;
|
2015-01-02 03:50:14 +00:00
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
for (auto & task : tasks)
|
2015-01-02 03:50:14 +00:00
|
|
|
|
{
|
2015-03-19 05:15:38 +00:00
|
|
|
|
if (!task.valid())
|
|
|
|
|
continue;
|
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
blocks.emplace_back(task.get_future().get());
|
2015-01-02 03:50:14 +00:00
|
|
|
|
}
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
|
|
|
|
return blocks;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-08 02:01:46 +00:00
|
|
|
|
BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) const
|
2015-01-02 03:16:28 +00:00
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return BlocksList();
|
|
|
|
|
|
2015-03-07 01:05:06 +00:00
|
|
|
|
LOG_TRACE(log, "Converting aggregated data to blocks");
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
|
|
|
|
Stopwatch watch;
|
|
|
|
|
|
|
|
|
|
BlocksList blocks;
|
|
|
|
|
|
|
|
|
|
/// В какой структуре данных агрегированы данные?
|
|
|
|
|
if (data_variants.empty())
|
|
|
|
|
return blocks;
|
|
|
|
|
|
2016-08-02 01:46:05 +00:00
|
|
|
|
std::unique_ptr<ThreadPool> thread_pool;
|
2015-01-15 19:29:37 +00:00
|
|
|
|
if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Сделать настраиваемый порог.
|
2015-01-02 03:50:14 +00:00
|
|
|
|
&& data_variants.isTwoLevel()) /// TODO Использовать общий тред-пул с функцией merge.
|
2016-08-02 01:46:05 +00:00
|
|
|
|
thread_pool.reset(new ThreadPool(max_threads));
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return BlocksList();
|
2015-04-16 14:27:56 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
|
|
|
|
blocks.splice(blocks.end(), prepareBlocksAndFillWithoutKey(
|
|
|
|
|
data_variants, final, data_variants.type != AggregatedDataVariants::Type::without_key));
|
2015-01-02 03:16:28 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return BlocksList();
|
2015-04-16 14:27:56 +00:00
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (data_variants.type != AggregatedDataVariants::Type::without_key)
|
2015-01-02 03:50:14 +00:00
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
if (!data_variants.isTwoLevel())
|
|
|
|
|
blocks.splice(blocks.end(), prepareBlocksAndFillSingleLevel(data_variants, final));
|
|
|
|
|
else
|
|
|
|
|
blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, thread_pool.get()));
|
2014-05-22 18:58:41 +00:00
|
|
|
|
}
|
2014-02-26 11:44:54 +00:00
|
|
|
|
|
2013-11-03 23:35:18 +00:00
|
|
|
|
if (!final)
|
|
|
|
|
{
|
2015-12-23 07:29:20 +00:00
|
|
|
|
/// data_variants не будет уничтожать состояния агрегатных функций в деструкторе.
|
|
|
|
|
/// Теперь состояниями владеют ColumnAggregateFunction.
|
2014-04-08 07:31:51 +00:00
|
|
|
|
data_variants.aggregator = nullptr;
|
2013-11-03 23:35:18 +00:00
|
|
|
|
}
|
2013-02-16 18:59:05 +00:00
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return BlocksList();
|
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
size_t rows = 0;
|
|
|
|
|
size_t bytes = 0;
|
|
|
|
|
|
|
|
|
|
for (const auto & block : blocks)
|
|
|
|
|
{
|
|
|
|
|
rows += block.rowsInFirstColumn();
|
|
|
|
|
bytes += block.bytes();
|
|
|
|
|
}
|
2012-02-27 06:28:20 +00:00
|
|
|
|
|
2013-02-03 18:39:09 +00:00
|
|
|
|
double elapsed_seconds = watch.elapsedSeconds();
|
2013-02-04 03:31:53 +00:00
|
|
|
|
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
2015-01-16 21:01:41 +00:00
|
|
|
|
<< "Converted aggregated data to blocks. "
|
2015-01-02 03:16:28 +00:00
|
|
|
|
<< rows << " rows, " << bytes / 1048576.0 << " MiB"
|
2013-02-04 03:31:53 +00:00
|
|
|
|
<< " in " << elapsed_seconds << " sec."
|
2015-01-02 03:16:28 +00:00
|
|
|
|
<< " (" << rows / elapsed_seconds << " rows/sec., " << bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
2012-05-31 00:33:42 +00:00
|
|
|
|
|
2015-01-02 03:16:28 +00:00
|
|
|
|
return blocks;
|
2012-02-27 06:28:20 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::mergeDataImpl(
|
|
|
|
|
Table & table_dst,
|
|
|
|
|
Table & table_src) const
|
|
|
|
|
{
|
|
|
|
|
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
decltype(it) res_it;
|
|
|
|
|
bool inserted;
|
|
|
|
|
table_dst.emplace(it->first, res_it, inserted, it.getHash());
|
|
|
|
|
|
|
|
|
|
if (!inserted)
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->merge(
|
|
|
|
|
Method::getAggregateData(res_it->second) + offsets_of_aggregate_states[i],
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->destroy(
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
res_it->second = it->second;
|
|
|
|
|
}
|
2015-03-16 21:31:40 +00:00
|
|
|
|
|
|
|
|
|
Method::getAggregateData(it->second) = nullptr;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
2015-12-23 07:06:34 +00:00
|
|
|
|
|
|
|
|
|
table_src.clearAndShrink();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-10-22 01:44:33 +00:00
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
|
|
|
|
|
Table & table_dst,
|
|
|
|
|
AggregatedDataWithoutKey & overflows,
|
|
|
|
|
Table & table_src) const
|
|
|
|
|
{
|
|
|
|
|
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
decltype(it) res_it = table_dst.find(it->first, it.getHash());
|
|
|
|
|
|
|
|
|
|
AggregateDataPtr res_data = table_dst.end() == res_it
|
|
|
|
|
? overflows
|
|
|
|
|
: Method::getAggregateData(res_it->second);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-10-22 01:44:33 +00:00
|
|
|
|
aggregate_functions[i]->merge(
|
|
|
|
|
res_data + offsets_of_aggregate_states[i],
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-10-22 01:44:33 +00:00
|
|
|
|
aggregate_functions[i]->destroy(
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
|
|
|
|
Method::getAggregateData(it->second) = nullptr;
|
|
|
|
|
}
|
2015-12-23 07:06:34 +00:00
|
|
|
|
|
|
|
|
|
table_src.clearAndShrink();
|
2015-10-22 01:44:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
|
|
|
|
|
Table & table_dst,
|
|
|
|
|
Table & table_src) const
|
|
|
|
|
{
|
|
|
|
|
for (auto it = table_src.begin(); it != table_src.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
decltype(it) res_it = table_dst.find(it->first, it.getHash());
|
|
|
|
|
|
|
|
|
|
if (table_dst.end() == res_it)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
AggregateDataPtr res_data = Method::getAggregateData(res_it->second);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-10-22 01:44:33 +00:00
|
|
|
|
aggregate_functions[i]->merge(
|
|
|
|
|
res_data + offsets_of_aggregate_states[i],
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-10-22 01:44:33 +00:00
|
|
|
|
aggregate_functions[i]->destroy(
|
|
|
|
|
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
|
|
|
|
Method::getAggregateData(it->second) = nullptr;
|
|
|
|
|
}
|
2015-12-23 07:06:34 +00:00
|
|
|
|
|
|
|
|
|
table_src.clearAndShrink();
|
2015-10-22 01:44:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
|
|
|
|
|
ManyAggregatedDataVariants & non_empty_data) const
|
|
|
|
|
{
|
|
|
|
|
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
|
|
|
|
|
|
|
|
|
/// Все результаты агрегации соединяем с первым.
|
|
|
|
|
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
AggregatedDataWithoutKey & res_data = res->without_key;
|
|
|
|
|
AggregatedDataWithoutKey & current_data = non_empty_data[i]->without_key;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->merge(res_data + offsets_of_aggregate_states[i], current_data + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->destroy(current_data + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
|
|
|
|
current_data = nullptr;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method>
|
|
|
|
|
void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
|
|
|
|
|
ManyAggregatedDataVariants & non_empty_data) const
|
|
|
|
|
{
|
|
|
|
|
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
2015-10-22 01:44:33 +00:00
|
|
|
|
bool no_more_keys = false;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Все результаты агрегации соединяем с первым.
|
|
|
|
|
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
|
|
|
|
{
|
2015-10-22 01:44:33 +00:00
|
|
|
|
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
|
|
|
|
break;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
AggregatedDataVariants & current = *non_empty_data[i];
|
|
|
|
|
|
2015-10-22 01:44:33 +00:00
|
|
|
|
if (!no_more_keys)
|
|
|
|
|
mergeDataImpl<Method>(
|
|
|
|
|
getDataVariant<Method>(*res).data,
|
|
|
|
|
getDataVariant<Method>(current).data);
|
2015-10-30 02:29:53 +00:00
|
|
|
|
else if (res->without_key)
|
2015-10-22 01:44:33 +00:00
|
|
|
|
mergeDataNoMoreKeysImpl<Method>(
|
|
|
|
|
getDataVariant<Method>(*res).data,
|
|
|
|
|
res->without_key,
|
|
|
|
|
getDataVariant<Method>(current).data);
|
2015-10-30 02:29:53 +00:00
|
|
|
|
else
|
|
|
|
|
mergeDataOnlyExistingKeysImpl<Method>(
|
|
|
|
|
getDataVariant<Method>(*res).data,
|
|
|
|
|
getDataVariant<Method>(current).data);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// current не будет уничтожать состояния агрегатных функций в деструкторе
|
|
|
|
|
current.aggregator = nullptr;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-06 16:22:01 +00:00
|
|
|
|
template <typename Method>
|
|
|
|
|
void NO_INLINE Aggregator::mergeBucketImpl(
|
|
|
|
|
ManyAggregatedDataVariants & data, Int32 bucket) const
|
|
|
|
|
{
|
|
|
|
|
/// Все результаты агрегации соединяем с первым.
|
|
|
|
|
AggregatedDataVariantsPtr & res = data[0];
|
|
|
|
|
for (size_t i = 1, size = data.size(); i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
AggregatedDataVariants & current = *data[i];
|
|
|
|
|
|
|
|
|
|
mergeDataImpl<Method>(
|
|
|
|
|
getDataVariant<Method>(*res).data.impls[bucket],
|
|
|
|
|
getDataVariant<Method>(current).data.impls[bucket]);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** Объединят вместе состояния агрегации, превращает их в блоки и выдаёт потоково.
|
|
|
|
|
* Если состояния агрегации двухуровневые, то выдаёт блоки строго по порядку bucket_num.
|
|
|
|
|
* (Это важно при распределённой обработке.)
|
|
|
|
|
* При этом, может обрабатывать разные bucket-ы параллельно, используя до threads потоков.
|
|
|
|
|
*/
|
|
|
|
|
class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
/** На вход подаётся набор непустых множеств частично агрегированных данных,
|
|
|
|
|
* которые все либо являются одноуровневыми, либо являются двухуровневыми.
|
|
|
|
|
*/
|
2015-12-08 02:01:46 +00:00
|
|
|
|
MergingAndConvertingBlockInputStream(const Aggregator & aggregator_, ManyAggregatedDataVariants & data_, bool final_, size_t threads_)
|
2015-12-06 16:22:01 +00:00
|
|
|
|
: aggregator(aggregator_), data(data_), final(final_), threads(threads_) {}
|
|
|
|
|
|
|
|
|
|
String getName() const override { return "MergingAndConverting"; }
|
|
|
|
|
|
|
|
|
|
String getID() const override
|
|
|
|
|
{
|
|
|
|
|
std::stringstream res;
|
|
|
|
|
res << this;
|
|
|
|
|
return res.str();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected:
|
|
|
|
|
Block readImpl() override
|
|
|
|
|
{
|
|
|
|
|
if (data.empty())
|
|
|
|
|
return {};
|
|
|
|
|
|
|
|
|
|
if (current_bucket_num >= NUM_BUCKETS)
|
|
|
|
|
return {};
|
|
|
|
|
|
|
|
|
|
AggregatedDataVariantsPtr & first = data[0];
|
|
|
|
|
|
|
|
|
|
if (current_bucket_num == -1)
|
|
|
|
|
{
|
|
|
|
|
++current_bucket_num;
|
|
|
|
|
|
|
|
|
|
if (first->type == AggregatedDataVariants::Type::without_key || aggregator.params.overflow_row)
|
|
|
|
|
{
|
|
|
|
|
aggregator.mergeWithoutKeyDataImpl(data);
|
|
|
|
|
return aggregator.prepareBlocksAndFillWithoutKey(
|
|
|
|
|
*first, final, first->type != AggregatedDataVariants::Type::without_key).front();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!first->isTwoLevel())
|
|
|
|
|
{
|
|
|
|
|
if (current_bucket_num > 0)
|
|
|
|
|
return {};
|
|
|
|
|
|
2015-12-06 16:42:17 +00:00
|
|
|
|
if (first->type == AggregatedDataVariants::Type::without_key)
|
|
|
|
|
return {};
|
|
|
|
|
|
2015-12-06 16:22:01 +00:00
|
|
|
|
++current_bucket_num;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (first->type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
aggregator.mergeSingleLevelDataImpl<decltype(first->NAME)::element_type>(data);
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
|
|
|
|
|
#undef M
|
2015-12-06 16:42:17 +00:00
|
|
|
|
else
|
2015-12-06 16:22:01 +00:00
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
|
|
|
|
return aggregator.prepareBlocksAndFillSingleLevel(*first, final).front();
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
if (!parallel_merge_data)
|
|
|
|
|
{
|
|
|
|
|
parallel_merge_data.reset(new ParallelMergeData(threads));
|
|
|
|
|
for (size_t i = 0; i < threads; ++i)
|
|
|
|
|
scheduleThreadForNextBucket();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Block res;
|
|
|
|
|
|
|
|
|
|
while (true)
|
|
|
|
|
{
|
|
|
|
|
std::unique_lock<std::mutex> lock(parallel_merge_data->mutex);
|
|
|
|
|
|
|
|
|
|
if (parallel_merge_data->exception)
|
|
|
|
|
std::rethrow_exception(parallel_merge_data->exception);
|
|
|
|
|
|
|
|
|
|
auto it = parallel_merge_data->ready_blocks.find(current_bucket_num);
|
|
|
|
|
if (it != parallel_merge_data->ready_blocks.end())
|
|
|
|
|
{
|
|
|
|
|
++current_bucket_num;
|
|
|
|
|
scheduleThreadForNextBucket();
|
|
|
|
|
|
2015-12-11 01:11:59 +00:00
|
|
|
|
if (it->second)
|
2015-12-06 16:22:01 +00:00
|
|
|
|
{
|
|
|
|
|
res.swap(it->second);
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else if (current_bucket_num >= NUM_BUCKETS)
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
parallel_merge_data->condvar.wait(lock);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private:
|
2015-12-08 02:01:46 +00:00
|
|
|
|
const Aggregator & aggregator;
|
2015-12-06 16:22:01 +00:00
|
|
|
|
ManyAggregatedDataVariants data;
|
|
|
|
|
bool final;
|
|
|
|
|
size_t threads;
|
|
|
|
|
|
|
|
|
|
Int32 current_bucket_num = -1;
|
|
|
|
|
Int32 max_scheduled_bucket_num = -1;
|
|
|
|
|
static constexpr Int32 NUM_BUCKETS = 256;
|
|
|
|
|
|
|
|
|
|
struct ParallelMergeData
|
|
|
|
|
{
|
2016-08-02 01:46:05 +00:00
|
|
|
|
ThreadPool pool;
|
2015-12-06 16:22:01 +00:00
|
|
|
|
std::map<Int32, Block> ready_blocks;
|
|
|
|
|
std::exception_ptr exception;
|
|
|
|
|
std::mutex mutex;
|
|
|
|
|
std::condition_variable condvar;
|
|
|
|
|
|
|
|
|
|
ParallelMergeData(size_t threads) : pool(threads) {}
|
2015-12-23 07:35:36 +00:00
|
|
|
|
|
|
|
|
|
~ParallelMergeData()
|
|
|
|
|
{
|
|
|
|
|
LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
|
|
|
|
|
pool.wait();
|
|
|
|
|
}
|
2015-12-06 16:22:01 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
std::unique_ptr<ParallelMergeData> parallel_merge_data;
|
|
|
|
|
|
|
|
|
|
void scheduleThreadForNextBucket()
|
|
|
|
|
{
|
|
|
|
|
++max_scheduled_bucket_num;
|
|
|
|
|
if (max_scheduled_bucket_num >= NUM_BUCKETS)
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this,
|
|
|
|
|
max_scheduled_bucket_num, current_memory_tracker));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void thread(Int32 bucket_num, MemoryTracker * memory_tracker)
|
|
|
|
|
{
|
|
|
|
|
current_memory_tracker = memory_tracker;
|
|
|
|
|
setThreadName("MergingAggregtd");
|
2016-01-21 01:47:28 +00:00
|
|
|
|
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
|
2015-12-06 16:22:01 +00:00
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
/// TODO Возможно, поддержать no_more_keys
|
|
|
|
|
|
|
|
|
|
auto & merged_data = *data[0];
|
|
|
|
|
auto method = merged_data.type;
|
|
|
|
|
Block block;
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (method == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
{ \
|
|
|
|
|
aggregator.mergeBucketImpl<decltype(merged_data.NAME)::element_type>(data, bucket_num); \
|
|
|
|
|
block = aggregator.convertOneBucketToBlock(merged_data, *merged_data.NAME, final, bucket_num); \
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
|
|
|
|
|
std::lock_guard<std::mutex> lock(parallel_merge_data->mutex);
|
|
|
|
|
parallel_merge_data->ready_blocks[bucket_num] = std::move(block);
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(parallel_merge_data->mutex);
|
2015-12-23 07:06:34 +00:00
|
|
|
|
if (!parallel_merge_data->exception)
|
|
|
|
|
parallel_merge_data->exception = std::current_exception();
|
2015-12-06 16:22:01 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
parallel_merge_data->condvar.notify_all();
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2015-12-23 07:29:20 +00:00
|
|
|
|
std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
|
|
|
|
|
ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const
|
2015-12-06 16:22:01 +00:00
|
|
|
|
{
|
|
|
|
|
if (data_variants.empty())
|
|
|
|
|
throw Exception("Empty data passed to Aggregator::mergeAndConvertToBlocks.", ErrorCodes::EMPTY_DATA_PASSED);
|
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Merging aggregated data");
|
|
|
|
|
|
|
|
|
|
ManyAggregatedDataVariants non_empty_data;
|
|
|
|
|
non_empty_data.reserve(data_variants.size());
|
|
|
|
|
for (auto & data : data_variants)
|
|
|
|
|
if (!data->empty())
|
|
|
|
|
non_empty_data.push_back(data);
|
|
|
|
|
|
|
|
|
|
if (non_empty_data.empty())
|
2016-05-28 12:22:22 +00:00
|
|
|
|
return std::make_unique<NullBlockInputStream>();
|
2015-12-06 16:22:01 +00:00
|
|
|
|
|
|
|
|
|
if (non_empty_data.size() > 1)
|
|
|
|
|
{
|
|
|
|
|
/// Отсортируем состояния по убыванию размера, чтобы мердж был более эффективным (так как все состояния мерджатся в первое).
|
|
|
|
|
std::sort(non_empty_data.begin(), non_empty_data.end(),
|
|
|
|
|
[](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
|
|
|
|
|
{
|
|
|
|
|
return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow();
|
|
|
|
|
});
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если хотя бы один из вариантов двухуровневый, то переконвертируем все варианты в двухуровневые, если есть не такие.
|
|
|
|
|
/// Замечание - возможно, было бы более оптимально не конвертировать одноуровневые варианты перед мерджем, а мерджить их отдельно, в конце.
|
|
|
|
|
|
|
|
|
|
bool has_at_least_one_two_level = false;
|
|
|
|
|
for (const auto & variant : non_empty_data)
|
|
|
|
|
{
|
|
|
|
|
if (variant->isTwoLevel())
|
|
|
|
|
{
|
|
|
|
|
has_at_least_one_two_level = true;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (has_at_least_one_two_level)
|
|
|
|
|
for (auto & variant : non_empty_data)
|
|
|
|
|
if (!variant->isTwoLevel())
|
|
|
|
|
variant->convertToTwoLevel();
|
|
|
|
|
|
|
|
|
|
AggregatedDataVariantsPtr & first = non_empty_data[0];
|
|
|
|
|
|
|
|
|
|
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
2015-12-23 06:46:48 +00:00
|
|
|
|
{
|
2015-12-06 16:22:01 +00:00
|
|
|
|
if (first->type != non_empty_data[i]->type)
|
|
|
|
|
throw Exception("Cannot merge different aggregated data variants.", ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS);
|
|
|
|
|
|
2015-12-23 06:46:48 +00:00
|
|
|
|
/** В первое множество данных могут быть перемещены элементы из остальных множеств.
|
|
|
|
|
* Поэтому, оно должно владеть всеми аренами всех остальных множеств.
|
|
|
|
|
*/
|
|
|
|
|
first->aggregates_pools.insert(first->aggregates_pools.end(),
|
|
|
|
|
non_empty_data[i]->aggregates_pools.begin(), non_empty_data[i]->aggregates_pools.end());
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-28 12:22:22 +00:00
|
|
|
|
return std::make_unique<MergingAndConvertingBlockInputStream>(*this, non_empty_data, final, max_threads);
|
2015-12-06 16:22:01 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
template <bool no_more_keys, typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::mergeStreamsImplCase(
|
2015-01-03 05:39:21 +00:00
|
|
|
|
Block & block,
|
2015-03-23 01:10:06 +00:00
|
|
|
|
const Sizes & key_sizes,
|
2015-01-03 06:49:32 +00:00
|
|
|
|
Arena * aggregates_pool,
|
2015-01-03 05:39:21 +00:00
|
|
|
|
Method & method,
|
2015-10-23 01:43:42 +00:00
|
|
|
|
Table & data,
|
|
|
|
|
AggregateDataPtr overflow_row) const
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
ConstColumnPlainPtrs key_columns(params.keys_size);
|
|
|
|
|
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Запоминаем столбцы, с которыми будем работать
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = block.getByPosition(i).column.get();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
aggregate_columns[i] = &typeid_cast<ColumnAggregateFunction &>(*block.getByPosition(params.keys_size + i).column).getData();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-01-03 06:49:32 +00:00
|
|
|
|
typename Method::State state;
|
|
|
|
|
state.init(key_columns);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Для всех строчек.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
StringRefs keys(params.keys_size);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
size_t rows = block.rowsInFirstColumn();
|
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
|
{
|
|
|
|
|
typename Table::iterator it;
|
2015-10-23 01:43:42 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
2015-10-23 01:43:42 +00:00
|
|
|
|
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Получаем ключ для вставки в хэш-таблицу.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
auto key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
if (!no_more_keys)
|
|
|
|
|
{
|
|
|
|
|
data.emplace(key, it, inserted);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
inserted = false;
|
|
|
|
|
it = data.find(key);
|
|
|
|
|
if (data.end() == it)
|
|
|
|
|
overflow = true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
|
|
|
|
if (no_more_keys && overflow && !overflow_row)
|
|
|
|
|
{
|
|
|
|
|
method.onExistingKey(key, keys, *aggregates_pool);
|
|
|
|
|
continue;
|
|
|
|
|
}
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (inserted)
|
|
|
|
|
{
|
|
|
|
|
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
2015-03-07 01:05:06 +00:00
|
|
|
|
aggregate_data = nullptr;
|
2015-03-23 01:10:06 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
|
2015-03-23 01:10:06 +00:00
|
|
|
|
|
2015-03-07 01:05:06 +00:00
|
|
|
|
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
|
|
|
|
|
createAggregateStates(place);
|
|
|
|
|
aggregate_data = place;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
2015-02-22 16:09:16 +00:00
|
|
|
|
else
|
|
|
|
|
method.onExistingKey(key, keys, *aggregates_pool);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
/// Мерджим состояния агрегатных функций.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t j = 0; j < params.aggregates_size; ++j)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[j]->merge(
|
2015-10-23 01:43:42 +00:00
|
|
|
|
value + offsets_of_aggregate_states[j],
|
2015-01-03 05:39:21 +00:00
|
|
|
|
(*aggregate_columns[j])[i]);
|
|
|
|
|
}
|
2015-01-27 23:02:05 +00:00
|
|
|
|
|
|
|
|
|
/// Пораньше освобождаем память.
|
|
|
|
|
block.clear();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
template <typename Method, typename Table>
|
|
|
|
|
void NO_INLINE Aggregator::mergeStreamsImpl(
|
|
|
|
|
Block & block,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
Arena * aggregates_pool,
|
|
|
|
|
Method & method,
|
|
|
|
|
Table & data,
|
|
|
|
|
AggregateDataPtr overflow_row,
|
|
|
|
|
bool no_more_keys) const
|
|
|
|
|
{
|
|
|
|
|
if (!no_more_keys)
|
|
|
|
|
mergeStreamsImplCase<false>(block, key_sizes, aggregates_pool, method, data, overflow_row);
|
|
|
|
|
else
|
|
|
|
|
mergeStreamsImplCase<true>(block, key_sizes, aggregates_pool, method, data, overflow_row);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
|
|
|
|
Block & block,
|
|
|
|
|
AggregatedDataVariants & result) const
|
|
|
|
|
{
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Запоминаем столбцы, с которыми будем работать
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
aggregate_columns[i] = &typeid_cast<ColumnAggregateFunction &>(*block.getByPosition(params.keys_size + i).column).getData();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
AggregatedDataWithoutKey & res = result.without_key;
|
|
|
|
|
if (!res)
|
|
|
|
|
{
|
2015-03-07 01:05:06 +00:00
|
|
|
|
AggregateDataPtr place = result.aggregates_pool->alloc(total_size_of_aggregate_states);
|
|
|
|
|
createAggregateStates(place);
|
|
|
|
|
res = place;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавляем значения
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns[i])[0]);
|
2015-01-27 23:02:05 +00:00
|
|
|
|
|
|
|
|
|
/// Пораньше освобождаем память.
|
|
|
|
|
block.clear();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads)
|
2012-05-30 01:38:02 +00:00
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
StringRefs key(params.keys_size);
|
|
|
|
|
ConstColumnPlainPtrs key_columns(params.keys_size);
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-12-09 02:56:18 +00:00
|
|
|
|
initialize({});
|
2013-02-13 19:24:19 +00:00
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-01-03 03:18:49 +00:00
|
|
|
|
/** Если на удалённых серверах использовался двухуровневый метод агрегации,
|
|
|
|
|
* то в блоках будет расположена информация о номере корзины.
|
|
|
|
|
* Тогда вычисления можно будет распараллелить по корзинам.
|
|
|
|
|
* Разложим блоки по указанным в них номерам корзин.
|
|
|
|
|
*/
|
|
|
|
|
using BucketToBlocks = std::map<Int32, BlocksList>;
|
|
|
|
|
BucketToBlocks bucket_to_blocks;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
/// Читаем все данные.
|
2015-01-03 05:39:21 +00:00
|
|
|
|
LOG_TRACE(log, "Reading blocks of partially aggregated data.");
|
|
|
|
|
|
|
|
|
|
size_t total_input_rows = 0;
|
|
|
|
|
size_t total_input_blocks = 0;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
while (Block block = stream->read())
|
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
total_input_rows += block.rowsInFirstColumn();
|
|
|
|
|
++total_input_blocks;
|
|
|
|
|
bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Read " << total_input_blocks << " blocks of partially aggregated data, total " << total_input_rows << " rows.");
|
|
|
|
|
|
|
|
|
|
if (bucket_to_blocks.empty())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-12-09 02:56:18 +00:00
|
|
|
|
setSampleBlock(bucket_to_blocks.begin()->second.front());
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/// Каким способом выполнять агрегацию?
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = sample.getByPosition(i).column.get();
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
Sizes key_sizes;
|
|
|
|
|
AggregatedDataVariants::Type method = chooseAggregationMethod(key_columns, key_sizes);
|
|
|
|
|
|
|
|
|
|
/** Минус единицей обозначается отсутствие информации о корзине
|
|
|
|
|
* - в случае одноуровневой агрегации, а также для блоков с "переполнившимися" значениями.
|
|
|
|
|
* Если есть хотя бы один блок с номером корзины больше нуля, значит была двухуровневая агрегация.
|
|
|
|
|
*/
|
2015-03-24 00:39:30 +00:00
|
|
|
|
auto max_bucket = bucket_to_blocks.rbegin()->first;
|
2015-03-19 05:15:38 +00:00
|
|
|
|
size_t has_two_level = max_bucket > 0;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
if (has_two_level)
|
|
|
|
|
{
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
if (method == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
method = AggregatedDataVariants::Type::NAME ## _two_level;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
#undef M
|
|
|
|
|
}
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-03-23 01:10:06 +00:00
|
|
|
|
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
|
|
|
|
result.aggregator = this;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
result.init(method);
|
2015-11-30 16:57:05 +00:00
|
|
|
|
result.keys_size = params.keys_size;
|
2015-01-03 05:39:21 +00:00
|
|
|
|
result.key_sizes = key_sizes;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-01-17 04:49:13 +00:00
|
|
|
|
bool has_blocks_with_unknown_bucket = bucket_to_blocks.count(-1);
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
/// Сначала параллельно мерджим для отдельных bucket-ов. Затем домердживаем данные, не распределённые по bucket-ам.
|
|
|
|
|
if (has_two_level)
|
|
|
|
|
{
|
2015-10-23 01:43:42 +00:00
|
|
|
|
/** В этом случае, no_more_keys не поддерживается в связи с тем, что
|
|
|
|
|
* из разных потоков трудно обновлять общее состояние для "остальных" ключей (overflows).
|
|
|
|
|
* То есть, ключей в итоге может оказаться существенно больше, чем max_rows_to_group_by.
|
|
|
|
|
*/
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
LOG_TRACE(log, "Merging partially aggregated two-level data.");
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-03-23 01:10:06 +00:00
|
|
|
|
auto merge_bucket = [&bucket_to_blocks, &result, &key_sizes, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker)
|
2013-12-16 02:32:00 +00:00
|
|
|
|
{
|
2015-01-03 05:39:21 +00:00
|
|
|
|
current_memory_tracker = memory_tracker;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
for (Block & block : bucket_to_blocks[bucket])
|
2012-05-30 01:38:02 +00:00
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
return;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
2015-10-23 01:43:42 +00:00
|
|
|
|
mergeStreamsImpl(block, key_sizes, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
2015-01-03 06:49:32 +00:00
|
|
|
|
else
|
2015-01-03 05:39:21 +00:00
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
2012-05-30 01:38:02 +00:00
|
|
|
|
}
|
2015-01-03 05:39:21 +00:00
|
|
|
|
};
|
2012-05-30 03:30:29 +00:00
|
|
|
|
|
2016-08-02 01:46:05 +00:00
|
|
|
|
std::unique_ptr<ThreadPool> thread_pool;
|
2015-03-16 19:24:57 +00:00
|
|
|
|
if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог.
|
|
|
|
|
&& has_two_level)
|
2016-08-02 01:46:05 +00:00
|
|
|
|
thread_pool.reset(new ThreadPool(max_threads));
|
2015-03-16 19:24:57 +00:00
|
|
|
|
|
2015-03-23 01:10:06 +00:00
|
|
|
|
for (const auto & bucket_blocks : bucket_to_blocks)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-03-23 01:10:06 +00:00
|
|
|
|
const auto bucket = bucket_blocks.first;
|
2015-01-17 04:49:13 +00:00
|
|
|
|
|
|
|
|
|
if (bucket == -1)
|
|
|
|
|
continue;
|
|
|
|
|
|
2016-05-28 10:15:36 +00:00
|
|
|
|
result.aggregates_pools.push_back(std::make_shared<Arena>());
|
2015-01-03 06:49:32 +00:00
|
|
|
|
Arena * aggregates_pool = result.aggregates_pools.back().get();
|
|
|
|
|
|
2016-08-02 01:46:05 +00:00
|
|
|
|
auto task = std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
if (thread_pool)
|
2016-08-02 01:46:05 +00:00
|
|
|
|
thread_pool->schedule(task);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
else
|
2016-08-02 01:46:05 +00:00
|
|
|
|
task();
|
2012-05-30 01:38:02 +00:00
|
|
|
|
}
|
2013-05-06 11:45:28 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (thread_pool)
|
|
|
|
|
thread_pool->wait();
|
2014-12-30 10:16:23 +00:00
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
LOG_TRACE(log, "Merged partially aggregated two-level data.");
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
{
|
|
|
|
|
result.invalidate();
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2015-01-17 04:49:13 +00:00
|
|
|
|
if (has_blocks_with_unknown_bucket)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_TRACE(log, "Merging partially aggregated single-level data.");
|
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
bool no_more_keys = false;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
BlocksList & blocks = bucket_to_blocks[-1];
|
|
|
|
|
for (Block & block : blocks)
|
|
|
|
|
{
|
2015-04-16 14:27:56 +00:00
|
|
|
|
if (isCancelled())
|
|
|
|
|
{
|
|
|
|
|
result.invalidate();
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2015-10-23 01:43:42 +00:00
|
|
|
|
if (!checkLimits(result.sizeWithoutOverflowRow(), no_more_keys))
|
|
|
|
|
break;
|
|
|
|
|
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
|
|
|
|
|
mergeWithoutKeyStreamsImpl(block, result);
|
|
|
|
|
|
|
|
|
|
#define M(NAME, IS_TWO_LEVEL) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
2015-10-23 01:43:42 +00:00
|
|
|
|
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else if (result.type != AggregatedDataVariants::Type::without_key)
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Merged partially aggregated single-level data.");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-07-30 23:41:02 +00:00
|
|
|
|
Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
|
|
|
|
{
|
|
|
|
|
if (blocks.empty())
|
|
|
|
|
return {};
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
StringRefs key(params.keys_size);
|
|
|
|
|
ConstColumnPlainPtrs key_columns(params.keys_size);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-12-09 02:56:18 +00:00
|
|
|
|
initialize({});
|
|
|
|
|
setSampleBlock(blocks.front());
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
/// Каким способом выполнять агрегацию?
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = sample.getByPosition(i).column.get();
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
Sizes key_sizes;
|
|
|
|
|
AggregatedDataVariants::Type method = chooseAggregationMethod(key_columns, key_sizes);
|
|
|
|
|
|
|
|
|
|
/// Временные данные для агрегации.
|
|
|
|
|
AggregatedDataVariants result;
|
|
|
|
|
|
|
|
|
|
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
|
|
|
|
result.aggregator = this;
|
|
|
|
|
|
|
|
|
|
result.init(method);
|
2015-11-30 16:57:05 +00:00
|
|
|
|
result.keys_size = params.keys_size;
|
2015-07-30 23:41:02 +00:00
|
|
|
|
result.key_sizes = key_sizes;
|
|
|
|
|
|
2015-12-11 00:34:00 +00:00
|
|
|
|
auto bucket_num = blocks.front().info.bucket_num;
|
|
|
|
|
LOG_TRACE(log, "Merging partially aggregated blocks (bucket = " << bucket_num << ").");
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
for (Block & block : blocks)
|
|
|
|
|
{
|
|
|
|
|
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
|
|
|
|
|
mergeWithoutKeyStreamsImpl(block, result);
|
|
|
|
|
|
|
|
|
|
#define M(NAME, IS_TWO_LEVEL) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
2015-10-23 01:43:42 +00:00
|
|
|
|
mergeStreamsImpl(block, key_sizes, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else if (result.type != AggregatedDataVariants::Type::without_key)
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-07 20:08:02 +00:00
|
|
|
|
BlocksList merged_blocks = convertToBlocks(result, final, 1);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2015-09-07 20:08:02 +00:00
|
|
|
|
if (merged_blocks.size() > 1)
|
|
|
|
|
{
|
|
|
|
|
/** Может быть два блока. Один с is_overflows, другой - нет.
|
|
|
|
|
* Если есть непустой блок не is_overflows, то удаляем блок с is_overflows.
|
|
|
|
|
* Если есть пустой блок не is_overflows и блок с is_overflows, то удаляем пустой блок.
|
|
|
|
|
*
|
|
|
|
|
* Это делаем, потому что исходим из допущения, что в функцию передаются
|
|
|
|
|
* либо все блоки не is_overflows, либо все блоки is_overflows.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
bool has_nonempty_nonoverflows = false;
|
|
|
|
|
bool has_overflows = false;
|
|
|
|
|
|
|
|
|
|
for (const auto & block : merged_blocks)
|
|
|
|
|
{
|
2015-09-14 21:09:33 +00:00
|
|
|
|
if (block && block.rowsInFirstColumn() && !block.info.is_overflows)
|
2015-09-07 20:08:02 +00:00
|
|
|
|
has_nonempty_nonoverflows = true;
|
|
|
|
|
else if (block.info.is_overflows)
|
|
|
|
|
has_overflows = true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (has_nonempty_nonoverflows)
|
|
|
|
|
{
|
|
|
|
|
for (auto it = merged_blocks.begin(); it != merged_blocks.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
if (it->info.is_overflows)
|
|
|
|
|
{
|
|
|
|
|
merged_blocks.erase(it);
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else if (has_overflows)
|
|
|
|
|
{
|
|
|
|
|
for (auto it = merged_blocks.begin(); it != merged_blocks.end(); ++it)
|
|
|
|
|
{
|
2015-09-14 21:09:33 +00:00
|
|
|
|
if (!*it || it->rowsInFirstColumn() == 0)
|
2015-09-07 20:08:02 +00:00
|
|
|
|
{
|
|
|
|
|
merged_blocks.erase(it);
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (merged_blocks.size() > 1)
|
|
|
|
|
throw Exception("Logical error: temporary result is not single-level", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
}
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Merged partially aggregated blocks.");
|
|
|
|
|
|
2015-09-07 20:08:02 +00:00
|
|
|
|
if (merged_blocks.empty())
|
2015-07-30 23:41:02 +00:00
|
|
|
|
return {};
|
|
|
|
|
|
2015-12-11 00:34:00 +00:00
|
|
|
|
auto res = std::move(merged_blocks.front());
|
|
|
|
|
res.info.bucket_num = bucket_num;
|
|
|
|
|
return res;
|
2015-07-30 23:41:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-09-07 07:40:14 +00:00
|
|
|
|
template <typename Method>
|
|
|
|
|
void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
|
|
|
|
|
Method & method,
|
|
|
|
|
Arena * pool,
|
|
|
|
|
ConstColumnPlainPtrs & key_columns,
|
|
|
|
|
const Sizes & key_sizes,
|
|
|
|
|
StringRefs & keys,
|
|
|
|
|
const Block & source,
|
|
|
|
|
std::vector<Block> & destinations) const
|
|
|
|
|
{
|
|
|
|
|
typename Method::State state;
|
|
|
|
|
state.init(key_columns);
|
|
|
|
|
|
|
|
|
|
size_t rows = source.rowsInFirstColumn();
|
|
|
|
|
size_t columns = source.columns();
|
|
|
|
|
|
2015-09-10 01:58:37 +00:00
|
|
|
|
/// Для каждого номера корзины создадим фильтр, где будут отмечены строки, относящиеся к этой корзине.
|
|
|
|
|
std::vector<IColumn::Filter> filters(destinations.size());
|
|
|
|
|
|
2015-09-07 07:40:14 +00:00
|
|
|
|
/// Для всех строчек.
|
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
|
{
|
|
|
|
|
/// Получаем ключ. Вычисляем на его основе номер корзины.
|
2015-11-30 16:57:05 +00:00
|
|
|
|
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *pool);
|
2015-09-07 07:40:14 +00:00
|
|
|
|
|
|
|
|
|
auto hash = method.data.hash(key);
|
|
|
|
|
auto bucket = method.data.getBucketFromHash(hash);
|
|
|
|
|
|
|
|
|
|
/// Этот ключ нам больше не нужен.
|
|
|
|
|
method.onExistingKey(key, keys, *pool);
|
|
|
|
|
|
2015-09-10 01:58:37 +00:00
|
|
|
|
auto & filter = filters[bucket];
|
|
|
|
|
|
|
|
|
|
if (unlikely(filter.empty()))
|
|
|
|
|
filter.resize_fill(rows);
|
|
|
|
|
|
|
|
|
|
filter[i] = 1;
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-05 07:01:18 +00:00
|
|
|
|
ssize_t size_hint = ((source.rowsInFirstColumn() + method.data.NUM_BUCKETS - 1)
|
|
|
|
|
/ method.data.NUM_BUCKETS) * 1.1; /// Число 1.1 выбрано наугад.
|
|
|
|
|
|
2015-09-10 01:58:37 +00:00
|
|
|
|
for (size_t bucket = 0, size = destinations.size(); bucket < size; ++bucket)
|
|
|
|
|
{
|
|
|
|
|
const auto & filter = filters[bucket];
|
|
|
|
|
|
|
|
|
|
if (filter.empty())
|
|
|
|
|
continue;
|
|
|
|
|
|
2015-09-07 07:40:14 +00:00
|
|
|
|
Block & dst = destinations[bucket];
|
2015-09-10 01:58:37 +00:00
|
|
|
|
dst.info.bucket_num = bucket;
|
2015-09-07 07:40:14 +00:00
|
|
|
|
|
|
|
|
|
for (size_t j = 0; j < columns; ++j)
|
2015-09-10 01:58:37 +00:00
|
|
|
|
{
|
|
|
|
|
const ColumnWithTypeAndName & src_col = source.unsafeGetByPosition(j);
|
2015-12-05 07:01:18 +00:00
|
|
|
|
dst.insert({src_col.column->filter(filter, size_hint), src_col.type, src_col.name});
|
2015-09-10 01:58:37 +00:00
|
|
|
|
|
|
|
|
|
/** Вставленные в блок столбцы типа ColumnAggregateFunction будут владеть состояниями агрегатных функций
|
2016-05-28 17:31:50 +00:00
|
|
|
|
* путём удержания shared_ptr-а на исходный столбец. См. ColumnAggregateFunction.h
|
2015-09-10 01:58:37 +00:00
|
|
|
|
*/
|
|
|
|
|
}
|
2015-09-07 07:40:14 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
|
|
|
|
|
{
|
|
|
|
|
if (!block)
|
|
|
|
|
return {};
|
|
|
|
|
|
2015-12-09 02:56:18 +00:00
|
|
|
|
initialize({});
|
|
|
|
|
setSampleBlock(block);
|
2015-09-08 20:19:30 +00:00
|
|
|
|
|
2015-09-07 07:40:14 +00:00
|
|
|
|
AggregatedDataVariants data;
|
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
StringRefs key(params.keys_size);
|
|
|
|
|
ConstColumnPlainPtrs key_columns(params.keys_size);
|
2015-09-07 07:40:14 +00:00
|
|
|
|
Sizes key_sizes;
|
|
|
|
|
|
|
|
|
|
/// Запоминаем столбцы, с которыми будем работать
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys_size; ++i)
|
2016-05-28 05:31:36 +00:00
|
|
|
|
key_columns[i] = block.getByPosition(i).column.get();
|
2015-09-07 07:40:14 +00:00
|
|
|
|
|
|
|
|
|
AggregatedDataVariants::Type type = chooseAggregationMethod(key_columns, key_sizes);
|
2015-11-30 16:57:05 +00:00
|
|
|
|
data.keys_size = params.keys_size;
|
2015-09-09 21:32:29 +00:00
|
|
|
|
data.key_sizes = key_sizes;
|
2015-09-07 07:40:14 +00:00
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
type = AggregatedDataVariants::Type::NAME ## _two_level;
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
|
|
|
|
data.init(type);
|
|
|
|
|
|
|
|
|
|
size_t num_buckets = 0;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (data.type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
num_buckets = data.NAME->data.NUM_BUCKETS;
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
|
|
|
|
std::vector<Block> splitted_blocks(num_buckets);
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
else if (data.type == AggregatedDataVariants::Type::NAME) \
|
|
|
|
|
convertBlockToTwoLevelImpl(*data.NAME, data.aggregates_pool, \
|
|
|
|
|
key_columns, data.key_sizes, key, block, splitted_blocks);
|
|
|
|
|
|
|
|
|
|
if (false) {}
|
|
|
|
|
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
|
|
|
|
#undef M
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
|
|
|
|
|
|
|
|
|
return splitted_blocks;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-12-06 19:42:28 +00:00
|
|
|
|
template <typename Method, typename Table>
|
2015-01-03 05:39:21 +00:00
|
|
|
|
void NO_INLINE Aggregator::destroyImpl(
|
2015-12-06 19:42:28 +00:00
|
|
|
|
Method & method,
|
2015-12-23 07:03:50 +00:00
|
|
|
|
Table & table) const
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-12-23 07:03:50 +00:00
|
|
|
|
for (auto elem : table)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
{
|
2015-12-23 05:06:32 +00:00
|
|
|
|
AggregateDataPtr & data = Method::getAggregateData(elem.second);
|
2015-01-03 05:39:21 +00:00
|
|
|
|
|
|
|
|
|
/** Если исключение (обычно нехватка памяти, кидается MemoryTracker-ом) возникло
|
|
|
|
|
* после вставки ключа в хэш-таблицу, но до создания всех состояний агрегатных функций,
|
|
|
|
|
* то data будет равен nullptr-у.
|
|
|
|
|
*/
|
|
|
|
|
if (nullptr == data)
|
|
|
|
|
continue;
|
2012-05-31 00:33:42 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
2015-01-03 05:39:21 +00:00
|
|
|
|
if (!aggregate_functions[i]->isState())
|
|
|
|
|
aggregate_functions[i]->destroy(data + offsets_of_aggregate_states[i]);
|
2015-12-06 19:42:28 +00:00
|
|
|
|
|
|
|
|
|
data = nullptr;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void Aggregator::destroyWithoutKey(AggregatedDataVariants & result) const
|
|
|
|
|
{
|
|
|
|
|
AggregatedDataWithoutKey & res_data = result.without_key;
|
|
|
|
|
|
|
|
|
|
if (nullptr != res_data)
|
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
if (!aggregate_functions[i]->isState())
|
|
|
|
|
aggregate_functions[i]->destroy(res_data + offsets_of_aggregate_states[i]);
|
|
|
|
|
|
|
|
|
|
res_data = nullptr;
|
2012-05-30 01:38:02 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-05-19 19:41:56 +00:00
|
|
|
|
void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result)
|
2013-02-16 18:59:05 +00:00
|
|
|
|
{
|
|
|
|
|
if (result.size() == 0)
|
|
|
|
|
return;
|
|
|
|
|
|
2013-11-03 23:35:18 +00:00
|
|
|
|
LOG_TRACE(log, "Destroying aggregate states");
|
2013-02-16 18:59:05 +00:00
|
|
|
|
|
|
|
|
|
/// В какой структуре данных агрегированы данные?
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
2015-12-06 19:42:28 +00:00
|
|
|
|
destroyWithoutKey(result);
|
2013-02-16 18:59:05 +00:00
|
|
|
|
|
2014-12-30 10:16:23 +00:00
|
|
|
|
#define M(NAME, IS_TWO_LEVEL) \
|
|
|
|
|
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
2015-12-06 19:42:28 +00:00
|
|
|
|
destroyImpl(*result.NAME, result.NAME->data);
|
2014-12-30 10:16:23 +00:00
|
|
|
|
|
|
|
|
|
if (false) {}
|
2015-01-02 03:16:28 +00:00
|
|
|
|
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
2014-12-30 10:16:23 +00:00
|
|
|
|
#undef M
|
|
|
|
|
else if (result.type != AggregatedDataVariants::Type::without_key)
|
2014-05-10 00:31:22 +00:00
|
|
|
|
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
2013-02-16 18:59:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-05-03 10:20:53 +00:00
|
|
|
|
|
|
|
|
|
String Aggregator::getID() const
|
|
|
|
|
{
|
|
|
|
|
std::stringstream res;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2015-11-30 16:57:05 +00:00
|
|
|
|
if (params.keys.empty())
|
2013-05-03 10:20:53 +00:00
|
|
|
|
{
|
|
|
|
|
res << "key_names";
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.key_names.size(); ++i)
|
|
|
|
|
res << ", " << params.key_names[i];
|
2013-05-03 10:20:53 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
res << "keys";
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.keys.size(); ++i)
|
|
|
|
|
res << ", " << params.keys[i];
|
2013-05-03 10:20:53 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
res << ", aggregates";
|
2015-11-30 16:57:05 +00:00
|
|
|
|
for (size_t i = 0; i < params.aggregates_size; ++i)
|
|
|
|
|
res << ", " << params.aggregates[i].column_name;
|
2013-05-03 10:20:53 +00:00
|
|
|
|
|
|
|
|
|
return res.str();
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-16 14:27:56 +00:00
|
|
|
|
void Aggregator::setCancellationHook(const CancellationHook cancellation_hook)
|
|
|
|
|
{
|
|
|
|
|
isCancelled = cancellation_hook;
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
2011-09-19 01:42:16 +00:00
|
|
|
|
}
|