ClickHouse/dbms/src/Interpreters/Aggregator.cpp

802 lines
26 KiB
C++
Raw Normal View History

2011-09-26 07:25:22 +00:00
#include <openssl/md5.h>
2011-09-19 03:34:23 +00:00
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/Columns/ColumnAggregateFunction.h>
2011-09-26 07:25:22 +00:00
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnsNumber.h>
2012-07-15 23:13:08 +00:00
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
2011-09-19 03:34:23 +00:00
#include <DB/Interpreters/Aggregator.h>
2011-09-19 01:42:16 +00:00
namespace DB
{
2011-09-26 07:25:22 +00:00
2012-03-05 07:58:34 +00:00
void Aggregator::initialize(Block & block)
{
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
if (initialized)
return;
initialized = true;
/// Преобразуем имена столбцов в номера, если номера не заданы
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));
/// Создадим пример блока, описывающего результат
if (!sample)
{
for (size_t i = 0, size = keys.size(); i < size; ++i)
2012-05-31 01:13:15 +00:00
{
2012-03-05 07:58:34 +00:00
sample.insert(block.getByPosition(keys[i]).cloneEmpty());
2012-05-31 01:13:15 +00:00
if (sample.getByPosition(i).column->isConst())
sample.getByPosition(i).column = dynamic_cast<IColumnConst &>(*sample.getByPosition(i).column).convertToFullColumn();
}
2012-03-05 07:58:34 +00:00
for (size_t i = 0, size = aggregates.size(); i < size; ++i)
{
ColumnWithNameAndType col;
col.name = aggregates[i].column_name;
col.type = new DataTypeAggregateFunction;
col.column = new ColumnAggregateFunction;
sample.insert(col);
}
/// Вставим в блок результата все столбцы-константы из исходного блока, так как они могут ещё пригодиться.
size_t columns = block.columns();
for (size_t i = 0; i < columns; ++i)
if (block.getByPosition(i).column->isConst())
sample.insert(block.getByPosition(i).cloneEmpty());
}
}
2012-05-30 01:38:02 +00:00
AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(Columns & key_columns, bool & keys_fit_128_bits, Sizes & key_sizes)
{
/* bool has_strings = false;
for (size_t j = 0; j < keys_size; ++j)
if (dynamic_cast<const ColumnString *>(&*key_columns[j]) || dynamic_cast<const ColumnFixedString *>(&*key_columns[j]))
has_strings = true;*/
keys_fit_128_bits = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isNumeric())
{
keys_fit_128_bits = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfField();
keys_bytes += key_sizes[j];
}
if (keys_bytes > 16)
keys_fit_128_bits = false;
/// Если ключей нет
if (keys_size == 0)
return AggregatedDataVariants::WITHOUT_KEY;
/// Если есть один ключ, который помещается в 64 бита, и это не число с плавающей запятой
if (keys_size == 1 && key_columns[0]->isNumeric()
&& !dynamic_cast<ColumnFloat32 *>(&*key_columns[0]) && !dynamic_cast<ColumnFloat64 *>(&*key_columns[0]))
return AggregatedDataVariants::KEY_64;
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1
&& (dynamic_cast<ColumnString *>(&*key_columns[0]) || dynamic_cast<ColumnFixedString *>(&*key_columns[0])))
return AggregatedDataVariants::KEY_STRING;
/// Если много ключей - будем агрегировать по хэшу от них
return AggregatedDataVariants::HASHED;
}
2011-09-28 05:24:38 +00:00
/** Результат хранится в оперативке и должен полностью помещаться в оперативку.
2011-09-19 01:42:16 +00:00
*/
2011-09-26 07:25:22 +00:00
void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & result)
2011-09-19 01:42:16 +00:00
{
size_t aggregates_size = aggregates.size();
Row key(keys_size);
Columns key_columns(keys_size);
typedef std::vector<Columns> AggregateColumns;
AggregateColumns aggregate_columns(aggregates_size);
typedef std::vector<Row> Rows;
Rows aggregate_arguments(aggregates_size);
/// Читаем все данные
while (Block block = stream->read())
{
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Aggregating block");
2012-03-05 07:58:34 +00:00
initialize(block);
2011-09-25 05:07:47 +00:00
for (size_t i = 0; i < aggregates_size; ++i)
{
aggregate_arguments[i].resize(aggregates[i].arguments.size());
aggregate_columns[i].resize(aggregates[i].arguments.size());
}
2011-09-24 20:32:41 +00:00
2011-09-19 01:42:16 +00:00
/// Запоминаем столбцы, с которыми будем работать
2012-05-30 01:38:02 +00:00
for (size_t i = 0; i < keys_size; ++i)
2011-09-19 01:42:16 +00:00
key_columns[i] = block.getByPosition(keys[i]).column;
for (size_t i = 0; i < aggregates_size; ++i)
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
aggregate_columns[i][j] = block.getByPosition(aggregates[i].arguments[j]).column;
size_t rows = block.rows();
2011-09-26 07:25:22 +00:00
/// Каким способом выполнять агрегацию?
2012-05-30 01:38:02 +00:00
bool keys_fit_128_bits = false;
Sizes key_sizes;
result.type = chooseAggregationMethod(key_columns, keys_fit_128_bits, key_sizes);
2011-09-26 12:50:50 +00:00
2012-05-30 01:38:02 +00:00
if (result.type == AggregatedDataVariants::WITHOUT_KEY)
2011-09-19 01:42:16 +00:00
{
2011-09-26 07:25:22 +00:00
AggregatedDataWithoutKey & res = result.without_key;
2011-09-26 11:58:35 +00:00
if (res.empty())
{
res.resize(aggregates_size);
for (size_t i = 0; i < aggregates_size; ++i)
res[i] = aggregates[i].function->cloneEmpty();
}
2012-07-15 23:13:08 +00:00
/// Оптимизация в случае единственной агрегатной функции count.
AggregateFunctionCount * agg_count = dynamic_cast<AggregateFunctionCount *>(res[0]);
if (aggregates_size == 1 && agg_count)
agg_count->addDelta(rows);
else
2011-09-26 07:25:22 +00:00
{
2012-07-15 23:13:08 +00:00
for (size_t i = 0; i < rows; ++i)
2011-09-26 07:25:22 +00:00
{
2012-07-15 23:13:08 +00:00
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
res[j]->add(aggregate_arguments[j]);
}
2011-09-26 07:25:22 +00:00
}
}
}
2012-05-30 01:38:02 +00:00
else if (result.type == AggregatedDataVariants::KEY_64)
2011-09-26 07:25:22 +00:00
{
AggregatedDataWithUInt64Key & res = result.key64;
2011-09-26 12:50:50 +00:00
const FieldVisitorToUInt64 visitor;
2011-09-26 07:25:22 +00:00
IColumn & column = *key_columns[0];
2011-09-19 01:42:16 +00:00
2011-09-26 07:25:22 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2011-09-19 01:42:16 +00:00
{
2011-09-26 07:25:22 +00:00
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
2011-12-19 02:00:40 +00:00
AggregatedDataWithUInt64Key::iterator it;
bool inserted;
res.emplace(key, it, inserted);
if (inserted)
2011-09-26 07:25:22 +00:00
{
2011-12-19 08:06:31 +00:00
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
2011-12-19 02:00:40 +00:00
2011-09-26 07:25:22 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
2011-09-19 01:42:16 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
2011-09-26 07:25:22 +00:00
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
it->second[j]->add(aggregate_arguments[j]);
}
2011-09-19 01:42:16 +00:00
}
2011-09-26 07:25:22 +00:00
}
2012-05-30 01:38:02 +00:00
else if (result.type == AggregatedDataVariants::KEY_STRING)
2011-09-26 15:22:25 +00:00
{
AggregatedDataWithStringKey & res = result.key_string;
IColumn & column = *key_columns[0];
2012-05-31 05:41:56 +00:00
if (const ColumnString * column_string = dynamic_cast<const ColumnString *>(&column))
2011-09-26 15:22:25 +00:00
{
2012-05-31 05:41:56 +00:00
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnUInt8::Container_t & data = dynamic_cast<const ColumnUInt8 &>(column_string->getData()).getData();
2011-09-26 15:22:25 +00:00
2012-05-31 05:41:56 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2011-09-26 15:22:25 +00:00
{
2012-05-31 05:41:56 +00:00
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
2011-09-26 15:22:25 +00:00
2012-05-31 05:41:56 +00:00
AggregatedDataWithStringKey::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
{
it->first.data = result.string_pool.insert(ref.data, ref.size);
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
2011-09-26 15:22:25 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
2012-05-31 05:41:56 +00:00
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
it->second[j]->add(aggregate_arguments[j]);
}
2011-09-26 15:22:25 +00:00
}
2012-05-31 05:41:56 +00:00
}
else if (const ColumnFixedString * column_string = dynamic_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnUInt8::Container_t & data = dynamic_cast<const ColumnUInt8 &>(column_string->getData()).getData();
2011-09-26 15:22:25 +00:00
2012-05-31 05:41:56 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2011-09-26 15:22:25 +00:00
{
2012-05-31 05:41:56 +00:00
/// Строим ключ
StringRef ref(&data[i * n], n);
2011-09-26 15:22:25 +00:00
2012-05-31 05:41:56 +00:00
AggregatedDataWithStringKey::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
{
it->first.data = result.string_pool.insert(ref.data, ref.size);
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
it->second[j]->add(aggregate_arguments[j]);
}
2011-09-26 15:22:25 +00:00
}
}
2012-05-31 05:41:56 +00:00
else
throw Exception("Illegal type of column when aggregating by string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
2011-09-26 15:22:25 +00:00
}
2012-05-30 01:38:02 +00:00
else if (result.type == AggregatedDataVariants::HASHED)
2011-09-26 07:25:22 +00:00
{
AggregatedDataHashed & res = result.hashed;
2011-09-26 12:50:50 +00:00
const FieldVisitorToUInt64 to_uint64_visitor;
2011-09-19 01:42:16 +00:00
2011-09-26 07:25:22 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2011-09-19 01:42:16 +00:00
{
2011-12-19 02:00:40 +00:00
AggregatedDataHashed::iterator it;
bool inserted;
2012-08-23 23:49:28 +00:00
res.emplace(pack128(i, keys_fit_128_bits, keys_size, key, key_columns, key_sizes), it, inserted);
2011-12-19 02:00:40 +00:00
if (inserted)
2011-09-26 07:25:22 +00:00
{
2011-12-19 08:06:31 +00:00
new(&it->second) AggregatedDataHashed::mapped_type(key, AggregateFunctionsPlainPtrs(aggregates_size));
2011-09-19 01:42:16 +00:00
2011-09-26 07:25:22 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
it->second.second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
it->second.second[j]->add(aggregate_arguments[j]);
}
2011-09-19 01:42:16 +00:00
}
}
2012-05-30 01:38:02 +00:00
else if (result.type == AggregatedDataVariants::GENERIC)
2011-09-26 07:25:22 +00:00
{
/// Общий способ
AggregatedData & res = result.generic;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
for (size_t j = 0; j < keys_size; ++j)
key[j] = (*key_columns[j])[i];
AggregatedData::iterator it = res.find(key);
if (it == res.end())
{
2011-12-19 08:06:31 +00:00
it = res.insert(std::make_pair(key, AggregateFunctionsPlainPtrs(aggregates_size))).first;
2011-09-26 07:25:22 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
2011-09-19 01:42:16 +00:00
2011-09-26 07:25:22 +00:00
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
{
for (size_t k = 0, size = aggregate_arguments[j].size(); k < size; ++k)
aggregate_arguments[j][k] = (*aggregate_columns[j][k])[i];
it->second[j]->add(aggregate_arguments[j]);
}
}
2012-05-30 01:38:02 +00:00
}
else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Aggregated block");
2011-09-26 07:25:22 +00:00
}
2011-09-19 01:42:16 +00:00
}
2012-02-27 06:28:20 +00:00
Block Aggregator::convertToBlock(AggregatedDataVariants & data_variants)
{
Block res = getSampleBlock();
size_t rows = 0;
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Converting aggregated data to block");
2012-02-27 06:28:20 +00:00
/// В какой структуре данных агрегированы данные?
2012-05-10 07:47:13 +00:00
if (data_variants.empty())
return res;
2012-02-27 06:28:20 +00:00
if (data_variants.type == AggregatedDataVariants::WITHOUT_KEY)
{
AggregatedDataWithoutKey & data = data_variants.without_key;
rows = 1;
size_t i = 0;
for (AggregateFunctionsPlainPtrs::const_iterator jt = data.begin(); jt != data.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
}
else if (data_variants.type == AggregatedDataVariants::KEY_64)
{
AggregatedDataWithUInt64Key & data = data_variants.key64;
rows = data.size();
IColumn & first_column = *res.getByPosition(0).column;
bool is_signed = dynamic_cast<ColumnInt8 *>(&first_column) || dynamic_cast<ColumnInt16 *>(&first_column)
|| dynamic_cast<ColumnInt32 *>(&first_column) || dynamic_cast<ColumnInt64 *>(&first_column);
for (AggregatedDataWithUInt64Key::const_iterator it = data.begin(); it != data.end(); ++it)
{
if (is_signed)
first_column.insert(static_cast<Int64>(it->first));
else
first_column.insert(it->first);
size_t i = 1;
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
}
}
else if (data_variants.type == AggregatedDataVariants::KEY_STRING)
{
AggregatedDataWithStringKey & data = data_variants.key_string;
rows = data.size();
IColumn & first_column = *res.getByPosition(0).column;
for (AggregatedDataWithStringKey::const_iterator it = data.begin(); it != data.end(); ++it)
{
2012-05-31 05:41:56 +00:00
first_column.insert(String(it->first.data, it->first.size));
2012-02-27 06:28:20 +00:00
size_t i = 1;
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
}
}
else if (data_variants.type == AggregatedDataVariants::HASHED)
{
AggregatedDataHashed & data = data_variants.hashed;
rows = data.size();
for (AggregatedDataHashed::const_iterator it = data.begin(); it != data.end(); ++it)
{
size_t i = 0;
for (Row::const_iterator jt = it->second.first.begin(); jt != it->second.first.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.second.begin(); jt != it->second.second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
}
}
else if (data_variants.type == AggregatedDataVariants::GENERIC)
{
AggregatedData & data = data_variants.generic;
rows = data.size();
for (AggregatedData::const_iterator it = data.begin(); it != data.end(); ++it)
{
size_t i = 0;
for (Row::const_iterator jt = it->first.begin(); jt != it->first.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
res.getByPosition(i).column->insert(*jt);
}
}
else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
/// Изменяем размер столбцов-констант в блоке.
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
if (res.getByPosition(i).column->isConst())
res.getByPosition(i).column->cut(0, rows);
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Converted aggregated data to block");
2012-02-27 06:28:20 +00:00
return res;
}
AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_variants)
{
if (data_variants.empty())
2012-02-27 07:54:16 +00:00
throw Exception("Empty data passed to Aggregator::merge().", ErrorCodes::EMPTY_DATA_PASSED);
2012-02-27 06:28:20 +00:00
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Merging aggregated data");
2012-08-21 18:34:55 +00:00
AggregatedDataVariantsPtr res = data_variants[0];
2012-02-27 06:28:20 +00:00
/// Все результаты агрегации соединяем с первым.
for (size_t i = 1, size = data_variants.size(); i < size; ++i)
{
AggregatedDataVariants & current = *data_variants[i];
2012-05-10 07:47:13 +00:00
if (current.empty())
continue;
2012-08-21 18:34:55 +00:00
if (res->empty())
2012-05-10 07:47:13 +00:00
{
2012-08-21 18:34:55 +00:00
res = data_variants[i];
2012-05-10 07:47:13 +00:00
continue;
}
2012-08-21 18:34:55 +00:00
if (res->type != current.type)
2012-02-27 06:28:20 +00:00
throw Exception("Cannot merge different aggregated data variants.", ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS);
/// В какой структуре данных агрегированы данные?
2012-08-21 18:34:55 +00:00
if (res->type == AggregatedDataVariants::WITHOUT_KEY)
2012-02-27 06:28:20 +00:00
{
2012-08-21 18:34:55 +00:00
AggregatedDataWithoutKey & res_data = res->without_key;
2012-02-27 06:28:20 +00:00
AggregatedDataWithoutKey & current_data = current.without_key;
size_t i = 0;
for (AggregateFunctionsPlainPtrs::const_iterator jt = current_data.begin(); jt != current_data.end(); ++jt, ++i)
{
res_data[i]->merge(**jt);
delete *jt;
}
}
2012-08-21 18:34:55 +00:00
else if (res->type == AggregatedDataVariants::KEY_64)
2012-02-27 06:28:20 +00:00
{
2012-08-21 18:34:55 +00:00
AggregatedDataWithUInt64Key & res_data = res->key64;
2012-02-27 06:28:20 +00:00
AggregatedDataWithUInt64Key & current_data = current.key64;
2012-02-27 16:23:28 +00:00
for (AggregatedDataWithUInt64Key::const_iterator it = current_data.begin(); it != current_data.end(); ++it)
2012-02-27 06:28:20 +00:00
{
AggregatedDataWithUInt64Key::iterator res_it;
bool inserted;
res_data.emplace(it->first, res_it, inserted);
if (!inserted)
{
2012-02-27 07:54:16 +00:00
size_t i = 0;
2012-02-27 06:28:20 +00:00
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
{
res_it->second[i]->merge(**jt);
delete *jt;
}
}
else
res_it->second = it->second;
}
}
2012-08-21 18:34:55 +00:00
else if (res->type == AggregatedDataVariants::KEY_STRING)
2012-02-27 06:28:20 +00:00
{
2012-08-21 18:34:55 +00:00
AggregatedDataWithStringKey & res_data = res->key_string;
2012-02-27 06:28:20 +00:00
AggregatedDataWithStringKey & current_data = current.key_string;
2012-02-27 16:23:28 +00:00
for (AggregatedDataWithStringKey::const_iterator it = current_data.begin(); it != current_data.end(); ++it)
2012-02-27 06:28:20 +00:00
{
2012-05-31 05:41:56 +00:00
AggregatedDataWithStringKey::iterator res_it;
bool inserted;
res_data.emplace(it->first, res_it, inserted);
if (!inserted)
2012-02-27 06:28:20 +00:00
{
2012-02-27 07:54:16 +00:00
size_t i = 0;
2012-02-27 06:28:20 +00:00
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
{
2012-05-31 05:41:56 +00:00
res_it->second[i]->merge(**jt);
2012-02-27 06:28:20 +00:00
delete *jt;
}
}
else
2012-05-31 05:41:56 +00:00
res_it->second = it->second;
2012-02-27 06:28:20 +00:00
}
}
2012-08-21 18:34:55 +00:00
else if (res->type == AggregatedDataVariants::HASHED)
2012-02-27 06:28:20 +00:00
{
2012-08-21 18:34:55 +00:00
AggregatedDataHashed & res_data = res->hashed;
2012-02-27 06:28:20 +00:00
AggregatedDataHashed & current_data = current.hashed;
2012-02-27 16:23:28 +00:00
for (AggregatedDataHashed::const_iterator it = current_data.begin(); it != current_data.end(); ++it)
2012-02-27 06:28:20 +00:00
{
AggregatedDataHashed::iterator res_it;
bool inserted;
res_data.emplace(it->first, res_it, inserted);
if (!inserted)
{
2012-02-27 07:54:16 +00:00
size_t i = 0;
2012-02-27 06:28:20 +00:00
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.second.begin(); jt != it->second.second.end(); ++jt, ++i)
{
res_it->second.second[i]->merge(**jt);
delete *jt;
}
}
else
res_it->second = it->second;
}
}
2012-08-21 18:34:55 +00:00
else if (res->type == AggregatedDataVariants::GENERIC)
2012-02-27 06:28:20 +00:00
{
2012-08-21 18:34:55 +00:00
AggregatedData & res_data = res->generic;
2012-02-27 06:28:20 +00:00
AggregatedData & current_data = current.generic;
2012-02-27 16:23:28 +00:00
for (AggregatedData::const_iterator it = current_data.begin(); it != current_data.end(); ++it)
2012-02-27 06:28:20 +00:00
{
AggregateFunctionsPlainPtrs & res_row = res_data[it->first];
if (!res_row.empty())
{
2012-02-27 07:54:16 +00:00
size_t i = 0;
2012-02-27 06:28:20 +00:00
for (AggregateFunctionsPlainPtrs::const_iterator jt = it->second.begin(); jt != it->second.end(); ++jt, ++i)
{
res_row[i]->merge(**jt);
delete *jt;
}
}
else
res_row = it->second;
}
}
else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Merged aggregated data");
2012-08-21 18:34:55 +00:00
return res;
2012-02-27 06:28:20 +00:00
}
2012-05-30 01:38:02 +00:00
void Aggregator::merge(BlockInputStreamPtr stream, AggregatedDataVariants & result)
{
size_t aggregates_size = aggregates.size();
Row key(keys_size);
Columns key_columns(keys_size);
typedef ColumnAggregateFunction::Container_t * AggregateColumn;
typedef std::vector<AggregateColumn> AggregateColumns;
AggregateColumns aggregate_columns(aggregates_size);
/// Читаем все данные
while (Block block = stream->read())
{
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Merging aggregated block");
2012-05-30 03:30:29 +00:00
if (!sample)
for (size_t i = 0; i < keys_size + aggregates_size; ++i)
sample.insert(block.getByPosition(i).cloneEmpty());
2012-05-30 01:38:02 +00:00
/// Запоминаем столбцы, с которыми будем работать
for (size_t i = 0; i < keys_size; ++i)
key_columns[i] = block.getByPosition(i).column;
for (size_t i = 0; i < aggregates_size; ++i)
aggregate_columns[i] = &dynamic_cast<ColumnAggregateFunction &>(*block.getByPosition(keys_size + i).column).getData();
size_t rows = block.rows();
/// Каким способом выполнять агрегацию?
bool keys_fit_128_bits = false;
Sizes key_sizes;
result.type = chooseAggregationMethod(key_columns, keys_fit_128_bits, key_sizes);
if (result.type == AggregatedDataVariants::WITHOUT_KEY)
{
AggregatedDataWithoutKey & res = result.without_key;
if (res.empty())
{
res.resize(aggregates_size);
for (size_t i = 0; i < aggregates_size; ++i)
2012-05-30 03:30:29 +00:00
res[i] = aggregates[i].function->cloneEmpty();
2012-05-30 01:38:02 +00:00
}
2012-05-30 03:30:29 +00:00
/// Добавляем значения
for (size_t i = 0; i < aggregates_size; ++i)
res[i]->merge(*(*aggregate_columns[i])[0]);
2012-05-30 01:38:02 +00:00
}
else if (result.type == AggregatedDataVariants::KEY_64)
{
AggregatedDataWithUInt64Key & res = result.key64;
const FieldVisitorToUInt64 visitor;
IColumn & column = *key_columns[0];
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
Field field = column[i];
UInt64 key = boost::apply_visitor(visitor, field);
AggregatedDataWithUInt64Key::iterator it;
bool inserted;
res.emplace(key, it, inserted);
if (inserted)
{
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j]->merge(*(*aggregate_columns[j])[i]);
}
}
else if (result.type == AggregatedDataVariants::KEY_STRING)
{
AggregatedDataWithStringKey & res = result.key_string;
IColumn & column = *key_columns[0];
2012-05-31 05:41:56 +00:00
if (const ColumnString * column_string = dynamic_cast<const ColumnString *>(&column))
{
const ColumnString::Offsets_t & offsets = column_string->getOffsets();
const ColumnUInt8::Container_t & data = dynamic_cast<const ColumnUInt8 &>(column_string->getData()).getData();
2012-05-30 01:38:02 +00:00
2012-05-31 05:41:56 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2012-05-30 01:38:02 +00:00
{
2012-05-31 05:41:56 +00:00
/// Строим ключ
StringRef ref(&data[i == 0 ? 0 : offsets[i - 1]], (i == 0 ? offsets[i] : (offsets[i] - offsets[i - 1])) - 1);
2012-05-30 01:38:02 +00:00
2012-05-31 05:41:56 +00:00
AggregatedDataWithStringKey::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
{
it->first.data = result.string_pool.insert(ref.data, ref.size);
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
2012-05-30 01:38:02 +00:00
for (size_t j = 0; j < aggregates_size; ++j)
2012-05-31 05:41:56 +00:00
it->second[j]->merge(*(*aggregate_columns[j])[i]);
2012-05-30 01:38:02 +00:00
}
2012-05-31 05:41:56 +00:00
}
else if (const ColumnFixedString * column_string = dynamic_cast<const ColumnFixedString *>(&column))
{
size_t n = column_string->getN();
const ColumnUInt8::Container_t & data = dynamic_cast<const ColumnUInt8 &>(column_string->getData()).getData();
2012-05-30 01:38:02 +00:00
2012-05-31 05:41:56 +00:00
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
StringRef ref(&data[i * n], n);
AggregatedDataWithStringKey::iterator it;
bool inserted;
res.emplace(ref, it, inserted);
if (inserted)
{
it->first.data = result.string_pool.insert(ref.data, ref.size);
new(&it->second) AggregateFunctionsPlainPtrs(aggregates_size);
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j]->merge(*(*aggregate_columns[j])[i]);
}
2012-05-30 01:38:02 +00:00
}
2012-05-31 05:41:56 +00:00
else
throw Exception("Illegal type of column when aggregating by string key: " + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
2012-05-30 01:38:02 +00:00
}
else if (result.type == AggregatedDataVariants::HASHED)
{
AggregatedDataHashed & res = result.hashed;
const FieldVisitorToUInt64 to_uint64_visitor;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
AggregatedDataHashed::iterator it;
bool inserted;
2012-08-23 23:49:28 +00:00
res.emplace(pack128(i, keys_fit_128_bits, keys_size, key, key_columns, key_sizes), it, inserted);
2012-05-30 01:38:02 +00:00
if (inserted)
{
new(&it->second) AggregatedDataHashed::mapped_type(key, AggregateFunctionsPlainPtrs(aggregates_size));
for (size_t j = 0; j < aggregates_size; ++j)
it->second.second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
it->second.second[j]->merge(*(*aggregate_columns[j])[i]);
}
}
else if (result.type == AggregatedDataVariants::GENERIC)
{
/// Общий способ
AggregatedData & res = result.generic;
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
for (size_t j = 0; j < keys_size; ++j)
key[j] = (*key_columns[j])[i];
AggregatedData::iterator it = res.find(key);
if (it == res.end())
{
it = res.insert(std::make_pair(key, AggregateFunctionsPlainPtrs(aggregates_size))).first;
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j] = aggregates[j].function->cloneEmpty();
}
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
it->second[j]->merge(*(*aggregate_columns[j])[i]);
}
}
else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
2012-05-31 00:33:42 +00:00
LOG_TRACE(log, "Merged aggregated block");
2012-05-30 01:38:02 +00:00
}
}
2011-09-19 01:42:16 +00:00
}