2014-02-27 12:49:21 +00:00
|
|
|
|
#include <DB/DataStreams/TotalsHavingBlockInputStream.h>
|
|
|
|
|
#include <DB/Columns/ColumnAggregateFunction.h>
|
|
|
|
|
#include <DB/Columns/ColumnsNumber.h>
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
static void finalize(Block & block)
|
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
|
{
|
|
|
|
|
ColumnWithNameAndType & current = block.getByPosition(i);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
ColumnAggregateFunction * unfinalized_column = typeid_cast<ColumnAggregateFunction *>(&*current.column);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
if (unfinalized_column)
|
|
|
|
|
{
|
|
|
|
|
current.type = unfinalized_column->getAggregateFunction()->getReturnType();
|
|
|
|
|
current.column = unfinalized_column->convertToValues();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-21 16:09:04 +00:00
|
|
|
|
|
|
|
|
|
const Block & TotalsHavingBlockInputStream::getTotals()
|
|
|
|
|
{
|
|
|
|
|
if (!totals)
|
|
|
|
|
{
|
|
|
|
|
/** Если totals_mode == AFTER_HAVING_AUTO, нужно решить, добавлять ли в TOTALS агрегаты для строк,
|
|
|
|
|
* не прошедших max_rows_to_group_by.
|
|
|
|
|
*/
|
2015-01-02 05:28:21 +00:00
|
|
|
|
if (overflow_aggregates)
|
|
|
|
|
{
|
|
|
|
|
if (totals_mode == TotalsMode::BEFORE_HAVING
|
|
|
|
|
|| totals_mode == TotalsMode::AFTER_HAVING_INCLUSIVE
|
|
|
|
|
|| (totals_mode == TotalsMode::AFTER_HAVING_AUTO
|
|
|
|
|
&& static_cast<double>(passed_keys) / total_keys >= auto_include_threshold))
|
|
|
|
|
addToTotals(current_totals, overflow_aggregates, nullptr);
|
|
|
|
|
}
|
2014-04-21 16:09:04 +00:00
|
|
|
|
|
|
|
|
|
finalize(current_totals);
|
|
|
|
|
totals = current_totals;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (totals && expression)
|
|
|
|
|
expression->execute(totals);
|
|
|
|
|
|
|
|
|
|
return totals;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-02-27 12:49:21 +00:00
|
|
|
|
Block TotalsHavingBlockInputStream::readImpl()
|
|
|
|
|
{
|
|
|
|
|
Block finalized;
|
|
|
|
|
Block block;
|
|
|
|
|
|
|
|
|
|
while (1)
|
|
|
|
|
{
|
|
|
|
|
block = children[0]->read();
|
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
/// В этом случае, первый блок - блок со значениями, не вошедшими в max_rows_to_group_by. Отложим его.
|
|
|
|
|
if (overflow_row && !overflow_aggregates && block)
|
|
|
|
|
{
|
|
|
|
|
overflow_aggregates = block;
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-27 12:49:21 +00:00
|
|
|
|
if (!block)
|
|
|
|
|
return finalized;
|
|
|
|
|
|
|
|
|
|
finalized = block;
|
|
|
|
|
finalize(finalized);
|
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
total_keys += finalized.rows();
|
2014-02-27 12:49:21 +00:00
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
if (filter_column_name.empty())
|
2014-02-27 12:49:21 +00:00
|
|
|
|
{
|
|
|
|
|
addToTotals(current_totals, block, nullptr);
|
|
|
|
|
}
|
2015-01-02 05:28:21 +00:00
|
|
|
|
else
|
2014-02-27 12:49:21 +00:00
|
|
|
|
{
|
2015-01-02 05:28:21 +00:00
|
|
|
|
/// Вычисляем выражение в HAVING.
|
2014-02-27 12:49:21 +00:00
|
|
|
|
expression->execute(finalized);
|
|
|
|
|
|
|
|
|
|
size_t filter_column_pos = finalized.getPositionByName(filter_column_name);
|
|
|
|
|
ColumnPtr filter_column_ptr = finalized.getByPosition(filter_column_pos).column;
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
ColumnConstUInt8 * column_const = typeid_cast<ColumnConstUInt8 *>(&*filter_column_ptr);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
if (column_const)
|
|
|
|
|
filter_column_ptr = column_const->convertToFullColumn();
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
ColumnUInt8 * filter_column = typeid_cast<ColumnUInt8 *>(&*filter_column_ptr);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
if (!filter_column)
|
|
|
|
|
throw Exception("Filter column must have type UInt8, found " +
|
|
|
|
|
finalized.getByPosition(filter_column_pos).type->getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
|
|
|
|
|
|
|
|
|
IColumn::Filter & filter = filter_column->getData();
|
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
/// Прибавляем значения в totals (если это не было сделано ранее).
|
|
|
|
|
if (totals_mode == TotalsMode::BEFORE_HAVING)
|
|
|
|
|
addToTotals(current_totals, block, nullptr);
|
|
|
|
|
else
|
|
|
|
|
addToTotals(current_totals, block, &filter);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
/// Фильтруем блок по выражению в HAVING.
|
2014-02-27 12:49:21 +00:00
|
|
|
|
size_t columns = finalized.columns();
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
|
{
|
|
|
|
|
ColumnWithNameAndType & current_column = finalized.getByPosition(i);
|
|
|
|
|
current_column.column = current_column.column->filter(filter);
|
|
|
|
|
if (current_column.column->empty())
|
|
|
|
|
{
|
|
|
|
|
finalized.clear();
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!finalized)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
passed_keys += finalized.rows();
|
|
|
|
|
return finalized;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-01-02 05:28:21 +00:00
|
|
|
|
void TotalsHavingBlockInputStream::addToTotals(Block & totals, Block & block, const IColumn::Filter * filter)
|
2014-02-27 12:49:21 +00:00
|
|
|
|
{
|
|
|
|
|
bool init = !totals;
|
|
|
|
|
|
|
|
|
|
ArenaPtr arena;
|
|
|
|
|
if (init)
|
|
|
|
|
arena = new Arena;
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
|
{
|
2014-04-21 16:09:04 +00:00
|
|
|
|
const ColumnWithNameAndType & current = block.getByPosition(i);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(&*current.column);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
|
|
|
|
|
if (!column)
|
|
|
|
|
{
|
|
|
|
|
if (init)
|
|
|
|
|
{
|
|
|
|
|
ColumnPtr new_column = current.type->createColumn();
|
|
|
|
|
new_column->insertDefault();
|
|
|
|
|
totals.insert(ColumnWithNameAndType(new_column, current.type, current.name));
|
|
|
|
|
}
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ColumnAggregateFunction * target;
|
|
|
|
|
IAggregateFunction * function;
|
|
|
|
|
AggregateDataPtr data;
|
|
|
|
|
|
|
|
|
|
if (init)
|
|
|
|
|
{
|
|
|
|
|
function = column->getAggregateFunction();
|
2014-06-08 22:17:44 +00:00
|
|
|
|
target = new ColumnAggregateFunction(column->getAggregateFunction(), Arenas(1, arena));
|
2014-02-27 12:49:21 +00:00
|
|
|
|
totals.insert(ColumnWithNameAndType(target, current.type, current.name));
|
|
|
|
|
|
|
|
|
|
data = arena->alloc(function->sizeOfData());
|
|
|
|
|
function->create(data);
|
|
|
|
|
target->getData().push_back(data);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
target = typeid_cast<ColumnAggregateFunction *>(&*totals.getByPosition(i).column);
|
2014-02-27 12:49:21 +00:00
|
|
|
|
if (!target)
|
|
|
|
|
throw Exception("Unexpected type of column: " + totals.getByPosition(i).column->getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
function = target->getAggregateFunction();
|
|
|
|
|
data = target->getData()[0];
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-21 16:09:04 +00:00
|
|
|
|
const ColumnAggregateFunction::Container_t & vec = column->getData();
|
2015-01-02 05:28:21 +00:00
|
|
|
|
size_t size = vec.size();
|
2014-02-27 12:49:21 +00:00
|
|
|
|
|
|
|
|
|
if (filter)
|
|
|
|
|
{
|
|
|
|
|
for (size_t j = 0; j < size; ++j)
|
|
|
|
|
if ((*filter)[j])
|
|
|
|
|
function->merge(data, vec[j]);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
for (size_t j = 0; j < size; ++j)
|
|
|
|
|
function->merge(data, vec[j]);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|