2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/TotalsHavingBlockInputStream.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2018-01-07 00:35:44 +00:00
|
|
|
#include <DataTypes/DataTypeAggregateFunction.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnAggregateFunction.h>
|
2017-12-15 01:34:30 +00:00
|
|
|
#include <Columns/FilterDescription.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2014-02-27 12:49:21 +00:00
|
|
|
|
2017-12-15 01:34:30 +00:00
|
|
|
|
2014-02-27 12:49:21 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2016-01-13 00:32:59 +00:00
|
|
|
TotalsHavingBlockInputStream::TotalsHavingBlockInputStream(
|
2017-09-08 03:47:27 +00:00
|
|
|
const BlockInputStreamPtr & input_,
|
|
|
|
bool overflow_row_, const ExpressionActionsPtr & expression_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_)
|
|
|
|
: overflow_row(overflow_row_),
|
|
|
|
expression(expression_), filter_column_name(filter_column_), totals_mode(totals_mode_),
|
|
|
|
auto_include_threshold(auto_include_threshold_)
|
2016-01-13 00:32:59 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
children.push_back(input_);
|
2018-03-03 01:28:13 +00:00
|
|
|
|
|
|
|
/// Initialize current totals with initial state.
|
|
|
|
|
|
|
|
arena = std::make_shared<Arena>();
|
|
|
|
Block source_header = children.at(0)->getHeader();
|
|
|
|
|
|
|
|
current_totals.reserve(source_header.columns());
|
|
|
|
for (const auto & elem : source_header)
|
|
|
|
{
|
|
|
|
if (const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(elem.column.get()))
|
|
|
|
{
|
|
|
|
/// Create ColumnAggregateFunction with initial aggregate function state.
|
|
|
|
|
|
|
|
IAggregateFunction * function = column->getAggregateFunction().get();
|
|
|
|
auto target = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena));
|
|
|
|
AggregateDataPtr data = arena->alloc(function->sizeOfData());
|
|
|
|
function->create(data);
|
|
|
|
target->getData().push_back(data);
|
|
|
|
current_totals.emplace_back(std::move(target));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
|
|
|
|
/// Not an aggregate function state. Just create a column with default value.
|
|
|
|
|
|
|
|
MutableColumnPtr new_column = elem.type->createColumn();
|
|
|
|
elem.type->insertDefaultInto(*new_column);
|
|
|
|
current_totals.emplace_back(std::move(new_column));
|
|
|
|
}
|
|
|
|
}
|
2016-01-13 00:32:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-02-27 12:49:21 +00:00
|
|
|
static void finalize(Block & block)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
2018-03-03 01:28:13 +00:00
|
|
|
ColumnWithTypeAndName & current = block.getByPosition(i);
|
2018-01-07 00:35:44 +00:00
|
|
|
const DataTypeAggregateFunction * unfinalized_type = typeid_cast<const DataTypeAggregateFunction *>(current.type.get());
|
2017-12-15 02:04:28 +00:00
|
|
|
|
2018-01-07 00:35:44 +00:00
|
|
|
if (unfinalized_type)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-01-07 00:35:44 +00:00
|
|
|
current.type = unfinalized_type->getReturnType();
|
|
|
|
if (current.column)
|
|
|
|
current.column = typeid_cast<const ColumnAggregateFunction &>(*current.column).convertToValues();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2014-02-27 12:49:21 +00:00
|
|
|
}
|
|
|
|
|
2014-04-21 16:09:04 +00:00
|
|
|
|
2018-02-23 10:43:24 +00:00
|
|
|
Block TotalsHavingBlockInputStream::getTotals()
|
2014-04-21 16:09:04 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!totals)
|
|
|
|
{
|
|
|
|
/** If totals_mode == AFTER_HAVING_AUTO, you need to decide whether to add aggregates to TOTALS for strings,
|
|
|
|
* not passed max_rows_to_group_by.
|
|
|
|
*/
|
|
|
|
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))
|
2018-01-13 01:33:55 +00:00
|
|
|
addToTotals(overflow_aggregates, nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-02-16 19:04:27 +00:00
|
|
|
totals = children.at(0)->getHeader().cloneWithColumns(std::move(current_totals));
|
2017-12-15 02:04:28 +00:00
|
|
|
finalize(totals);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (totals && expression)
|
|
|
|
expression->execute(totals);
|
|
|
|
|
|
|
|
return totals;
|
2014-04-21 16:09:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-02-18 03:23:48 +00:00
|
|
|
Block TotalsHavingBlockInputStream::getHeader() const
|
2018-01-06 18:10:44 +00:00
|
|
|
{
|
2018-01-09 00:19:58 +00:00
|
|
|
Block res = children.at(0)->getHeader();
|
2018-01-06 18:10:44 +00:00
|
|
|
finalize(res);
|
2018-01-09 00:19:58 +00:00
|
|
|
if (expression)
|
|
|
|
expression->execute(res);
|
2018-01-06 18:10:44 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-02-27 12:49:21 +00:00
|
|
|
Block TotalsHavingBlockInputStream::readImpl()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Block finalized;
|
|
|
|
Block block;
|
|
|
|
|
|
|
|
while (1)
|
|
|
|
{
|
|
|
|
block = children[0]->read();
|
|
|
|
|
|
|
|
/// Block with values not included in `max_rows_to_group_by`. We'll postpone it.
|
|
|
|
if (overflow_row && block && block.info.is_overflows)
|
|
|
|
{
|
|
|
|
overflow_aggregates = block;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!block)
|
|
|
|
return finalized;
|
|
|
|
|
|
|
|
finalized = block;
|
|
|
|
finalize(finalized);
|
|
|
|
|
|
|
|
total_keys += finalized.rows();
|
|
|
|
|
|
|
|
if (filter_column_name.empty())
|
|
|
|
{
|
2018-01-13 01:33:55 +00:00
|
|
|
addToTotals(block, nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Compute the expression in HAVING.
|
|
|
|
expression->execute(finalized);
|
|
|
|
|
|
|
|
size_t filter_column_pos = finalized.getPositionByName(filter_column_name);
|
|
|
|
ColumnPtr filter_column_ptr = finalized.safeGetByPosition(filter_column_pos).column;
|
|
|
|
|
2017-12-15 01:34:30 +00:00
|
|
|
if (ColumnPtr materialized = filter_column_ptr->convertToFullColumnIfConst())
|
|
|
|
filter_column_ptr = materialized;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-15 02:04:28 +00:00
|
|
|
FilterDescription filter_description(*filter_column_ptr);
|
2017-12-09 13:28:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Add values to `totals` (if it was not already done).
|
|
|
|
if (totals_mode == TotalsMode::BEFORE_HAVING)
|
2018-01-13 01:33:55 +00:00
|
|
|
addToTotals(block, nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2018-01-13 01:33:55 +00:00
|
|
|
addToTotals(block, filter_description.data);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Filter the block by expression in HAVING.
|
|
|
|
size_t columns = finalized.columns();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
{
|
|
|
|
ColumnWithTypeAndName & current_column = finalized.safeGetByPosition(i);
|
2017-12-15 01:34:30 +00:00
|
|
|
current_column.column = current_column.column->filter(*filter_description.data, -1);
|
2017-04-01 07:20:54 +00:00
|
|
|
if (current_column.column->empty())
|
|
|
|
{
|
|
|
|
finalized.clear();
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!finalized)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
passed_keys += finalized.rows();
|
|
|
|
return finalized;
|
|
|
|
}
|
2014-02-27 12:49:21 +00:00
|
|
|
}
|
|
|
|
|
2017-12-15 02:04:28 +00:00
|
|
|
|
2018-01-13 01:33:55 +00:00
|
|
|
void TotalsHavingBlockInputStream::addToTotals(const Block & block, const IColumn::Filter * filter)
|
2014-02-27 12:49:21 +00:00
|
|
|
{
|
2017-12-15 02:04:28 +00:00
|
|
|
for (size_t i = 0, num_columns = block.columns(); i < num_columns; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-03-03 01:28:13 +00:00
|
|
|
const ColumnWithTypeAndName & current = block.getByPosition(i);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-03 01:28:13 +00:00
|
|
|
if (const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(current.column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-01-13 01:33:55 +00:00
|
|
|
auto & target = typeid_cast<ColumnAggregateFunction &>(*current_totals[i]);
|
2018-03-03 01:28:13 +00:00
|
|
|
IAggregateFunction * function = target.getAggregateFunction().get();
|
|
|
|
AggregateDataPtr data = target.getData()[0];
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-03 01:28:13 +00:00
|
|
|
/// Accumulate all aggregate states into that value.
|
2017-12-15 02:04:28 +00:00
|
|
|
|
2018-03-03 01:28:13 +00:00
|
|
|
const ColumnAggregateFunction::Container & vec = column->getData();
|
|
|
|
size_t size = vec.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-03 01:28:13 +00:00
|
|
|
if (filter)
|
|
|
|
{
|
|
|
|
for (size_t j = 0; j < size; ++j)
|
|
|
|
if ((*filter)[j])
|
|
|
|
function->merge(data, vec[j], arena.get());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (size_t j = 0; j < size; ++j)
|
2017-04-01 07:20:54 +00:00
|
|
|
function->merge(data, vec[j], arena.get());
|
2018-03-03 01:28:13 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2014-02-27 12:49:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|