mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #9885 from ClickHouse/sorting-processors
Sorting processors
This commit is contained in:
commit
83e17ae327
@ -311,6 +311,8 @@ add_object_library(clickhouse_processors_formats Processors/Formats)
|
||||
add_object_library(clickhouse_processors_formats_impl Processors/Formats/Impl)
|
||||
add_object_library(clickhouse_processors_transforms Processors/Transforms)
|
||||
add_object_library(clickhouse_processors_sources Processors/Sources)
|
||||
add_object_library(clickhouse_processors_merges Processors/Merges)
|
||||
add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms)
|
||||
|
||||
|
||||
if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES)
|
||||
|
@ -1,249 +0,0 @@
|
||||
#include <DataStreams/AggregatingSortedBlockInputStream.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
class RemovingLowCardinalityBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
RemovingLowCardinalityBlockInputStream(BlockInputStreamPtr input_, ColumnNumbers positions_)
|
||||
: input(std::move(input_)), positions(std::move(positions_))
|
||||
{
|
||||
header = transform(input->getHeader());
|
||||
}
|
||||
|
||||
Block transform(Block block)
|
||||
{
|
||||
if (block)
|
||||
{
|
||||
for (auto & pos : positions)
|
||||
{
|
||||
auto & col = block.safeGetByPosition(pos);
|
||||
col.column = recursiveRemoveLowCardinality(col.column);
|
||||
col.type = recursiveRemoveLowCardinality(col.type);
|
||||
}
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
String getName() const override { return "RemovingLowCardinality"; }
|
||||
Block getHeader() const override { return header; }
|
||||
const BlockMissingValues & getMissingValues() const override { return input->getMissingValues(); }
|
||||
bool isSortedOutput() const override { return input->isSortedOutput(); }
|
||||
const SortDescription & getSortDescription() const override { return input->getSortDescription(); }
|
||||
|
||||
protected:
|
||||
Block readImpl() override { return transform(input->read()); }
|
||||
|
||||
private:
|
||||
Block header;
|
||||
BlockInputStreamPtr input;
|
||||
ColumnNumbers positions;
|
||||
};
|
||||
|
||||
|
||||
AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_)
|
||||
{
|
||||
/// Fill in the column numbers that need to be aggregated.
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
ColumnWithTypeAndName & column = header.safeGetByPosition(i);
|
||||
|
||||
/// We leave only states of aggregate functions.
|
||||
if (!dynamic_cast<const DataTypeAggregateFunction *>(column.type.get()) && !dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Included into PK?
|
||||
SortDescription::const_iterator it = description.begin();
|
||||
for (; it != description.end(); ++it)
|
||||
if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i))
|
||||
break;
|
||||
|
||||
if (it != description.end())
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (auto simple_aggr = dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
|
||||
{
|
||||
// simple aggregate function
|
||||
SimpleAggregateDescription desc{simple_aggr->getFunction(), i};
|
||||
if (desc.function->allocatesMemoryInArena())
|
||||
allocatesMemoryInArena = true;
|
||||
|
||||
columns_to_simple_aggregate.emplace_back(std::move(desc));
|
||||
|
||||
if (recursiveRemoveLowCardinality(column.type).get() != column.type.get())
|
||||
converted_lc_columns.emplace_back(i);
|
||||
}
|
||||
else
|
||||
{
|
||||
// standard aggregate function
|
||||
column_numbers_to_aggregate.push_back(i);
|
||||
}
|
||||
}
|
||||
|
||||
result_header = header;
|
||||
|
||||
if (!converted_lc_columns.empty())
|
||||
{
|
||||
for (auto & input : children)
|
||||
input = std::make_shared<RemovingLowCardinalityBlockInputStream>(input, converted_lc_columns);
|
||||
|
||||
header = children.at(0)->getHeader();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Block AggregatingSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return Block();
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return Block();
|
||||
|
||||
columns_to_aggregate.resize(column_numbers_to_aggregate.size());
|
||||
for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i)
|
||||
columns_to_aggregate[i] = typeid_cast<ColumnAggregateFunction *>(merged_columns[column_numbers_to_aggregate[i]].get());
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
|
||||
for (auto & pos : converted_lc_columns)
|
||||
{
|
||||
auto & from_type = header.getByPosition(pos).type;
|
||||
auto & to_type = result_header.getByPosition(pos).type;
|
||||
merged_columns[pos] = (*recursiveTypeConversion(std::move(merged_columns[pos]), from_type, to_type)).mutate();
|
||||
}
|
||||
|
||||
return result_header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
||||
void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
/// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
|
||||
setPrimaryKeyRef(next_key, current);
|
||||
|
||||
bool key_differs;
|
||||
|
||||
if (current_key.empty()) /// The first key encountered.
|
||||
{
|
||||
setPrimaryKeyRef(current_key, current);
|
||||
key_differs = true;
|
||||
}
|
||||
else
|
||||
key_differs = next_key != current_key;
|
||||
|
||||
/// if there are enough rows accumulated and the last one is calculated completely
|
||||
if (key_differs && merged_rows >= max_block_size)
|
||||
{
|
||||
/// Write the simple aggregation result for the previous group.
|
||||
insertSimpleAggregationResult(merged_columns);
|
||||
return;
|
||||
}
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
current_key.swap(next_key);
|
||||
|
||||
/// We will write the data for the group. We copy the values of ordinary columns.
|
||||
for (size_t j : column_numbers_not_to_aggregate)
|
||||
merged_columns[j]->insertFrom(*current->all_columns[j], current->pos);
|
||||
|
||||
/// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function.
|
||||
for (auto & column_to_aggregate : columns_to_aggregate)
|
||||
column_to_aggregate->insertDefault();
|
||||
|
||||
/// Write the simple aggregation result for the previous group.
|
||||
if (merged_rows > 0)
|
||||
insertSimpleAggregationResult(merged_columns);
|
||||
|
||||
/// Reset simple aggregation states for next row
|
||||
for (auto & desc : columns_to_simple_aggregate)
|
||||
desc.createState();
|
||||
|
||||
if (allocatesMemoryInArena)
|
||||
arena = std::make_unique<Arena>();
|
||||
|
||||
++merged_rows;
|
||||
}
|
||||
|
||||
addRow(current);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We fetch the next block from the appropriate source, if there is one.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
}
|
||||
|
||||
/// Write the simple aggregation result for the previous group.
|
||||
if (merged_rows > 0)
|
||||
insertSimpleAggregationResult(merged_columns);
|
||||
|
||||
finished = true;
|
||||
}
|
||||
|
||||
|
||||
void AggregatingSortedBlockInputStream::addRow(SortCursor & cursor)
|
||||
{
|
||||
for (size_t i = 0, size = column_numbers_to_aggregate.size(); i < size; ++i)
|
||||
{
|
||||
size_t j = column_numbers_to_aggregate[i];
|
||||
columns_to_aggregate[i]->insertMergeFrom(*cursor->all_columns[j], cursor->pos);
|
||||
}
|
||||
|
||||
for (auto & desc : columns_to_simple_aggregate)
|
||||
{
|
||||
auto & col = cursor->all_columns[desc.column_number];
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get());
|
||||
}
|
||||
}
|
||||
|
||||
void AggregatingSortedBlockInputStream::insertSimpleAggregationResult(MutableColumns & merged_columns)
|
||||
{
|
||||
for (auto & desc : columns_to_simple_aggregate)
|
||||
{
|
||||
desc.function->insertResultInto(desc.state.data(), *merged_columns[desc.column_number]);
|
||||
desc.destroyState();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,123 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <memory>
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Arena;
|
||||
|
||||
/** Merges several sorted streams to one.
|
||||
* During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions,
|
||||
* corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type,
|
||||
* when merged, the first value is selected.
|
||||
*/
|
||||
class AggregatingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
AggregatingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_);
|
||||
|
||||
String getName() const override { return "AggregatingSorted"; }
|
||||
|
||||
bool isSortedOutput() const override { return true; }
|
||||
|
||||
Block getHeader() const override { return result_header; }
|
||||
|
||||
protected:
|
||||
/// Can return 1 more records than max_block_size.
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("AggregatingSortedBlockInputStream");
|
||||
|
||||
/// Read finished.
|
||||
bool finished = false;
|
||||
|
||||
struct SimpleAggregateDescription;
|
||||
|
||||
/// Columns with which numbers should be aggregated.
|
||||
ColumnNumbers column_numbers_to_aggregate;
|
||||
ColumnNumbers column_numbers_not_to_aggregate;
|
||||
std::vector<ColumnAggregateFunction *> columns_to_aggregate;
|
||||
std::vector<SimpleAggregateDescription> columns_to_simple_aggregate;
|
||||
|
||||
SharedBlockRowRef current_key; /// The current primary key.
|
||||
SharedBlockRowRef next_key; /// The primary key of the next row.
|
||||
|
||||
Block result_header;
|
||||
ColumnNumbers converted_lc_columns;
|
||||
|
||||
/** We support two different cursors - with Collation and without.
|
||||
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
|
||||
*/
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/** Extract all states of aggregate functions and merge them with the current group.
|
||||
*/
|
||||
void addRow(SortCursor & cursor);
|
||||
|
||||
/** Insert all values of current row for simple aggregate functions
|
||||
*/
|
||||
void insertSimpleAggregationResult(MutableColumns & merged_columns);
|
||||
|
||||
/// Does SimpleAggregateFunction allocates memory in arena?
|
||||
bool allocatesMemoryInArena = false;
|
||||
/// Memory pool for SimpleAggregateFunction
|
||||
/// (only when allocatesMemoryInArena == true).
|
||||
std::unique_ptr<Arena> arena;
|
||||
|
||||
/// Stores information for aggregation of SimpleAggregateFunction columns
|
||||
struct SimpleAggregateDescription
|
||||
{
|
||||
/// An aggregate function 'anyLast', 'sum'...
|
||||
AggregateFunctionPtr function;
|
||||
IAggregateFunction::AddFunc add_function;
|
||||
size_t column_number;
|
||||
AlignedBuffer state;
|
||||
bool created = false;
|
||||
|
||||
SimpleAggregateDescription(const AggregateFunctionPtr & function_, const size_t column_number_) : function(function_), column_number(column_number_)
|
||||
{
|
||||
add_function = function->getAddressOfAddFunction();
|
||||
state.reset(function->sizeOfData(), function->alignOfData());
|
||||
}
|
||||
|
||||
void createState()
|
||||
{
|
||||
if (created)
|
||||
return;
|
||||
function->create(state.data());
|
||||
created = true;
|
||||
}
|
||||
|
||||
void destroyState()
|
||||
{
|
||||
if (!created)
|
||||
return;
|
||||
function->destroy(state.data());
|
||||
created = false;
|
||||
}
|
||||
|
||||
/// Explicitly destroy aggregation state if the stream is terminated
|
||||
~SimpleAggregateDescription()
|
||||
{
|
||||
destroyState();
|
||||
}
|
||||
|
||||
SimpleAggregateDescription() = default;
|
||||
SimpleAggregateDescription(SimpleAggregateDescription &&) = default;
|
||||
SimpleAggregateDescription(const SimpleAggregateDescription &) = delete;
|
||||
};
|
||||
};
|
||||
|
||||
}
|
@ -1,201 +0,0 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataStreams/CollapsingSortedBlockInputStream.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
/// Maximum number of messages about incorrect data in the log.
|
||||
#define MAX_ERROR_MESSAGES 10
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
void CollapsingSortedBlockInputStream::reportIncorrectData()
|
||||
{
|
||||
std::stringstream s;
|
||||
s << "Incorrect data: number of rows with sign = 1 (" << count_positive
|
||||
<< ") differs with number of rows with sign = -1 (" << count_negative
|
||||
<< ") by more than one (for key: ";
|
||||
|
||||
for (size_t i = 0, size = current_key.size(); i < size; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
s << ", ";
|
||||
s << applyVisitor(FieldVisitorToString(), (*(*current_key.columns)[i])[current_key.row_num]);
|
||||
}
|
||||
|
||||
s << ").";
|
||||
|
||||
/** Fow now we limit ourselves to just logging such situations,
|
||||
* since the data is generated by external programs.
|
||||
* With inconsistent data, this is an unavoidable error that can not be easily corrected by admins. Therefore Warning.
|
||||
*/
|
||||
LOG_WARNING(log, s.rdbuf());
|
||||
}
|
||||
|
||||
|
||||
void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_columns, size_t block_size, MergeStopCondition & condition)
|
||||
{
|
||||
if (count_positive == 0 && count_negative == 0)
|
||||
{
|
||||
/// No input rows have been read.
|
||||
return;
|
||||
}
|
||||
|
||||
if (last_is_positive || count_positive != count_negative)
|
||||
{
|
||||
if (count_positive <= count_negative)
|
||||
{
|
||||
condition.addRowWithGranularity(block_size);
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*(*first_negative.columns)[i], first_negative.row_num);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources[first_negative_pos].setSkipFlag(false);
|
||||
}
|
||||
|
||||
if (count_positive >= count_negative)
|
||||
{
|
||||
condition.addRowWithGranularity(block_size);
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*(*last_positive.columns)[i], last_positive.row_num);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources[last_positive_pos].setSkipFlag(false);
|
||||
}
|
||||
|
||||
if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1))
|
||||
{
|
||||
if (count_incorrect_data < MAX_ERROR_MESSAGES)
|
||||
reportIncorrectData();
|
||||
++count_incorrect_data;
|
||||
}
|
||||
}
|
||||
|
||||
if (out_row_sources_buf)
|
||||
out_row_sources_buf->write(
|
||||
reinterpret_cast<const char *>(current_row_sources.data()),
|
||||
current_row_sources.size() * sizeof(RowSourcePart));
|
||||
}
|
||||
|
||||
|
||||
Block CollapsingSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return {};
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return {};
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
||||
void CollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
{
|
||||
|
||||
MergeStopCondition stop_condition(average_block_sizes, max_block_size);
|
||||
size_t current_block_granularity;
|
||||
/// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size`
|
||||
for (; queue.isValid(); ++current_pos)
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
current_block_granularity = current->rows;
|
||||
|
||||
if (current_key.empty())
|
||||
setPrimaryKeyRef(current_key, current);
|
||||
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
setPrimaryKeyRef(next_key, current);
|
||||
|
||||
bool key_differs = next_key != current_key;
|
||||
|
||||
/// if there are enough rows and the last one is calculated completely
|
||||
if (key_differs && stop_condition.checkStop())
|
||||
{
|
||||
++blocks_written;
|
||||
return;
|
||||
}
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
/// We write data for the previous primary key.
|
||||
insertRows(merged_columns, current_block_granularity, stop_condition);
|
||||
|
||||
current_key.swap(next_key);
|
||||
|
||||
count_negative = 0;
|
||||
count_positive = 0;
|
||||
|
||||
current_pos = 0;
|
||||
first_negative_pos = 0;
|
||||
last_positive_pos = 0;
|
||||
last_negative_pos = 0;
|
||||
current_row_sources.resize(0);
|
||||
}
|
||||
|
||||
/// Initially, skip all rows. On insert, unskip "corner" rows.
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace_back(current.impl->order, true);
|
||||
|
||||
if (sign == 1)
|
||||
{
|
||||
++count_positive;
|
||||
last_is_positive = true;
|
||||
|
||||
setRowRef(last_positive, current);
|
||||
last_positive_pos = current_pos;
|
||||
}
|
||||
else if (sign == -1)
|
||||
{
|
||||
if (!count_negative)
|
||||
{
|
||||
setRowRef(first_negative, current);
|
||||
first_negative_pos = current_pos;
|
||||
}
|
||||
|
||||
if (!blocks_written && stop_condition.empty())
|
||||
{
|
||||
setRowRef(last_negative, current);
|
||||
last_negative_pos = current_pos;
|
||||
}
|
||||
|
||||
++count_negative;
|
||||
last_is_positive = false;
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).",
|
||||
ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We take next block from the corresponding source, if there is one.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
}
|
||||
|
||||
/// Write data for last primary key.
|
||||
insertRows(merged_columns, /*some_granularity*/ 0, stop_condition);
|
||||
|
||||
finished = true;
|
||||
}
|
||||
|
||||
}
|
@ -1,84 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted streams to one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* keeps no more than one row with the value of the column `sign_column = -1` ("negative row")
|
||||
* and no more than a row with the value of the column `sign_column = 1` ("positive row").
|
||||
* That is, it collapses the records from the change log.
|
||||
*
|
||||
* If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written.
|
||||
* If the number of positive and negative rows is the same, and the last line is negative, it writes nothing.
|
||||
* If the positive by 1 is greater than the negative rows, then only the last positive row is written.
|
||||
* If negative by 1 is greater than positive rows, then only the first negative row is written.
|
||||
* Otherwise, a logical error.
|
||||
*/
|
||||
class CollapsingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
CollapsingSortedBlockInputStream(
|
||||
BlockInputStreams inputs_, const SortDescription & description_,
|
||||
const String & sign_column, size_t max_block_size_,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr, bool average_block_sizes_ = false)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_)
|
||||
{
|
||||
sign_column_number = header.getPositionByName(sign_column);
|
||||
}
|
||||
|
||||
String getName() const override { return "CollapsingSorted"; }
|
||||
|
||||
protected:
|
||||
/// Can return 1 more records than max_block_size.
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
size_t sign_column_number;
|
||||
|
||||
Logger * log = &Logger::get("CollapsingSortedBlockInputStream");
|
||||
|
||||
/// Read is finished.
|
||||
bool finished = false;
|
||||
|
||||
SharedBlockRowRef current_key; /// The current primary key.
|
||||
SharedBlockRowRef next_key; /// The primary key of the next row.
|
||||
|
||||
SharedBlockRowRef first_negative; /// The first negative row for the current primary key.
|
||||
SharedBlockRowRef last_positive; /// The last positive row for the current primary key.
|
||||
SharedBlockRowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output.
|
||||
|
||||
size_t count_positive = 0; /// The number of positive rows for the current primary key.
|
||||
size_t count_negative = 0; /// The number of negative rows for the current primary key.
|
||||
bool last_is_positive = false; /// true if the last row for the current primary key is positive.
|
||||
|
||||
size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log.
|
||||
|
||||
size_t blocks_written = 0;
|
||||
|
||||
/// Fields specific for VERTICAL merge algorithm.
|
||||
/// Row numbers are relative to the start of current primary key.
|
||||
size_t current_pos = 0; /// Current row number
|
||||
size_t first_negative_pos = 0; /// Row number of first_negative
|
||||
size_t last_positive_pos = 0; /// Row number of last_positive
|
||||
size_t last_negative_pos = 0; /// Row number of last_negative
|
||||
PODArray<RowSourcePart> current_row_sources; /// Sources of rows with the current primary key
|
||||
|
||||
/** We support two different cursors - with Collation and without.
|
||||
* Templates are used instead of polymorphic SortCursors and calls to virtual functions.
|
||||
*/
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/// Output to result rows for the current primary key.
|
||||
void insertRows(MutableColumns & merged_columns, size_t block_size, MergeStopCondition & condition);
|
||||
|
||||
void reportIncorrectData();
|
||||
};
|
||||
|
||||
}
|
@ -1,241 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Intended for implementation of "rollup" - aggregation (rounding) of older data
|
||||
* for a table with Graphite data (Graphite is the system for time series monitoring).
|
||||
*
|
||||
* Table with graphite data has at least the following columns (accurate to the name):
|
||||
* Path, Time, Value, Version
|
||||
*
|
||||
* Path - name of metric (sensor);
|
||||
* Time - time of measurement;
|
||||
* Value - value of measurement;
|
||||
* Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version.
|
||||
*
|
||||
* Each row in a table correspond to one value of one sensor.
|
||||
*
|
||||
* Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well:
|
||||
* * Aggregation OR retention patterns should be first
|
||||
* * Then aggregation AND retention full patterns have to be placed
|
||||
* * default pattern without regexp must be the last
|
||||
*
|
||||
* Rollup rules are specified in the following way:
|
||||
*
|
||||
* pattern
|
||||
* regexp
|
||||
* function
|
||||
* pattern
|
||||
* regexp
|
||||
* age -> precision
|
||||
* age -> precision
|
||||
* ...
|
||||
* pattern
|
||||
* regexp
|
||||
* function
|
||||
* age -> precision
|
||||
* age -> precision
|
||||
* ...
|
||||
* pattern
|
||||
* ...
|
||||
* default
|
||||
* function
|
||||
* age -> precision
|
||||
* ...
|
||||
*
|
||||
* regexp - pattern for sensor name
|
||||
* default - if no pattern has matched
|
||||
*
|
||||
* age - minimal data age (in seconds), to start rounding with specified precision.
|
||||
* precision - rounding precision (in seconds)
|
||||
*
|
||||
* function - name of aggregate function to be applied for values, that time was rounded to same.
|
||||
*
|
||||
* Example:
|
||||
*
|
||||
* <graphite_rollup>
|
||||
* <pattern>
|
||||
* <regexp>\.max$</regexp>
|
||||
* <function>max</function>
|
||||
* </pattern>
|
||||
* <pattern>
|
||||
* <regexp>click_cost</regexp>
|
||||
* <function>any</function>
|
||||
* <retention>
|
||||
* <age>0</age>
|
||||
* <precision>5</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>86400</age>
|
||||
* <precision>60</precision>
|
||||
* </retention>
|
||||
* </pattern>
|
||||
* <default>
|
||||
* <function>max</function>
|
||||
* <retention>
|
||||
* <age>0</age>
|
||||
* <precision>60</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>3600</age>
|
||||
* <precision>300</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>86400</age>
|
||||
* <precision>3600</precision>
|
||||
* </retention>
|
||||
* </default>
|
||||
* </graphite_rollup>
|
||||
*/
|
||||
|
||||
namespace Graphite
|
||||
{
|
||||
struct Retention
|
||||
{
|
||||
UInt32 age;
|
||||
UInt32 precision;
|
||||
};
|
||||
|
||||
using Retentions = std::vector<Retention>;
|
||||
|
||||
struct Pattern
|
||||
{
|
||||
std::shared_ptr<OptimizedRegularExpression> regexp;
|
||||
std::string regexp_str;
|
||||
AggregateFunctionPtr function;
|
||||
Retentions retentions; /// Must be ordered by 'age' descending.
|
||||
enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically
|
||||
};
|
||||
|
||||
using Patterns = std::vector<Pattern>;
|
||||
using RetentionPattern = Pattern;
|
||||
using AggregationPattern = Pattern;
|
||||
|
||||
struct Params
|
||||
{
|
||||
String config_name;
|
||||
String path_column_name;
|
||||
String time_column_name;
|
||||
String value_column_name;
|
||||
String version_column_name;
|
||||
Graphite::Patterns patterns;
|
||||
};
|
||||
|
||||
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
|
||||
}
|
||||
|
||||
/** Merges several sorted streams into one.
|
||||
*
|
||||
* For each group of consecutive identical values of the `path` column,
|
||||
* and the same `time` values, rounded to some precision
|
||||
* (where rounding accuracy depends on the template set for `path`
|
||||
* and the amount of time elapsed from `time` to the specified time),
|
||||
* keeps one line,
|
||||
* performing the rounding of time,
|
||||
* merge `value` values using the specified aggregate functions,
|
||||
* as well as keeping the maximum value of the `version` column.
|
||||
*/
|
||||
class GraphiteRollupSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
GraphiteRollupSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_,
|
||||
const Graphite::Params & params_, time_t time_of_merge_);
|
||||
|
||||
String getName() const override { return "GraphiteRollupSorted"; }
|
||||
|
||||
~GraphiteRollupSortedBlockInputStream() override
|
||||
{
|
||||
if (aggregate_state_created)
|
||||
std::get<1>(current_rule)->function->destroy(place_for_aggregate_state.data());
|
||||
}
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("GraphiteRollupSortedBlockInputStream");
|
||||
|
||||
const Graphite::Params params;
|
||||
|
||||
size_t path_column_num;
|
||||
size_t time_column_num;
|
||||
size_t value_column_num;
|
||||
size_t version_column_num;
|
||||
|
||||
/// All columns other than 'time', 'value', 'version'. They are unmodified during rollup.
|
||||
ColumnNumbers unmodified_column_numbers;
|
||||
|
||||
time_t time_of_merge;
|
||||
|
||||
/// No data has been read.
|
||||
bool is_first = true;
|
||||
|
||||
/// All data has been read.
|
||||
bool finished = false;
|
||||
|
||||
/* | path | time | rounded_time | version | value | unmodified |
|
||||
* -----------------------------------------------------------------------------------
|
||||
* | A | 11 | 10 | 1 | 1 | a | |
|
||||
* | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) |
|
||||
* | A | 11 | 10 | 2 | 3 | c | |> group(A, 10)
|
||||
* ----------------------------------------------------------------------------------|>
|
||||
* | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a)
|
||||
* | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) |
|
||||
* -----------------------------------------------------------------------------------
|
||||
* | A | 21 | 20 | 1 | 6 | f |
|
||||
* | B | 11 | 10 | 1 | 7 | g |
|
||||
* ...
|
||||
*/
|
||||
|
||||
/// Path name of current bucket
|
||||
StringRef current_group_path;
|
||||
|
||||
/// Last row with maximum version for current primary key (time bucket).
|
||||
SharedBlockRowRef current_subgroup_newest_row;
|
||||
|
||||
/// Time of last read row
|
||||
time_t current_time = 0;
|
||||
time_t current_time_rounded = 0;
|
||||
|
||||
Graphite::RollupRule current_rule = {nullptr, nullptr};
|
||||
AlignedBuffer place_for_aggregate_state;
|
||||
bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL.
|
||||
|
||||
const Graphite::Pattern undef_pattern =
|
||||
{ /// temporary empty pattern for selectPatternForPath
|
||||
nullptr,
|
||||
"",
|
||||
nullptr,
|
||||
DB::Graphite::Retentions(),
|
||||
undef_pattern.TypeUndef,
|
||||
};
|
||||
Graphite::RollupRule selectPatternForPath(StringRef path) const;
|
||||
UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const;
|
||||
|
||||
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/// Insert the values into the resulting columns, which will not be changed in the future.
|
||||
template <typename TSortCursor>
|
||||
void startNextGroup(MutableColumns & merged_columns, TSortCursor & cursor, Graphite::RollupRule next_rule);
|
||||
|
||||
/// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows.
|
||||
void finishCurrentGroup(MutableColumns & merged_columns);
|
||||
|
||||
/// Update the state of the aggregate function with the new `value`.
|
||||
void accumulateRow(SharedBlockRowRef & row);
|
||||
};
|
||||
|
||||
}
|
@ -18,10 +18,10 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
MergingSortedBlockInputStream::MergingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_,
|
||||
size_t max_block_size_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_, bool average_block_sizes_)
|
||||
: description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, average_block_sizes(average_block_sizes_), source_blocks(inputs_.size())
|
||||
const BlockInputStreams & inputs_, SortDescription description_,
|
||||
size_t max_block_size_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool quiet_)
|
||||
: description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, source_blocks(inputs_.size())
|
||||
, cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_)
|
||||
, log(&Logger::get("MergingSortedBlockInputStream"))
|
||||
{
|
||||
@ -39,14 +39,14 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns)
|
||||
|
||||
for (size_t i = 0; i < source_blocks.size(); ++i)
|
||||
{
|
||||
SharedBlockPtr & shared_block_ptr = source_blocks[i];
|
||||
Block & block = source_blocks[i];
|
||||
|
||||
if (shared_block_ptr.get())
|
||||
if (block)
|
||||
continue;
|
||||
|
||||
shared_block_ptr = new detail::SharedBlock(children[i]->read());
|
||||
block = children[i]->read();
|
||||
|
||||
const size_t rows = shared_block_ptr->rows();
|
||||
const size_t rows = block.rows();
|
||||
|
||||
if (rows == 0)
|
||||
continue;
|
||||
@ -54,9 +54,7 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns)
|
||||
if (expected_block_size < rows)
|
||||
expected_block_size = std::min(rows, max_block_size);
|
||||
|
||||
cursors[i] = SortCursorImpl(*shared_block_ptr, description, i);
|
||||
shared_block_ptr->all_columns = cursors[i].all_columns;
|
||||
shared_block_ptr->sort_columns = cursors[i].sort_columns;
|
||||
cursors[i] = SortCursorImpl(block, description, i);
|
||||
has_collation |= cursors[i].has_collation;
|
||||
}
|
||||
|
||||
@ -67,12 +65,12 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns)
|
||||
}
|
||||
|
||||
/// Let's check that all source blocks have the same structure.
|
||||
for (const SharedBlockPtr & shared_block_ptr : source_blocks)
|
||||
for (const auto & block : source_blocks)
|
||||
{
|
||||
if (!*shared_block_ptr)
|
||||
if (!block)
|
||||
continue;
|
||||
|
||||
assertBlocksHaveEqualStructure(*shared_block_ptr, header, getName());
|
||||
assertBlocksHaveEqualStructure(block, header, getName());
|
||||
}
|
||||
|
||||
merged_columns.resize(num_columns);
|
||||
@ -118,51 +116,33 @@ void MergingSortedBlockInputStream::fetchNextBlock(const TSortCursor & current,
|
||||
|
||||
while (true)
|
||||
{
|
||||
source_blocks[order] = new detail::SharedBlock(children[order]->read()); /// intrusive ptr
|
||||
source_blocks[order] = children[order]->read();
|
||||
|
||||
if (!*source_blocks[order])
|
||||
if (!source_blocks[order])
|
||||
{
|
||||
queue.removeTop();
|
||||
break;
|
||||
}
|
||||
|
||||
if (source_blocks[order]->rows())
|
||||
if (source_blocks[order].rows())
|
||||
{
|
||||
cursors[order].reset(*source_blocks[order]);
|
||||
cursors[order].reset(source_blocks[order]);
|
||||
queue.replaceTop(&cursors[order]);
|
||||
|
||||
source_blocks[order]->all_columns = cursors[order].all_columns;
|
||||
source_blocks[order]->sort_columns = cursors[order].sort_columns;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool MergingSortedBlockInputStream::MergeStopCondition::checkStop() const
|
||||
{
|
||||
if (!count_average)
|
||||
return sum_rows_count == max_block_size;
|
||||
|
||||
if (sum_rows_count == 0)
|
||||
return false;
|
||||
|
||||
size_t average = sum_blocks_granularity / sum_rows_count;
|
||||
return sum_rows_count >= average;
|
||||
}
|
||||
|
||||
|
||||
template <typename TSortingHeap>
|
||||
void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSortingHeap & queue)
|
||||
{
|
||||
size_t merged_rows = 0;
|
||||
|
||||
MergeStopCondition stop_condition(average_block_sizes, max_block_size);
|
||||
|
||||
/** Increase row counters.
|
||||
* Return true if it's time to finish generating the current data block.
|
||||
*/
|
||||
auto count_row_and_check_limit = [&, this](size_t current_granularity)
|
||||
auto count_row_and_check_limit = [&, this]()
|
||||
{
|
||||
++total_merged_rows;
|
||||
if (limit && total_merged_rows == limit)
|
||||
@ -174,15 +154,13 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort
|
||||
}
|
||||
|
||||
++merged_rows;
|
||||
stop_condition.addRowWithGranularity(current_granularity);
|
||||
return stop_condition.checkStop();
|
||||
return merged_rows >= max_block_size;
|
||||
};
|
||||
|
||||
/// Take rows in required order and put them into `merged_columns`, while the number of rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
auto current = queue.current();
|
||||
size_t current_block_granularity = current->rows;
|
||||
|
||||
/** And what if the block is totally less or equal than the rest for the current cursor?
|
||||
* Or is there only one data source left in the queue? Then you can take the entire block on current cursor.
|
||||
@ -207,7 +185,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort
|
||||
throw Exception("Logical error in MergingSortedBlockInputStream", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i] = (*std::move(source_blocks[source_num]->getByPosition(i).column)).mutate();
|
||||
merged_columns[i] = (*std::move(source_blocks[source_num].getByPosition(i).column)).mutate();
|
||||
|
||||
// std::cerr << "copied columns\n";
|
||||
|
||||
@ -267,7 +245,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, TSort
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
|
||||
if (count_row_and_check_limit(current_block_granularity))
|
||||
if (count_row_and_check_limit())
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SharedBlockRowRef.h>
|
||||
|
||||
#include <Core/Row.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/SortCursor.h>
|
||||
@ -17,12 +15,6 @@ namespace Poco { class Logger; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CORRUPTED_DATA;
|
||||
}
|
||||
|
||||
|
||||
/** Merges several sorted streams into one sorted stream.
|
||||
*/
|
||||
class MergingSortedBlockInputStream : public IBlockInputStream
|
||||
@ -33,8 +25,8 @@ public:
|
||||
* quiet - don't log profiling info
|
||||
*/
|
||||
MergingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_,
|
||||
UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool average_block_sizes_ = false);
|
||||
const BlockInputStreams & inputs_, SortDescription description_, size_t max_block_size_,
|
||||
UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false);
|
||||
|
||||
String getName() const override { return "MergingSorted"; }
|
||||
|
||||
@ -44,38 +36,6 @@ public:
|
||||
Block getHeader() const override { return header; }
|
||||
|
||||
protected:
|
||||
/// Simple class, which allows to check stop condition during merge process
|
||||
/// in simple case it just compare amount of merged rows with max_block_size
|
||||
/// in `count_average` case it compares amount of merged rows with linear combination
|
||||
/// of block sizes from which these rows were taken.
|
||||
struct MergeStopCondition
|
||||
{
|
||||
size_t sum_blocks_granularity = 0;
|
||||
size_t sum_rows_count = 0;
|
||||
bool count_average;
|
||||
size_t max_block_size;
|
||||
|
||||
MergeStopCondition(bool count_average_, size_t max_block_size_)
|
||||
: count_average(count_average_)
|
||||
, max_block_size(max_block_size_)
|
||||
{}
|
||||
|
||||
/// add single row from block size `granularity`
|
||||
void addRowWithGranularity(size_t granularity)
|
||||
{
|
||||
sum_blocks_granularity += granularity;
|
||||
sum_rows_count++;
|
||||
}
|
||||
|
||||
/// check that sum_rows_count is enough
|
||||
bool checkStop() const;
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return sum_blocks_granularity == 0;
|
||||
}
|
||||
};
|
||||
|
||||
Block readImpl() override;
|
||||
|
||||
void readSuffixImpl() override;
|
||||
@ -87,7 +47,6 @@ protected:
|
||||
template <typename TSortCursor>
|
||||
void fetchNextBlock(const TSortCursor & current, SortingHeap<TSortCursor> & queue);
|
||||
|
||||
|
||||
Block header;
|
||||
|
||||
const SortDescription description;
|
||||
@ -98,14 +57,13 @@ protected:
|
||||
bool first = true;
|
||||
bool has_collation = false;
|
||||
bool quiet = false;
|
||||
bool average_block_sizes = false;
|
||||
|
||||
/// May be smaller or equal to max_block_size. To do 'reserve' for columns.
|
||||
size_t expected_block_size = 0;
|
||||
|
||||
/// Blocks currently being merged.
|
||||
size_t num_columns = 0;
|
||||
std::vector<SharedBlockPtr> source_blocks;
|
||||
Blocks source_blocks;
|
||||
|
||||
SortCursorImpls cursors;
|
||||
|
||||
@ -116,58 +74,6 @@ protected:
|
||||
/// If it is not nullptr then it should be populated during execution
|
||||
WriteBuffer * out_row_sources_buf;
|
||||
|
||||
|
||||
/// These methods are used in Collapsing/Summing/Aggregating... SortedBlockInputStream-s.
|
||||
|
||||
/// Save the row pointed to by cursor in `row`.
|
||||
template <typename TSortCursor>
|
||||
void setRow(Row & row, TSortCursor & cursor)
|
||||
{
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
try
|
||||
{
|
||||
cursor->all_columns[i]->get(cursor->pos, row[i]);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
/// Find out the name of the column and throw more informative exception.
|
||||
|
||||
String column_name;
|
||||
for (const auto & block : source_blocks)
|
||||
{
|
||||
if (i < block->columns())
|
||||
{
|
||||
column_name = block->safeGetByPosition(i).name;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos)
|
||||
+ " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"),
|
||||
ErrorCodes::CORRUPTED_DATA);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <typename TSortCursor>
|
||||
void setRowRef(SharedBlockRowRef & row_ref, TSortCursor & cursor)
|
||||
{
|
||||
row_ref.row_num = cursor.impl->pos;
|
||||
row_ref.shared_block = source_blocks[cursor.impl->order];
|
||||
row_ref.columns = &row_ref.shared_block->all_columns;
|
||||
}
|
||||
|
||||
template <typename TSortCursor>
|
||||
void setPrimaryKeyRef(SharedBlockRowRef & row_ref, TSortCursor & cursor)
|
||||
{
|
||||
row_ref.row_num = cursor.impl->pos;
|
||||
row_ref.shared_block = source_blocks[cursor.impl->order];
|
||||
row_ref.columns = &row_ref.shared_block->sort_columns;
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
/** We support two different cursors - with Collation and without.
|
||||
|
@ -1,116 +0,0 @@
|
||||
#include <DataStreams/ReplacingSortedBlockInputStream.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
void ReplacingSortedBlockInputStream::insertRow(MutableColumns & merged_columns)
|
||||
{
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
/// true flag value means "skip row"
|
||||
current_row_sources[max_pos].setSkipFlag(false);
|
||||
|
||||
out_row_sources_buf->write(reinterpret_cast<const char *>(current_row_sources.data()),
|
||||
current_row_sources.size() * sizeof(RowSourcePart));
|
||||
current_row_sources.resize(0);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*(*selected_row.columns)[i], selected_row.row_num);
|
||||
}
|
||||
|
||||
|
||||
Block ReplacingSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return Block();
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return Block();
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
||||
void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
{
|
||||
MergeStopCondition stop_condition(average_block_sizes, max_block_size);
|
||||
|
||||
/// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
size_t current_block_granularity = current->rows;
|
||||
|
||||
if (current_key.empty())
|
||||
setPrimaryKeyRef(current_key, current);
|
||||
|
||||
setPrimaryKeyRef(next_key, current);
|
||||
|
||||
bool key_differs = next_key != current_key;
|
||||
|
||||
/// if there are enough rows and the last one is calculated completely
|
||||
if (key_differs && stop_condition.checkStop())
|
||||
return;
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
/// Write the data for the previous primary key.
|
||||
insertRow(merged_columns);
|
||||
stop_condition.addRowWithGranularity(current_block_granularity);
|
||||
selected_row.reset();
|
||||
current_key.swap(next_key);
|
||||
}
|
||||
|
||||
/// Initially, skip all rows. Unskip last on insert.
|
||||
size_t current_pos = current_row_sources.size();
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace_back(current.impl->order, true);
|
||||
|
||||
/// A non-strict comparison, since we select the last row for the same version values.
|
||||
if (version_column_number == -1
|
||||
|| selected_row.empty()
|
||||
|| current->all_columns[version_column_number]->compareAt(
|
||||
current->pos, selected_row.row_num,
|
||||
*(*selected_row.columns)[version_column_number],
|
||||
/* nan_direction_hint = */ 1) >= 0)
|
||||
{
|
||||
max_pos = current_pos;
|
||||
setRowRef(selected_row, current);
|
||||
}
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the corresponding source, if there is one.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
}
|
||||
|
||||
/// We will write the data for the last primary key.
|
||||
if (!selected_row.empty())
|
||||
insertRow(merged_columns);
|
||||
|
||||
finished = true;
|
||||
}
|
||||
|
||||
}
|
@ -1,61 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted streams into one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* keeps row with max `version` value.
|
||||
*/
|
||||
class ReplacingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
ReplacingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_,
|
||||
const String & version_column, size_t max_block_size_, WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool average_block_sizes_ = false)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_)
|
||||
{
|
||||
if (!version_column.empty())
|
||||
version_column_number = header.getPositionByName(version_column);
|
||||
}
|
||||
|
||||
String getName() const override { return "ReplacingSorted"; }
|
||||
|
||||
protected:
|
||||
/// Can return 1 more records than max_block_size.
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
ssize_t version_column_number = -1;
|
||||
|
||||
Logger * log = &Logger::get("ReplacingSortedBlockInputStream");
|
||||
|
||||
/// All data has been read.
|
||||
bool finished = false;
|
||||
|
||||
/// Primary key of current row.
|
||||
SharedBlockRowRef current_key;
|
||||
/// Primary key of next row.
|
||||
SharedBlockRowRef next_key;
|
||||
/// Last row with maximum version for current primary key.
|
||||
SharedBlockRowRef selected_row;
|
||||
/// The position (into current_row_sources) of the row with the highest version.
|
||||
size_t max_pos = 0;
|
||||
|
||||
/// Sources of rows with the current primary key.
|
||||
PODArray<RowSourcePart> current_row_sources;
|
||||
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/// Output into result the rows for current primary key.
|
||||
void insertRow(MutableColumns & merged_columns);
|
||||
};
|
||||
|
||||
}
|
@ -1,522 +0,0 @@
|
||||
#include <DataStreams/SummingSortedBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number)
|
||||
{
|
||||
for (auto & desc : description)
|
||||
if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Returns true if merge result is not empty
|
||||
bool mergeMap(const SummingSortedBlockInputStream::MapDescription & desc, Row & row, SortCursor & cursor)
|
||||
{
|
||||
/// Strongly non-optimal.
|
||||
|
||||
Row & left = row;
|
||||
Row right(left.size());
|
||||
|
||||
for (size_t col_num : desc.key_col_nums)
|
||||
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
|
||||
|
||||
for (size_t col_num : desc.val_col_nums)
|
||||
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
|
||||
|
||||
auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field &
|
||||
{
|
||||
return matrix[i].get<Array>()[j];
|
||||
};
|
||||
|
||||
auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array
|
||||
{
|
||||
size_t size = col_nums.size();
|
||||
Array res(size);
|
||||
for (size_t col_num_index = 0; col_num_index < size; ++col_num_index)
|
||||
res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j);
|
||||
return res;
|
||||
};
|
||||
|
||||
std::map<Array, Array> merged;
|
||||
|
||||
auto accumulate = [](Array & dst, const Array & src)
|
||||
{
|
||||
bool has_non_zero = false;
|
||||
size_t size = dst.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (applyVisitor(FieldVisitorSum(src[i]), dst[i]))
|
||||
has_non_zero = true;
|
||||
return has_non_zero;
|
||||
};
|
||||
|
||||
auto merge = [&](const Row & matrix)
|
||||
{
|
||||
size_t rows = matrix[desc.key_col_nums[0]].get<Array>().size();
|
||||
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j);
|
||||
Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j);
|
||||
|
||||
auto it = merged.find(key);
|
||||
if (merged.end() == it)
|
||||
merged.emplace(std::move(key), std::move(value));
|
||||
else
|
||||
{
|
||||
if (!accumulate(it->second, value))
|
||||
merged.erase(it);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
merge(left);
|
||||
merge(right);
|
||||
|
||||
for (size_t col_num : desc.key_col_nums)
|
||||
row[col_num] = Array(merged.size());
|
||||
for (size_t col_num : desc.val_col_nums)
|
||||
row[col_num] = Array(merged.size());
|
||||
|
||||
size_t row_num = 0;
|
||||
for (const auto & key_value : merged)
|
||||
{
|
||||
for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index)
|
||||
row[desc.key_col_nums[col_num_index]].get<Array>()[row_num] = key_value.first[col_num_index];
|
||||
|
||||
for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index)
|
||||
row[desc.val_col_nums[col_num_index]].get<Array>()[row_num] = key_value.second[col_num_index];
|
||||
|
||||
++row_num;
|
||||
}
|
||||
|
||||
return row_num != 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
SummingSortedBlockInputStream::SummingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_,
|
||||
const SortDescription & description_,
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum,
|
||||
size_t max_block_size_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_),
|
||||
log(&Logger::get("SummingSortedBlockInputStream"))
|
||||
{
|
||||
current_row.resize(num_columns);
|
||||
|
||||
/// name of nested structure -> the column numbers that refer to it.
|
||||
std::unordered_map<std::string, std::vector<size_t>> discovered_maps;
|
||||
|
||||
/** Fill in the column numbers, which must be summed.
|
||||
* This can only be numeric columns that are not part of the sort key.
|
||||
* If a non-empty column_names_to_sum is specified, then we only take these columns.
|
||||
* Some columns from column_names_to_sum may not be found. This is ignored.
|
||||
*/
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = header.safeGetByPosition(i);
|
||||
|
||||
/// Discover nested Maps and find columns for summation
|
||||
if (typeid_cast<const DataTypeArray *>(column.type.get()))
|
||||
{
|
||||
const auto map_name = Nested::extractTableName(column.name);
|
||||
/// if nested table name ends with `Map` it is a possible candidate for special handling
|
||||
if (map_name == column.name || !endsWith(map_name, "Map"))
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
discovered_maps[map_name].emplace_back(i);
|
||||
}
|
||||
else
|
||||
{
|
||||
bool is_agg_func = WhichDataType(column.type).isAggregateFunction();
|
||||
|
||||
/// There are special const columns for example after prewere sections.
|
||||
if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column))
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Are they inside the PK?
|
||||
if (isInPrimaryKey(description, column.name, i))
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (column_names_to_sum.empty()
|
||||
|| column_names_to_sum.end() !=
|
||||
std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name))
|
||||
{
|
||||
// Create aggregator to sum this column
|
||||
AggregateDescription desc;
|
||||
desc.is_agg_func_type = is_agg_func;
|
||||
desc.column_numbers = {i};
|
||||
|
||||
if (!is_agg_func)
|
||||
{
|
||||
desc.init("sumWithOverflow", {column.type});
|
||||
}
|
||||
|
||||
columns_to_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// Column is not going to be summed, use last value
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// select actual nested Maps from list of candidates
|
||||
for (const auto & map : discovered_maps)
|
||||
{
|
||||
/// map should contain at least two elements (key -> value)
|
||||
if (map.second.size() < 2)
|
||||
{
|
||||
for (auto col : map.second)
|
||||
column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// no elements of map could be in primary key
|
||||
auto column_num_it = map.second.begin();
|
||||
for (; column_num_it != map.second.end(); ++column_num_it)
|
||||
if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it))
|
||||
break;
|
||||
if (column_num_it != map.second.end())
|
||||
{
|
||||
for (auto col : map.second)
|
||||
column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
DataTypes argument_types;
|
||||
AggregateDescription desc;
|
||||
MapDescription map_desc;
|
||||
|
||||
column_num_it = map.second.begin();
|
||||
for (; column_num_it != map.second.end(); ++column_num_it)
|
||||
{
|
||||
const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it);
|
||||
const String & name = key_col.name;
|
||||
const IDataType & nested_type = *static_cast<const DataTypeArray *>(key_col.type.get())->getNestedType();
|
||||
|
||||
if (column_num_it == map.second.begin()
|
||||
|| endsWith(name, "ID")
|
||||
|| endsWith(name, "Key")
|
||||
|| endsWith(name, "Type"))
|
||||
{
|
||||
if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type))
|
||||
break;
|
||||
|
||||
map_desc.key_col_nums.push_back(*column_num_it);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!nested_type.isSummable())
|
||||
break;
|
||||
|
||||
map_desc.val_col_nums.push_back(*column_num_it);
|
||||
}
|
||||
|
||||
// Add column to function arguments
|
||||
desc.column_numbers.push_back(*column_num_it);
|
||||
argument_types.push_back(key_col.type);
|
||||
}
|
||||
|
||||
if (column_num_it != map.second.end())
|
||||
{
|
||||
for (auto col : map.second)
|
||||
column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (map_desc.key_col_nums.size() == 1)
|
||||
{
|
||||
// Create summation for all value columns in the map
|
||||
desc.init("sumMapWithOverflow", argument_types);
|
||||
columns_to_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// Fall back to legacy mergeMaps for composite keys
|
||||
for (auto col : map.second)
|
||||
column_numbers_not_to_aggregate.push_back(col);
|
||||
maps_to_sum.emplace_back(std::move(map_desc));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns)
|
||||
{
|
||||
/// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate.
|
||||
if (columns_to_aggregate.empty())
|
||||
current_row_is_zero = false;
|
||||
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
// Do not insert if the aggregation state hasn't been created
|
||||
if (desc.created)
|
||||
{
|
||||
if (desc.is_agg_func_type)
|
||||
{
|
||||
current_row_is_zero = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
try
|
||||
{
|
||||
desc.function->insertResultInto(desc.state.data(), *desc.merged_column);
|
||||
|
||||
/// Update zero status of current row
|
||||
if (desc.column_numbers.size() == 1)
|
||||
{
|
||||
// Flag row as non-empty if at least one column number if non-zero
|
||||
current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// It is sumMapWithOverflow aggregate function.
|
||||
/// Assume that the row isn't empty in this case (just because it is compatible with previous version)
|
||||
current_row_is_zero = false;
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
desc.destroyState();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
desc.destroyState();
|
||||
}
|
||||
else
|
||||
desc.merged_column->insertDefault();
|
||||
}
|
||||
|
||||
/// If it is "zero" row, then rollback the insertion
|
||||
/// (at this moment we need rollback only cols from columns_to_aggregate)
|
||||
if (current_row_is_zero)
|
||||
{
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
desc.merged_column->popBack(1);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
for (auto i : column_numbers_not_to_aggregate)
|
||||
merged_columns[i]->insert(current_row[i]);
|
||||
|
||||
/// Update per-block and per-group flags
|
||||
++merged_rows;
|
||||
}
|
||||
|
||||
|
||||
Block SummingSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return Block();
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return {};
|
||||
|
||||
/// Update aggregation result columns for current block
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
// Wrap aggregated columns in a tuple to match function signature
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
MutableColumns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty();
|
||||
|
||||
desc.merged_column = ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
else
|
||||
desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty();
|
||||
}
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
Block res = header.cloneWithColumns(std::move(merged_columns));
|
||||
|
||||
/// Place aggregation results into block.
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
/// Unpack tuple into block.
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
res.getByPosition(desc.column_numbers[i]).column = assert_cast<const ColumnTuple &>(*desc.merged_column).getColumnPtr(i);
|
||||
}
|
||||
else
|
||||
res.getByPosition(desc.column_numbers[0]).column = std::move(desc.merged_column);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
{
|
||||
merged_rows = 0;
|
||||
|
||||
/// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
|
||||
setPrimaryKeyRef(next_key, current);
|
||||
|
||||
bool key_differs;
|
||||
|
||||
if (current_key.empty()) /// The first key encountered.
|
||||
{
|
||||
key_differs = true;
|
||||
current_row_is_zero = true;
|
||||
}
|
||||
else
|
||||
key_differs = next_key != current_key;
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
if (!current_key.empty())
|
||||
/// Write the data for the previous group.
|
||||
insertCurrentRowIfNeeded(merged_columns);
|
||||
|
||||
if (merged_rows >= max_block_size)
|
||||
{
|
||||
/// The block is now full and the last row is calculated completely.
|
||||
current_key.reset();
|
||||
return;
|
||||
}
|
||||
|
||||
current_key.swap(next_key);
|
||||
|
||||
setRow(current_row, current);
|
||||
|
||||
/// Reset aggregation states for next row
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
desc.createState();
|
||||
|
||||
// Start aggregations with current row
|
||||
addRow(current);
|
||||
|
||||
if (maps_to_sum.empty())
|
||||
{
|
||||
/// We have only columns_to_aggregate. The status of current row will be determined
|
||||
/// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions.
|
||||
current_row_is_zero = true; // NOLINT
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have complex maps that will be summed with 'mergeMap' method.
|
||||
/// The single row is considered non zero, and the status after merging with other rows
|
||||
/// will be determined in the branch below (when key_differs == false).
|
||||
current_row_is_zero = false; // NOLINT
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
addRow(current);
|
||||
|
||||
// Merge maps only for same rows
|
||||
for (const auto & desc : maps_to_sum)
|
||||
if (mergeMap(desc, current_row, current))
|
||||
current_row_is_zero = false;
|
||||
}
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the corresponding source, if there is one.
|
||||
fetchNextBlock(current, queue);
|
||||
}
|
||||
}
|
||||
|
||||
/// We will write the data for the last group, if it is non-zero.
|
||||
/// If it is zero, and without it the output stream will be empty, we will write it anyway.
|
||||
insertCurrentRowIfNeeded(merged_columns);
|
||||
finished = true;
|
||||
}
|
||||
|
||||
|
||||
void SummingSortedBlockInputStream::addRow(SortCursor & cursor)
|
||||
{
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
if (!desc.created)
|
||||
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (desc.is_agg_func_type)
|
||||
{
|
||||
// desc.state is not used for AggregateFunction types
|
||||
auto & col = cursor->all_columns[desc.column_numbers[0]];
|
||||
assert_cast<ColumnAggregateFunction &>(*desc.merged_column).insertMergeFrom(*col, cursor->pos);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Specialized case for unary functions
|
||||
if (desc.column_numbers.size() == 1)
|
||||
{
|
||||
auto & col = cursor->all_columns[desc.column_numbers[0]];
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Gather all source columns into a vector
|
||||
ColumnRawPtrs columns(desc.column_numbers.size());
|
||||
for (size_t i = 0; i < desc.column_numbers.size(); ++i)
|
||||
columns[i] = cursor->all_columns[desc.column_numbers[i]];
|
||||
|
||||
desc.add_function(desc.function.get(), desc.state.data(), columns.data(), cursor->pos, nullptr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,155 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <queue>
|
||||
|
||||
#include <Core/Row.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
/** Merges several sorted streams into one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* collapses them into one row, summing all the numeric columns except the primary key.
|
||||
* If in all numeric columns, except for the primary key, the result is zero, it deletes the row.
|
||||
*/
|
||||
class SummingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
SummingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_,
|
||||
const SortDescription & description_,
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum_,
|
||||
size_t max_block_size_);
|
||||
|
||||
String getName() const override { return "SummingSorted"; }
|
||||
|
||||
/// Stores numbers of key-columns and value-columns.
|
||||
struct MapDescription
|
||||
{
|
||||
std::vector<size_t> key_col_nums;
|
||||
std::vector<size_t> val_col_nums;
|
||||
};
|
||||
|
||||
protected:
|
||||
/// Can return 1 more records than max_block_size.
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Read up to the end.
|
||||
bool finished = false;
|
||||
|
||||
/// Columns with which values should be summed.
|
||||
ColumnNumbers column_numbers_not_to_aggregate;
|
||||
|
||||
/** A table can have nested tables that are treated in a special way.
|
||||
* If the name of the nested table ends in `Map` and it contains at least two columns,
|
||||
* satisfying the following criteria:
|
||||
* - the first column, as well as all columns whose names end with `ID`, `Key` or `Type` - numeric ((U)IntN, Date, DateTime);
|
||||
* (a tuple of such columns will be called `keys`)
|
||||
* - the remaining columns are arithmetic ((U)IntN, Float32/64), called (`values`...).
|
||||
* This nested table is treated as a mapping (keys...) => (values...) and when merge
|
||||
* its rows, the merge of the elements of two sets by (keys...) with summing of corresponding (values...).
|
||||
*
|
||||
* Example:
|
||||
* [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)]
|
||||
* [(1, 100)] + [(1, 150)] -> [(1, 250)]
|
||||
* [(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)]
|
||||
* [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)]
|
||||
*
|
||||
* This very unusual functionality is made exclusively for the banner system,
|
||||
* is not supposed for use by anyone else,
|
||||
* and can be deleted at any time.
|
||||
*/
|
||||
|
||||
/// Stores aggregation function, state, and columns to be used as function arguments
|
||||
struct AggregateDescription
|
||||
{
|
||||
/// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing.
|
||||
AggregateFunctionPtr function;
|
||||
IAggregateFunction::AddFunc add_function = nullptr;
|
||||
std::vector<size_t> column_numbers;
|
||||
MutableColumnPtr merged_column;
|
||||
AlignedBuffer state;
|
||||
bool created = false;
|
||||
|
||||
/// In case when column has type AggregateFunction: use the aggregate function from itself instead of 'function' above.
|
||||
bool is_agg_func_type = false;
|
||||
|
||||
void init(const char * function_name, const DataTypes & argument_types)
|
||||
{
|
||||
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
|
||||
add_function = function->getAddressOfAddFunction();
|
||||
state.reset(function->sizeOfData(), function->alignOfData());
|
||||
}
|
||||
|
||||
void createState()
|
||||
{
|
||||
if (created)
|
||||
return;
|
||||
if (is_agg_func_type)
|
||||
merged_column->insertDefault();
|
||||
else
|
||||
function->create(state.data());
|
||||
created = true;
|
||||
}
|
||||
|
||||
void destroyState()
|
||||
{
|
||||
if (!created)
|
||||
return;
|
||||
if (!is_agg_func_type)
|
||||
function->destroy(state.data());
|
||||
created = false;
|
||||
}
|
||||
|
||||
/// Explicitly destroy aggregation state if the stream is terminated
|
||||
~AggregateDescription()
|
||||
{
|
||||
destroyState();
|
||||
}
|
||||
|
||||
AggregateDescription() = default;
|
||||
AggregateDescription(AggregateDescription &&) = default;
|
||||
AggregateDescription(const AggregateDescription &) = delete;
|
||||
};
|
||||
|
||||
std::vector<AggregateDescription> columns_to_aggregate;
|
||||
std::vector<MapDescription> maps_to_sum;
|
||||
|
||||
SharedBlockRowRef current_key; /// The current primary key.
|
||||
SharedBlockRowRef next_key; /// The primary key of the next row.
|
||||
|
||||
Row current_row;
|
||||
bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally.
|
||||
|
||||
size_t merged_rows = 0; /// Number of rows merged into current result block
|
||||
|
||||
/** We support two different cursors - with Collation and without.
|
||||
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
|
||||
*/
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero".
|
||||
void insertCurrentRowIfNeeded(MutableColumns & merged_columns);
|
||||
|
||||
// Add the row under the cursor to the `row`.
|
||||
void addRow(SortCursor & cursor);
|
||||
};
|
||||
|
||||
}
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,181 +0,0 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataStreams/VersionedCollapsingSortedBlockInputStream.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
VersionedCollapsingSortedBlockInputStream::VersionedCollapsingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_,
|
||||
const String & sign_column_, size_t max_block_size_,
|
||||
WriteBuffer * out_row_sources_buf_, bool average_block_sizes_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_, false, average_block_sizes_)
|
||||
, max_rows_in_queue(std::min(std::max<size_t>(3, max_block_size_), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 2)
|
||||
, current_keys(max_rows_in_queue + 1)
|
||||
{
|
||||
sign_column_number = header.getPositionByName(sign_column_);
|
||||
}
|
||||
|
||||
|
||||
inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source)
|
||||
{
|
||||
if constexpr (sizeof(RowSourcePart) == 1)
|
||||
buffer.write(*reinterpret_cast<const char *>(&row_source));
|
||||
else
|
||||
buffer.write(reinterpret_cast<const char *>(&row_source), sizeof(RowSourcePart));
|
||||
}
|
||||
|
||||
void VersionedCollapsingSortedBlockInputStream::insertGap(size_t gap_size)
|
||||
{
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
for (size_t i = 0; i < gap_size; ++i)
|
||||
{
|
||||
writeRowSourcePart(*out_row_sources_buf, current_row_sources.front());
|
||||
current_row_sources.pop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns)
|
||||
{
|
||||
const auto & columns = row.shared_block->all_columns;
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i]->insertFrom(*columns[i], row.row_num);
|
||||
|
||||
insertGap(skip_rows);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
current_row_sources.front().setSkipFlag(false);
|
||||
writeRowSourcePart(*out_row_sources_buf, current_row_sources.front());
|
||||
current_row_sources.pop();
|
||||
}
|
||||
}
|
||||
|
||||
Block VersionedCollapsingSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return {};
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return {};
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
||||
void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
{
|
||||
MergeStopCondition stop_condition(average_block_sizes, max_block_size);
|
||||
|
||||
auto update_queue = [this, & queue](SortCursor & cursor)
|
||||
{
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace(cursor->order, true);
|
||||
|
||||
if (!cursor->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We take next block from the corresponding source, if there is one.
|
||||
fetchNextBlock(cursor, queue);
|
||||
}
|
||||
};
|
||||
|
||||
/// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
size_t current_block_granularity = current->rows;
|
||||
|
||||
SharedBlockRowRef next_key;
|
||||
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
|
||||
setPrimaryKeyRef(next_key, current);
|
||||
|
||||
size_t rows_to_merge = 0;
|
||||
|
||||
/// Each branch either updates queue or increases rows_to_merge.
|
||||
if (current_keys.empty())
|
||||
{
|
||||
sign_in_queue = sign;
|
||||
current_keys.pushBack(next_key);
|
||||
update_queue(current);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (current_keys.back() == next_key)
|
||||
{
|
||||
update_queue(current);
|
||||
|
||||
if (sign == sign_in_queue)
|
||||
current_keys.pushBack(next_key);
|
||||
else
|
||||
{
|
||||
current_keys.popBack();
|
||||
current_keys.pushGap(2);
|
||||
}
|
||||
}
|
||||
else
|
||||
rows_to_merge = current_keys.size();
|
||||
}
|
||||
|
||||
if (current_keys.size() > max_rows_in_queue)
|
||||
rows_to_merge = std::max(rows_to_merge, current_keys.size() - max_rows_in_queue);
|
||||
|
||||
while (rows_to_merge)
|
||||
{
|
||||
const auto & row = current_keys.front();
|
||||
auto gap = current_keys.frontGap();
|
||||
|
||||
insertRow(gap, row, merged_columns);
|
||||
|
||||
current_keys.popFront();
|
||||
|
||||
stop_condition.addRowWithGranularity(current_block_granularity);
|
||||
--rows_to_merge;
|
||||
|
||||
if (stop_condition.checkStop())
|
||||
{
|
||||
++blocks_written;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
while (!current_keys.empty())
|
||||
{
|
||||
const auto & row = current_keys.front();
|
||||
auto gap = current_keys.frontGap();
|
||||
|
||||
insertRow(gap, row, merged_columns);
|
||||
|
||||
current_keys.popFront();
|
||||
}
|
||||
|
||||
/// Write information about last collapsed rows.
|
||||
insertGap(current_keys.frontGap());
|
||||
|
||||
finished = true;
|
||||
}
|
||||
|
||||
}
|
@ -7,7 +7,6 @@
|
||||
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/CollapsingSortedBlockInputStream.h>
|
||||
#include <DataStreams/CollapsingFinalBlockInputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
@ -1,10 +1,13 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/BlocksListBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
@ -27,25 +30,25 @@ static Block getBlockWithSize(const std::vector<std::string> & columns, size_t r
|
||||
}
|
||||
|
||||
|
||||
static BlockInputStreams getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
|
||||
static Pipes getInputStreams(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
|
||||
{
|
||||
BlockInputStreams result;
|
||||
Pipes pipes;
|
||||
for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes)
|
||||
{
|
||||
BlocksList blocks;
|
||||
size_t start = stride;
|
||||
while (blocks_count--)
|
||||
blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start));
|
||||
result.push_back(std::make_shared<BlocksListBlockInputStream>(std::move(blocks)));
|
||||
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<BlocksListBlockInputStream>(std::move(blocks))));
|
||||
}
|
||||
return result;
|
||||
return pipes;
|
||||
|
||||
}
|
||||
|
||||
|
||||
static BlockInputStreams getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
|
||||
static Pipes getInputStreamsEqualStride(const std::vector<std::string> & column_names, const std::vector<std::tuple<size_t, size_t, size_t>> & block_sizes)
|
||||
{
|
||||
BlockInputStreams result;
|
||||
Pipes pipes;
|
||||
size_t i = 0;
|
||||
for (auto [block_size_in_bytes, blocks_count, stride] : block_sizes)
|
||||
{
|
||||
@ -53,10 +56,10 @@ static BlockInputStreams getInputStreamsEqualStride(const std::vector<std::strin
|
||||
size_t start = i;
|
||||
while (blocks_count--)
|
||||
blocks.push_back(getBlockWithSize(column_names, block_size_in_bytes, stride, start));
|
||||
result.push_back(std::make_shared<BlocksListBlockInputStream>(std::move(blocks)));
|
||||
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<BlocksListBlockInputStream>(std::move(blocks))));
|
||||
i++;
|
||||
}
|
||||
return result;
|
||||
return pipes;
|
||||
|
||||
}
|
||||
|
||||
@ -75,18 +78,21 @@ TEST(MergingSortedTest, SimpleBlockSizeTest)
|
||||
{
|
||||
std::vector<std::string> key_columns{"K1", "K2", "K3"};
|
||||
auto sort_description = getSortDescription(key_columns);
|
||||
auto streams = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}});
|
||||
auto pipes = getInputStreams(key_columns, {{5, 1, 1}, {10, 1, 2}, {21, 1, 3}});
|
||||
|
||||
EXPECT_EQ(streams.size(), 3);
|
||||
EXPECT_EQ(pipes.size(), 3);
|
||||
|
||||
MergingSortedBlockInputStream stream(streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipes.front().getHeader(), pipes.size(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
|
||||
auto stream = std::make_shared<TreeExecutorBlockInputStream>(Pipe(std::move(pipes), std::move(transform)));
|
||||
|
||||
size_t total_rows = 0;
|
||||
auto block1 = stream.read();
|
||||
auto block2 = stream.read();
|
||||
auto block3 = stream.read();
|
||||
auto block1 = stream->read();
|
||||
auto block2 = stream->read();
|
||||
auto block3 = stream->read();
|
||||
|
||||
EXPECT_EQ(stream.read(), Block());
|
||||
EXPECT_EQ(stream->read(), Block());
|
||||
|
||||
for (auto & block : {block1, block2, block3})
|
||||
total_rows += block.rows();
|
||||
@ -112,17 +118,20 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes)
|
||||
{
|
||||
std::vector<std::string> key_columns{"K1", "K2", "K3"};
|
||||
auto sort_description = getSortDescription(key_columns);
|
||||
auto streams = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}});
|
||||
auto pipes = getInputStreamsEqualStride(key_columns, {{1000, 1, 3}, {1500, 1, 3}, {1400, 1, 3}});
|
||||
|
||||
EXPECT_EQ(streams.size(), 3);
|
||||
EXPECT_EQ(pipes.size(), 3);
|
||||
|
||||
MergingSortedBlockInputStream stream(streams, sort_description, DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipes.front().getHeader(), pipes.size(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
|
||||
auto block1 = stream.read();
|
||||
auto block2 = stream.read();
|
||||
auto block3 = stream.read();
|
||||
auto stream = std::make_shared<TreeExecutorBlockInputStream>(Pipe(std::move(pipes), std::move(transform)));
|
||||
|
||||
EXPECT_EQ(stream.read(), Block());
|
||||
auto block1 = stream->read();
|
||||
auto block2 = stream->read();
|
||||
auto block3 = stream->read();
|
||||
|
||||
EXPECT_EQ(stream->read(), Block());
|
||||
|
||||
EXPECT_EQ(block1.rows(), (1000 + 1500 + 1400) / 3);
|
||||
EXPECT_EQ(block2.rows(), (1000 + 1500 + 1400) / 3);
|
||||
|
@ -72,6 +72,7 @@
|
||||
#include <ext/scope_guard.h>
|
||||
#include <memory>
|
||||
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
@ -84,7 +85,6 @@
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
||||
#include <Processors/Transforms/MergeSortingTransform.h>
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <Processors/Transforms/DistinctTransform.h>
|
||||
#include <Processors/Transforms/LimitByTransform.h>
|
||||
#include <Processors/Transforms/CreatingSetsTransform.h>
|
||||
|
30
src/Processors/Merges/AggregatingSortedTransform.h
Normal file
30
src/Processors/Merges/AggregatingSortedTransform.h
Normal file
@ -0,0 +1,30 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ColumnAggregateFunction;
|
||||
|
||||
/// Implementation of IMergingTransform via AggregatingSortedAlgorithm.
|
||||
class AggregatingSortedTransform final : public IMergingTransform<AggregatingSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
AggregatingSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
max_block_size)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "AggregatingSortedTransform"; }
|
||||
};
|
||||
|
||||
}
|
353
src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp
Normal file
353
src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp
Normal file
@ -0,0 +1,353 @@
|
||||
#include <Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h>
|
||||
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default;
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) noexcept = default;
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default;
|
||||
|
||||
/// Stores information for aggregation of AggregateFunction columns
|
||||
struct AggregatingSortedAlgorithm::AggregateDescription
|
||||
{
|
||||
ColumnAggregateFunction * column = nullptr;
|
||||
const size_t column_number = 0; /// Position in header.
|
||||
|
||||
AggregateDescription() = default;
|
||||
explicit AggregateDescription(size_t col_number) : column_number(col_number) {}
|
||||
};
|
||||
|
||||
/// Stores information for aggregation of SimpleAggregateFunction columns
|
||||
struct AggregatingSortedAlgorithm::SimpleAggregateDescription
|
||||
{
|
||||
/// An aggregate function 'anyLast', 'sum'...
|
||||
AggregateFunctionPtr function;
|
||||
IAggregateFunction::AddFunc add_function = nullptr;
|
||||
|
||||
size_t column_number = 0;
|
||||
IColumn * column = nullptr;
|
||||
|
||||
/// For LowCardinality, convert is converted to nested type. nested_type is nullptr if no conversion needed.
|
||||
const DataTypePtr nested_type; /// Nested type for LowCardinality, if it is.
|
||||
const DataTypePtr real_type; /// Type in header.
|
||||
|
||||
AlignedBuffer state;
|
||||
bool created = false;
|
||||
|
||||
SimpleAggregateDescription(
|
||||
AggregateFunctionPtr function_, const size_t column_number_,
|
||||
DataTypePtr nested_type_, DataTypePtr real_type_)
|
||||
: function(std::move(function_)), column_number(column_number_)
|
||||
, nested_type(std::move(nested_type_)), real_type(std::move(real_type_))
|
||||
{
|
||||
add_function = function->getAddressOfAddFunction();
|
||||
state.reset(function->sizeOfData(), function->alignOfData());
|
||||
}
|
||||
|
||||
void createState()
|
||||
{
|
||||
if (created)
|
||||
return;
|
||||
function->create(state.data());
|
||||
created = true;
|
||||
}
|
||||
|
||||
void destroyState()
|
||||
{
|
||||
if (!created)
|
||||
return;
|
||||
function->destroy(state.data());
|
||||
created = false;
|
||||
}
|
||||
|
||||
/// Explicitly destroy aggregation state if the stream is terminated
|
||||
~SimpleAggregateDescription()
|
||||
{
|
||||
destroyState();
|
||||
}
|
||||
|
||||
SimpleAggregateDescription() = default;
|
||||
SimpleAggregateDescription(SimpleAggregateDescription &&) = default;
|
||||
SimpleAggregateDescription(const SimpleAggregateDescription &) = delete;
|
||||
};
|
||||
|
||||
static AggregatingSortedAlgorithm::ColumnsDefinition defineColumns(
|
||||
const Block & header, const SortDescription & description)
|
||||
{
|
||||
AggregatingSortedAlgorithm::ColumnsDefinition def = {};
|
||||
size_t num_columns = header.columns();
|
||||
|
||||
/// Fill in the column numbers that need to be aggregated.
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = header.safeGetByPosition(i);
|
||||
|
||||
/// We leave only states of aggregate functions.
|
||||
if (!dynamic_cast<const DataTypeAggregateFunction *>(column.type.get())
|
||||
&& !dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
|
||||
{
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Included into PK?
|
||||
auto it = description.begin();
|
||||
for (; it != description.end(); ++it)
|
||||
if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i))
|
||||
break;
|
||||
|
||||
if (it != description.end())
|
||||
{
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (auto simple = dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(column.type->getCustomName()))
|
||||
{
|
||||
auto type = recursiveRemoveLowCardinality(column.type);
|
||||
if (type.get() == column.type.get())
|
||||
type = nullptr;
|
||||
|
||||
// simple aggregate function
|
||||
AggregatingSortedAlgorithm::SimpleAggregateDescription desc(simple->getFunction(), i, type, column.type);
|
||||
if (desc.function->allocatesMemoryInArena())
|
||||
def.allocates_memory_in_arena = true;
|
||||
|
||||
def.columns_to_simple_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// standard aggregate function
|
||||
def.columns_to_aggregate.emplace_back(i);
|
||||
}
|
||||
}
|
||||
|
||||
return def;
|
||||
}
|
||||
|
||||
static MutableColumns getMergedColumns(const Block & header, const AggregatingSortedAlgorithm::ColumnsDefinition & def)
|
||||
{
|
||||
MutableColumns columns;
|
||||
columns.resize(header.columns());
|
||||
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
{
|
||||
auto & type = desc.nested_type ? desc.nested_type
|
||||
: desc.real_type;
|
||||
columns[desc.column_number] = type->createColumn();
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
if (!columns[i])
|
||||
columns[i] = header.getByPosition(i).type->createColumn();
|
||||
|
||||
return columns;
|
||||
}
|
||||
|
||||
/// Remove constants and LowCardinality for SimpleAggregateFunction
|
||||
static void preprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
if (desc.nested_type)
|
||||
columns[desc.column_number] = recursiveRemoveLowCardinality(columns[desc.column_number]);
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
/// Return back LowCardinality for SimpleAggregateFunction
|
||||
static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::ColumnsDefinition & def)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
{
|
||||
if (desc.nested_type)
|
||||
{
|
||||
auto & from_type = desc.nested_type;
|
||||
auto & to_type = desc.real_type;
|
||||
columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type);
|
||||
}
|
||||
}
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
|
||||
AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData(
|
||||
MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_), def(def_)
|
||||
{
|
||||
initAggregateDescription();
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::AggregatingMergedData::startGroup(const ColumnRawPtrs & raw_columns, size_t row)
|
||||
{
|
||||
/// We will write the data for the group. We copy the values of ordinary columns.
|
||||
for (auto column_number : def.column_numbers_not_to_aggregate)
|
||||
columns[column_number]->insertFrom(*raw_columns[column_number], row);
|
||||
|
||||
/// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function.
|
||||
for (auto & column_to_aggregate : def.columns_to_aggregate)
|
||||
column_to_aggregate.column->insertDefault();
|
||||
|
||||
/// Reset simple aggregation states for next row
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
desc.createState();
|
||||
|
||||
if (def.allocates_memory_in_arena)
|
||||
arena = std::make_unique<Arena>();
|
||||
|
||||
is_group_started = true;
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::AggregatingMergedData::finishGroup()
|
||||
{
|
||||
/// Write the simple aggregation result for the current group.
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
{
|
||||
desc.function->insertResultInto(desc.state.data(), *desc.column);
|
||||
desc.destroyState();
|
||||
}
|
||||
|
||||
is_group_started = false;
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
/// TODO: sum_blocks_granularity += block_size;
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::AggregatingMergedData::addRow(SortCursor & cursor)
|
||||
{
|
||||
if (!is_group_started)
|
||||
throw Exception("Can't add a row to the group because it was not started.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
desc.column->insertMergeFrom(*cursor->all_columns[desc.column_number], cursor->pos);
|
||||
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
{
|
||||
auto & col = cursor->all_columns[desc.column_number];
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, arena.get());
|
||||
}
|
||||
}
|
||||
|
||||
Chunk AggregatingSortedAlgorithm::AggregatingMergedData::pull()
|
||||
{
|
||||
if (is_group_started)
|
||||
throw Exception("Can't pull chunk because group was not finished.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto chunk = MergedData::pull();
|
||||
postprocessChunk(chunk, def);
|
||||
|
||||
initAggregateDescription();
|
||||
|
||||
return chunk;
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::AggregatingMergedData::initAggregateDescription()
|
||||
{
|
||||
for (auto & desc : def.columns_to_simple_aggregate)
|
||||
desc.column = columns[desc.column_number].get();
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
desc.column = typeid_cast<ColumnAggregateFunction *>(columns[desc.column_number].get());
|
||||
}
|
||||
|
||||
|
||||
AggregatingSortedAlgorithm::AggregatingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size)
|
||||
: IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_))
|
||||
, columns_definition(defineColumns(header, description_))
|
||||
, merged_data(getMergedColumns(header, columns_definition), max_block_size, columns_definition)
|
||||
{
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::initialize(Chunks chunks)
|
||||
{
|
||||
for (auto & chunk : chunks)
|
||||
if (chunk)
|
||||
preprocessChunk(chunk, columns_definition);
|
||||
|
||||
initializeQueue(std::move(chunks));
|
||||
}
|
||||
|
||||
void AggregatingSortedAlgorithm::consume(Chunk chunk, size_t source_num)
|
||||
{
|
||||
preprocessChunk(chunk, columns_definition);
|
||||
updateCursor(std::move(chunk), source_num);
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status AggregatingSortedAlgorithm::merge()
|
||||
{
|
||||
/// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
bool key_differs;
|
||||
SortCursor current = queue.current();
|
||||
|
||||
{
|
||||
detail::RowRef current_key;
|
||||
current_key.set(current);
|
||||
|
||||
key_differs = last_key.empty() || !last_key.hasEqualSortColumnsWith(current_key);
|
||||
|
||||
last_key = current_key;
|
||||
last_chunk_sort_columns.clear();
|
||||
}
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
if (merged_data.isGroupStarted())
|
||||
merged_data.finishGroup();
|
||||
|
||||
/// if there are enough rows accumulated and the last one is calculated completely
|
||||
if (merged_data.hasEnoughRows())
|
||||
{
|
||||
last_key.reset();
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
merged_data.startGroup(current->all_columns, current->pos);
|
||||
}
|
||||
|
||||
merged_data.addRow(current);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
/// Write the simple aggregation result for the previous group.
|
||||
if (merged_data.isGroupStarted())
|
||||
merged_data.finishGroup();
|
||||
|
||||
last_chunk_sort_columns.clear();
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs to one.
|
||||
* During this for each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* merges them into one row. When merging, the data is pre-aggregated - merge of states of aggregate functions,
|
||||
* corresponding to a one value of the primary key. For columns that are not part of the primary key and which do not have the AggregateFunction type,
|
||||
* when merged, the first value is selected.
|
||||
*/
|
||||
class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk
|
||||
{
|
||||
public:
|
||||
AggregatingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size);
|
||||
|
||||
void initialize(Chunks chunks) override;
|
||||
void consume(Chunk chunk, size_t source_num) override;
|
||||
Status merge() override;
|
||||
|
||||
struct SimpleAggregateDescription;
|
||||
struct AggregateDescription;
|
||||
|
||||
/// This structure define columns into one of three types:
|
||||
/// * columns which are not aggregate functions and not needed to be aggregated
|
||||
/// * usual aggregate functions, which stores states into ColumnAggregateFunction
|
||||
/// * simple aggregate functions, which store states into ordinary columns
|
||||
struct ColumnsDefinition
|
||||
{
|
||||
ColumnsDefinition(); /// Is needed because destructor is defined.
|
||||
ColumnsDefinition(ColumnsDefinition &&) noexcept; /// Is needed because destructor is defined.
|
||||
~ColumnsDefinition(); /// Is needed because otherwise std::vector's destructor uses incomplete types.
|
||||
|
||||
/// Columns with which numbers should not be aggregated.
|
||||
ColumnNumbers column_numbers_not_to_aggregate;
|
||||
std::vector<AggregateDescription> columns_to_aggregate;
|
||||
std::vector<SimpleAggregateDescription> columns_to_simple_aggregate;
|
||||
|
||||
/// Does SimpleAggregateFunction allocates memory in arena?
|
||||
bool allocates_memory_in_arena = false;
|
||||
};
|
||||
|
||||
private:
|
||||
/// Specialization for AggregatingSortedAlgorithm.
|
||||
struct AggregatingMergedData : public MergedData
|
||||
{
|
||||
private:
|
||||
using MergedData::pull;
|
||||
using MergedData::insertRow;
|
||||
|
||||
public:
|
||||
AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_);
|
||||
|
||||
/// Group is a group of rows with the same sorting key. It represents single row in result.
|
||||
/// Algorithm is: start group, add several rows, finish group.
|
||||
/// Then pull chunk when enough groups were added.
|
||||
void startGroup(const ColumnRawPtrs & raw_columns, size_t row);
|
||||
void finishGroup();
|
||||
|
||||
bool isGroupStarted() const { return is_group_started; }
|
||||
void addRow(SortCursor & cursor); /// Possible only when group was started.
|
||||
|
||||
Chunk pull(); /// Possible only if group was finished.
|
||||
|
||||
private:
|
||||
ColumnsDefinition & def;
|
||||
|
||||
/// Memory pool for SimpleAggregateFunction
|
||||
/// (only when allocates_memory_in_arena == true).
|
||||
std::unique_ptr<Arena> arena;
|
||||
|
||||
bool is_group_started = false;
|
||||
|
||||
/// Initialize aggregate descriptions with columns.
|
||||
void initAggregateDescription();
|
||||
};
|
||||
|
||||
/// Order between members is important because merged_data has reference to columns_definition.
|
||||
ColumnsDefinition columns_definition;
|
||||
AggregatingMergedData merged_data;
|
||||
};
|
||||
|
||||
}
|
193
src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp
Normal file
193
src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp
Normal file
@ -0,0 +1,193 @@
|
||||
#include <Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
/// Maximum number of messages about incorrect data in the log.
|
||||
#define MAX_ERROR_MESSAGES 10
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DATA;
|
||||
}
|
||||
|
||||
CollapsingSortedAlgorithm::CollapsingSortedAlgorithm(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes,
|
||||
Logger * log_)
|
||||
: IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs)
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, sign_column_number(header.getPositionByName(sign_column))
|
||||
, log(log_)
|
||||
{
|
||||
}
|
||||
|
||||
void CollapsingSortedAlgorithm::reportIncorrectData()
|
||||
{
|
||||
if (!log)
|
||||
return;
|
||||
|
||||
std::stringstream s;
|
||||
s << "Incorrect data: number of rows with sign = 1 (" << count_positive
|
||||
<< ") differs with number of rows with sign = -1 (" << count_negative
|
||||
<< ") by more than one (for key: ";
|
||||
|
||||
auto & sort_columns = *last_row.sort_columns;
|
||||
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
s << ", ";
|
||||
s << applyVisitor(FieldVisitorToString(), (*sort_columns[i])[last_row.row_num]);
|
||||
}
|
||||
|
||||
s << ").";
|
||||
|
||||
/** Fow now we limit ourselves to just logging such situations,
|
||||
* since the data is generated by external programs.
|
||||
* With inconsistent data, this is an unavoidable error that can not be easily corrected by admins. Therefore Warning.
|
||||
*/
|
||||
LOG_WARNING(log, s.rdbuf());
|
||||
}
|
||||
|
||||
void CollapsingSortedAlgorithm::insertRow(RowRef & row)
|
||||
{
|
||||
merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows());
|
||||
}
|
||||
|
||||
void CollapsingSortedAlgorithm::insertRows()
|
||||
{
|
||||
if (count_positive == 0 && count_negative == 0)
|
||||
{
|
||||
/// No input rows have been read.
|
||||
return;
|
||||
}
|
||||
|
||||
if (last_is_positive || count_positive != count_negative)
|
||||
{
|
||||
if (count_positive <= count_negative)
|
||||
{
|
||||
insertRow(first_negative_row);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources[first_negative_pos].setSkipFlag(false);
|
||||
}
|
||||
|
||||
if (count_positive >= count_negative)
|
||||
{
|
||||
insertRow(last_positive_row);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources[last_positive_pos].setSkipFlag(false);
|
||||
}
|
||||
|
||||
if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1))
|
||||
{
|
||||
if (count_incorrect_data < MAX_ERROR_MESSAGES)
|
||||
reportIncorrectData();
|
||||
++count_incorrect_data;
|
||||
}
|
||||
}
|
||||
|
||||
first_negative_row.clear();
|
||||
last_positive_row.clear();
|
||||
|
||||
if (out_row_sources_buf)
|
||||
out_row_sources_buf->write(
|
||||
reinterpret_cast<const char *>(current_row_sources.data()),
|
||||
current_row_sources.size() * sizeof(RowSourcePart));
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge()
|
||||
{
|
||||
/// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
auto current = queue.current();
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
|
||||
RowRef current_row;
|
||||
setRowRef(current_row, current);
|
||||
|
||||
if (last_row.empty())
|
||||
setRowRef(last_row, current);
|
||||
|
||||
bool key_differs = !last_row.hasEqualSortColumnsWith(current_row);
|
||||
|
||||
/// if there are enough rows and the last one is calculated completely
|
||||
if (key_differs && merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
/// We write data for the previous primary key.
|
||||
insertRows();
|
||||
|
||||
current_row.swap(last_row);
|
||||
|
||||
count_negative = 0;
|
||||
count_positive = 0;
|
||||
|
||||
current_pos = 0;
|
||||
first_negative_pos = 0;
|
||||
last_positive_pos = 0;
|
||||
current_row_sources.resize(0);
|
||||
}
|
||||
|
||||
/// Initially, skip all rows. On insert, unskip "corner" rows.
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace_back(current.impl->order, true);
|
||||
|
||||
if (sign == 1)
|
||||
{
|
||||
++count_positive;
|
||||
last_is_positive = true;
|
||||
|
||||
setRowRef(last_positive_row, current);
|
||||
last_positive_pos = current_pos;
|
||||
}
|
||||
else if (sign == -1)
|
||||
{
|
||||
if (!count_negative)
|
||||
{
|
||||
setRowRef(first_negative_row, current);
|
||||
first_negative_pos = current_pos;
|
||||
}
|
||||
|
||||
++count_negative;
|
||||
last_is_positive = false;
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect data: Sign = " + toString(sign) + " (must be 1 or -1).",
|
||||
ErrorCodes::INCORRECT_DATA);
|
||||
|
||||
++current_pos;
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We take next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
insertRows();
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
}
|
71
src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h
Normal file
71
src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h
Normal file
@ -0,0 +1,71 @@
|
||||
#pragma once
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs to one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* keeps no more than one row with the value of the column `sign_column = -1` ("negative row")
|
||||
* and no more than a row with the value of the column `sign_column = 1` ("positive row").
|
||||
* That is, it collapses the records from the change log.
|
||||
*
|
||||
* If the number of positive and negative rows is the same, and the last row is positive, then the first negative and last positive rows are written.
|
||||
* If the number of positive and negative rows is the same, and the last line is negative, it writes nothing.
|
||||
* If the positive by 1 is greater than the negative rows, then only the last positive row is written.
|
||||
* If negative by 1 is greater than positive rows, then only the first negative row is written.
|
||||
* Otherwise, a logical error.
|
||||
*/
|
||||
class CollapsingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks
|
||||
{
|
||||
public:
|
||||
CollapsingSortedAlgorithm(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes,
|
||||
Logger * log_);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
private:
|
||||
MergedData merged_data;
|
||||
|
||||
const size_t sign_column_number;
|
||||
|
||||
static constexpr size_t max_row_refs = 4; /// first_negative, last_positive, last, current.
|
||||
RowRef first_negative_row;
|
||||
RowRef last_positive_row;
|
||||
RowRef last_row;
|
||||
|
||||
size_t count_positive = 0; /// The number of positive rows for the current primary key.
|
||||
size_t count_negative = 0; /// The number of negative rows for the current primary key.
|
||||
bool last_is_positive = false; /// true if the last row for the current primary key is positive.
|
||||
|
||||
/// Fields specific for VERTICAL merge algorithm.
|
||||
/// Row numbers are relative to the start of current primary key.
|
||||
size_t current_pos = 0; /// Current row number
|
||||
size_t first_negative_pos = 0; /// Row number of first_negative
|
||||
size_t last_positive_pos = 0; /// Row number of last_positive
|
||||
PODArray<RowSourcePart> current_row_sources; /// Sources of rows with the current primary key
|
||||
|
||||
size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log.
|
||||
Logger * log;
|
||||
|
||||
void reportIncorrectData();
|
||||
void insertRow(RowRef & row);
|
||||
void insertRows();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,24 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
#include <queue>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192;
|
||||
|
||||
|
||||
/* Deque with fixed memory size. Allows pushing gaps.
|
||||
* frontGap() returns the number of gaps were inserted before front.
|
||||
*
|
||||
@ -154,7 +143,7 @@ private:
|
||||
{
|
||||
if (size() + 1 == container.size())
|
||||
throw Exception("Not enough space to insert into FixedSizeDequeWithGaps with capacity "
|
||||
+ toString(container.size() - 1), ErrorCodes::LOGICAL_ERROR);
|
||||
+ std::to_string(container.size() - 1), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void checkHasValuesToRemove() const
|
||||
@ -170,46 +159,4 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
class VersionedCollapsingSortedBlockInputStream : public MergingSortedBlockInputStream
|
||||
{
|
||||
public:
|
||||
/// Don't need version column. It's in primary key.
|
||||
/// max_rows_in_queue should be about max_block_size_ if we won't store a lot of extra blocks (RowRef holds SharedBlockPtr).
|
||||
VersionedCollapsingSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_,
|
||||
const String & sign_column_, size_t max_block_size_,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr, bool average_block_sizes_ = false);
|
||||
|
||||
String getName() const override { return "VersionedCollapsingSorted"; }
|
||||
|
||||
protected:
|
||||
/// Can return 1 more records than max_block_size.
|
||||
Block readImpl() override;
|
||||
|
||||
private:
|
||||
size_t sign_column_number = 0;
|
||||
|
||||
Logger * log = &Logger::get("VersionedCollapsingSortedBlockInputStream");
|
||||
|
||||
/// Read is finished.
|
||||
bool finished = false;
|
||||
|
||||
Int8 sign_in_queue = 0;
|
||||
const size_t max_rows_in_queue;
|
||||
/// Rows with the same primary key and sign.
|
||||
FixedSizeDequeWithGaps<SharedBlockRowRef> current_keys;
|
||||
|
||||
size_t blocks_written = 0;
|
||||
|
||||
/// Sources of rows for VERTICAL merge algorithm. Size equals to (size + number of gaps) in current_keys.
|
||||
std::queue<RowSourcePart> current_row_sources;
|
||||
|
||||
void merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue);
|
||||
|
||||
/// Output to result row for the current primary key.
|
||||
void insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns);
|
||||
|
||||
void insertGap(size_t gap_size);
|
||||
};
|
||||
|
||||
}
|
133
src/Processors/Merges/Algorithms/Graphite.h
Normal file
133
src/Processors/Merges/Algorithms/Graphite.h
Normal file
@ -0,0 +1,133 @@
|
||||
#pragma once
|
||||
#include <Common/OptimizedRegularExpression.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
|
||||
}
|
||||
|
||||
/** Intended for implementation of "rollup" - aggregation (rounding) of older data
|
||||
* for a table with Graphite data (Graphite is the system for time series monitoring).
|
||||
*
|
||||
* Table with graphite data has at least the following columns (accurate to the name):
|
||||
* Path, Time, Value, Version
|
||||
*
|
||||
* Path - name of metric (sensor);
|
||||
* Time - time of measurement;
|
||||
* Value - value of measurement;
|
||||
* Version - a number, that for equal pairs of Path and Time, need to leave only record with maximum version.
|
||||
*
|
||||
* Each row in a table correspond to one value of one sensor.
|
||||
*
|
||||
* Pattern should contain function, retention scheme, or both of them. The order of patterns does mean as well:
|
||||
* * Aggregation OR retention patterns should be first
|
||||
* * Then aggregation AND retention full patterns have to be placed
|
||||
* * default pattern without regexp must be the last
|
||||
*
|
||||
* Rollup rules are specified in the following way:
|
||||
*
|
||||
* pattern
|
||||
* regexp
|
||||
* function
|
||||
* pattern
|
||||
* regexp
|
||||
* age -> precision
|
||||
* age -> precision
|
||||
* ...
|
||||
* pattern
|
||||
* regexp
|
||||
* function
|
||||
* age -> precision
|
||||
* age -> precision
|
||||
* ...
|
||||
* pattern
|
||||
* ...
|
||||
* default
|
||||
* function
|
||||
* age -> precision
|
||||
* ...
|
||||
*
|
||||
* regexp - pattern for sensor name
|
||||
* default - if no pattern has matched
|
||||
*
|
||||
* age - minimal data age (in seconds), to start rounding with specified precision.
|
||||
* precision - rounding precision (in seconds)
|
||||
*
|
||||
* function - name of aggregate function to be applied for values, that time was rounded to same.
|
||||
*
|
||||
* Example:
|
||||
*
|
||||
* <graphite_rollup>
|
||||
* <pattern>
|
||||
* <regexp>\.max$</regexp>
|
||||
* <function>max</function>
|
||||
* </pattern>
|
||||
* <pattern>
|
||||
* <regexp>click_cost</regexp>
|
||||
* <function>any</function>
|
||||
* <retention>
|
||||
* <age>0</age>
|
||||
* <precision>5</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>86400</age>
|
||||
* <precision>60</precision>
|
||||
* </retention>
|
||||
* </pattern>
|
||||
* <default>
|
||||
* <function>max</function>
|
||||
* <retention>
|
||||
* <age>0</age>
|
||||
* <precision>60</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>3600</age>
|
||||
* <precision>300</precision>
|
||||
* </retention>
|
||||
* <retention>
|
||||
* <age>86400</age>
|
||||
* <precision>3600</precision>
|
||||
* </retention>
|
||||
* </default>
|
||||
* </graphite_rollup>
|
||||
*/
|
||||
namespace DB::Graphite
|
||||
{
|
||||
|
||||
struct Retention
|
||||
{
|
||||
UInt32 age;
|
||||
UInt32 precision;
|
||||
};
|
||||
|
||||
using Retentions = std::vector<Retention>;
|
||||
|
||||
struct Pattern
|
||||
{
|
||||
std::shared_ptr<OptimizedRegularExpression> regexp;
|
||||
std::string regexp_str;
|
||||
AggregateFunctionPtr function;
|
||||
Retentions retentions; /// Must be ordered by 'age' descending.
|
||||
enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically
|
||||
};
|
||||
|
||||
using Patterns = std::vector<Pattern>;
|
||||
using RetentionPattern = Pattern;
|
||||
using AggregationPattern = Pattern;
|
||||
|
||||
struct Params
|
||||
{
|
||||
String config_name;
|
||||
String path_column_name;
|
||||
String time_column_name;
|
||||
String value_column_name;
|
||||
String version_column_name;
|
||||
Graphite::Patterns patterns;
|
||||
};
|
||||
|
||||
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
|
||||
|
||||
}
|
@ -1,21 +1,36 @@
|
||||
#include <DataStreams/GraphiteRollupSortedBlockInputStream.h>
|
||||
#include <type_traits>
|
||||
|
||||
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <common/DateLUTImpl.h>
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
static GraphiteRollupSortedAlgorithm::ColumnsDefinition defineColumns(
|
||||
const Block & header, const Graphite::Params & params)
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
GraphiteRollupSortedAlgorithm::ColumnsDefinition def;
|
||||
|
||||
def.path_column_num = header.getPositionByName(params.path_column_name);
|
||||
def.time_column_num = header.getPositionByName(params.time_column_name);
|
||||
def.value_column_num = header.getPositionByName(params.value_column_name);
|
||||
def.version_column_num = header.getPositionByName(params.version_column_name);
|
||||
|
||||
size_t num_columns = header.columns();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
if (i != def.time_column_num && i != def.value_column_num && i != def.version_column_num)
|
||||
def.unmodified_column_numbers.push_back(i);
|
||||
|
||||
return def;
|
||||
}
|
||||
|
||||
|
||||
GraphiteRollupSortedBlockInputStream::GraphiteRollupSortedBlockInputStream(
|
||||
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_,
|
||||
const Graphite::Params & params_, time_t time_of_merge_)
|
||||
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_),
|
||||
params(params_), time_of_merge(time_of_merge_)
|
||||
GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_)
|
||||
: IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), nullptr, max_row_refs)
|
||||
, merged_data(header.cloneEmptyColumns(), false, max_block_size)
|
||||
, params(std::move(params_)), time_of_merge(time_of_merge_)
|
||||
{
|
||||
size_t max_size_of_aggregate_state = 0;
|
||||
size_t max_alignment_of_aggregate_state = 1;
|
||||
@ -29,21 +44,11 @@ GraphiteRollupSortedBlockInputStream::GraphiteRollupSortedBlockInputStream(
|
||||
}
|
||||
}
|
||||
|
||||
place_for_aggregate_state.reset(max_size_of_aggregate_state, max_alignment_of_aggregate_state);
|
||||
|
||||
/// Memoize column numbers in block.
|
||||
path_column_num = header.getPositionByName(params.path_column_name);
|
||||
time_column_num = header.getPositionByName(params.time_column_name);
|
||||
value_column_num = header.getPositionByName(params.value_column_name);
|
||||
version_column_num = header.getPositionByName(params.version_column_name);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
if (i != time_column_num && i != value_column_num && i != version_column_num)
|
||||
unmodified_column_numbers.push_back(i);
|
||||
merged_data.allocMemForAggregates(max_size_of_aggregate_state, max_alignment_of_aggregate_state);
|
||||
columns_definition = defineColumns(header, params);
|
||||
}
|
||||
|
||||
|
||||
Graphite::RollupRule GraphiteRollupSortedBlockInputStream::selectPatternForPath(StringRef path) const
|
||||
Graphite::RollupRule GraphiteRollupSortedAlgorithm::selectPatternForPath(StringRef path) const
|
||||
{
|
||||
const Graphite::Pattern * first_match = &undef_pattern;
|
||||
|
||||
@ -74,8 +79,8 @@ Graphite::RollupRule GraphiteRollupSortedBlockInputStream::selectPatternForPath(
|
||||
/// General pattern with matched path
|
||||
if (pattern.type == pattern.TypeAll)
|
||||
{
|
||||
/// Only for not default patterns with both function and retention parameters
|
||||
return std::pair(&pattern, &pattern);
|
||||
/// Only for not default patterns with both function and retention parameters
|
||||
return std::pair(&pattern, &pattern);
|
||||
}
|
||||
if (first_match->type == first_match->TypeUndef)
|
||||
{
|
||||
@ -99,8 +104,7 @@ Graphite::RollupRule GraphiteRollupSortedBlockInputStream::selectPatternForPath(
|
||||
return {nullptr, nullptr};
|
||||
}
|
||||
|
||||
|
||||
UInt32 GraphiteRollupSortedBlockInputStream::selectPrecision(const Graphite::Retentions & retentions, time_t time) const
|
||||
UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions & retentions, time_t time) const
|
||||
{
|
||||
static_assert(is_signed_v<time_t>, "time_t must be signed type");
|
||||
|
||||
@ -114,7 +118,6 @@ UInt32 GraphiteRollupSortedBlockInputStream::selectPrecision(const Graphite::Ret
|
||||
return 1;
|
||||
}
|
||||
|
||||
|
||||
/** Round the unix timestamp to seconds precision.
|
||||
* In this case, the date should not change. The date is calculated using the local time zone.
|
||||
*
|
||||
@ -140,33 +143,11 @@ static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UI
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Block GraphiteRollupSortedBlockInputStream::readImpl()
|
||||
{
|
||||
if (finished)
|
||||
return Block();
|
||||
|
||||
MutableColumns merged_columns;
|
||||
init(merged_columns);
|
||||
|
||||
if (has_collation)
|
||||
throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (merged_columns.empty())
|
||||
return Block();
|
||||
|
||||
merge(merged_columns, queue_without_collation);
|
||||
return header.cloneWithColumns(std::move(merged_columns));
|
||||
}
|
||||
|
||||
|
||||
void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns, SortingHeap<SortCursor> & queue)
|
||||
IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
|
||||
{
|
||||
const DateLUTImpl & date_lut = DateLUT::instance();
|
||||
|
||||
size_t started_rows = 0; /// Number of times startNextRow() has been called.
|
||||
|
||||
/// Take rows in needed order and put them into `merged_columns` until we get `max_block_size` rows.
|
||||
/// Take rows in needed order and put them into `merged_data` until we get `max_block_size` rows.
|
||||
///
|
||||
/// Variables starting with current_* refer to the rows previously popped from the queue that will
|
||||
/// contribute towards current output row.
|
||||
@ -174,24 +155,24 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns
|
||||
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor next_cursor = queue.current();
|
||||
SortCursor current = queue.current();
|
||||
|
||||
StringRef next_path = next_cursor->all_columns[path_column_num]->getDataAt(next_cursor->pos);
|
||||
StringRef next_path = current->all_columns[columns_definition.path_column_num]->getDataAt(current->pos);
|
||||
bool new_path = is_first || next_path != current_group_path;
|
||||
|
||||
is_first = false;
|
||||
|
||||
time_t next_row_time = next_cursor->all_columns[time_column_num]->getUInt(next_cursor->pos);
|
||||
time_t next_row_time = current->all_columns[columns_definition.time_column_num]->getUInt(current->pos);
|
||||
/// Is new key before rounding.
|
||||
bool is_new_key = new_path || next_row_time != current_time;
|
||||
|
||||
if (is_new_key)
|
||||
{
|
||||
/// Accumulate the row that has maximum version in the previous group of rows with the same key:
|
||||
if (started_rows)
|
||||
if (merged_data.wasGroupStarted())
|
||||
accumulateRow(current_subgroup_newest_row);
|
||||
|
||||
Graphite::RollupRule next_rule = current_rule;
|
||||
Graphite::RollupRule next_rule = merged_data.currentRule();
|
||||
if (new_path)
|
||||
next_rule = selectPatternForPath(next_path);
|
||||
|
||||
@ -213,22 +194,21 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns
|
||||
|
||||
if (will_be_new_key)
|
||||
{
|
||||
if (started_rows)
|
||||
if (merged_data.wasGroupStarted())
|
||||
{
|
||||
finishCurrentGroup(merged_columns);
|
||||
finishCurrentGroup();
|
||||
|
||||
/// We have enough rows - return, but don't advance the loop. At the beginning of the
|
||||
/// next call to merge() the same next_cursor will be processed once more and
|
||||
/// the next output row will be created from it.
|
||||
if (started_rows >= max_block_size)
|
||||
return;
|
||||
if (merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
/// At this point previous row has been fully processed, so we can advance the loop
|
||||
/// (substitute current_* values for next_*, advance the cursor).
|
||||
|
||||
startNextGroup(merged_columns, next_cursor, next_rule);
|
||||
++started_rows;
|
||||
startNextGroup(current, next_rule);
|
||||
|
||||
current_time_rounded = next_time_rounded;
|
||||
}
|
||||
@ -239,12 +219,12 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns
|
||||
/// Within all rows with same key, we should leave only one row with maximum version;
|
||||
/// and for rows with same maximum version - only last row.
|
||||
if (is_new_key
|
||||
|| next_cursor->all_columns[version_column_num]->compareAt(
|
||||
next_cursor->pos, current_subgroup_newest_row.row_num,
|
||||
*(*current_subgroup_newest_row.columns)[version_column_num],
|
||||
|| current->all_columns[columns_definition.version_column_num]->compareAt(
|
||||
current->pos, current_subgroup_newest_row.row_num,
|
||||
*(*current_subgroup_newest_row.all_columns)[columns_definition.version_column_num],
|
||||
/* nan_direction_hint = */ 1) >= 0)
|
||||
{
|
||||
setRowRef(current_subgroup_newest_row, next_cursor);
|
||||
current_subgroup_newest_row.set(current, source_chunks[current.impl->order]);
|
||||
|
||||
/// Small hack: group and subgroups have the same path, so we can set current_group_path here instead of startNextGroup
|
||||
/// But since we keep in memory current_subgroup_newest_row's block, we could use StringRef for current_group_path and don't
|
||||
@ -252,37 +232,52 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns
|
||||
current_group_path = next_path;
|
||||
}
|
||||
|
||||
if (!next_cursor->isLast())
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the appropriate source, if there is one.
|
||||
fetchNextBlock(next_cursor, queue);
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
/// Write result row for the last group.
|
||||
if (started_rows)
|
||||
if (merged_data.wasGroupStarted())
|
||||
{
|
||||
accumulateRow(current_subgroup_newest_row);
|
||||
finishCurrentGroup(merged_columns);
|
||||
finishCurrentGroup();
|
||||
}
|
||||
|
||||
finished = true;
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
void GraphiteRollupSortedAlgorithm::startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule)
|
||||
{
|
||||
merged_data.startNextGroup(cursor->all_columns, cursor->pos, next_rule, columns_definition);
|
||||
}
|
||||
|
||||
template <typename TSortCursor>
|
||||
void GraphiteRollupSortedBlockInputStream::startNextGroup(MutableColumns & merged_columns, TSortCursor & cursor,
|
||||
Graphite::RollupRule next_rule)
|
||||
void GraphiteRollupSortedAlgorithm::finishCurrentGroup()
|
||||
{
|
||||
merged_data.insertRow(current_time_rounded, current_subgroup_newest_row, columns_definition);
|
||||
}
|
||||
|
||||
void GraphiteRollupSortedAlgorithm::accumulateRow(RowRef & row)
|
||||
{
|
||||
merged_data.accumulateRow(row, columns_definition);
|
||||
}
|
||||
|
||||
void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::startNextGroup(
|
||||
const ColumnRawPtrs & raw_columns, size_t row,
|
||||
Graphite::RollupRule next_rule, ColumnsDefinition & def)
|
||||
{
|
||||
const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(next_rule);
|
||||
|
||||
/// Copy unmodified column values (including path column).
|
||||
for (size_t j : unmodified_column_numbers)
|
||||
merged_columns[j]->insertFrom(*cursor->all_columns[j], cursor->pos);
|
||||
for (size_t j : def.unmodified_column_numbers)
|
||||
columns[j]->insertFrom(*raw_columns[j], row);
|
||||
|
||||
if (aggregation_pattern)
|
||||
{
|
||||
@ -291,34 +286,49 @@ void GraphiteRollupSortedBlockInputStream::startNextGroup(MutableColumns & merge
|
||||
}
|
||||
|
||||
current_rule = next_rule;
|
||||
was_group_started = true;
|
||||
}
|
||||
|
||||
|
||||
void GraphiteRollupSortedBlockInputStream::finishCurrentGroup(MutableColumns & merged_columns)
|
||||
void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::insertRow(
|
||||
time_t time, RowRef & row, ColumnsDefinition & def)
|
||||
{
|
||||
/// Insert calculated values of the columns `time`, `value`, `version`.
|
||||
merged_columns[time_column_num]->insert(current_time_rounded);
|
||||
merged_columns[version_column_num]->insertFrom(
|
||||
*(*current_subgroup_newest_row.columns)[version_column_num], current_subgroup_newest_row.row_num);
|
||||
columns[def.time_column_num]->insert(time);
|
||||
auto & row_ref_version_column = (*row.all_columns)[def.version_column_num];
|
||||
columns[def.version_column_num]->insertFrom(*row_ref_version_column, row.row_num);
|
||||
|
||||
auto & value_column = columns[def.value_column_num];
|
||||
const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule);
|
||||
if (aggregate_state_created)
|
||||
{
|
||||
aggregation_pattern->function->insertResultInto(place_for_aggregate_state.data(), *merged_columns[value_column_num]);
|
||||
aggregation_pattern->function->insertResultInto(place_for_aggregate_state.data(), *value_column);
|
||||
aggregation_pattern->function->destroy(place_for_aggregate_state.data());
|
||||
aggregate_state_created = false;
|
||||
}
|
||||
else
|
||||
merged_columns[value_column_num]->insertFrom(
|
||||
*(*current_subgroup_newest_row.columns)[value_column_num], current_subgroup_newest_row.row_num);
|
||||
value_column->insertFrom(*(*row.all_columns)[def.value_column_num], row.row_num);
|
||||
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
/// TODO: sum_blocks_granularity += block_size;
|
||||
|
||||
was_group_started = false;
|
||||
}
|
||||
|
||||
|
||||
void GraphiteRollupSortedBlockInputStream::accumulateRow(SharedBlockRowRef & row)
|
||||
void GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::accumulateRow(RowRef & row, ColumnsDefinition & def)
|
||||
{
|
||||
const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule);
|
||||
if (aggregate_state_created)
|
||||
aggregation_pattern->function->add(place_for_aggregate_state.data(), &(*row.columns)[value_column_num], row.row_num, nullptr);
|
||||
{
|
||||
auto & column = (*row.all_columns)[def.value_column_num];
|
||||
aggregation_pattern->function->add(place_for_aggregate_state.data(), &column, row.row_num, nullptr);
|
||||
}
|
||||
}
|
||||
|
||||
GraphiteRollupSortedAlgorithm::GraphiteRollupMergedData::~GraphiteRollupMergedData()
|
||||
{
|
||||
if (aggregate_state_created)
|
||||
std::get<1>(current_rule)->function->destroy(place_for_aggregate_state.data());
|
||||
}
|
||||
|
||||
}
|
125
src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h
Normal file
125
src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h
Normal file
@ -0,0 +1,125 @@
|
||||
#pragma once
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h>
|
||||
#include <Processors/Merges/Algorithms/Graphite.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs into one.
|
||||
*
|
||||
* For each group of consecutive identical values of the `path` column,
|
||||
* and the same `time` values, rounded to some precision
|
||||
* (where rounding accuracy depends on the template set for `path`
|
||||
* and the amount of time elapsed from `time` to the specified time),
|
||||
* keeps one line,
|
||||
* performing the rounding of time,
|
||||
* merge `value` values using the specified aggregate functions,
|
||||
* as well as keeping the maximum value of the `version` column.
|
||||
*/
|
||||
class GraphiteRollupSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks
|
||||
{
|
||||
public:
|
||||
GraphiteRollupSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
struct ColumnsDefinition
|
||||
{
|
||||
size_t path_column_num;
|
||||
size_t time_column_num;
|
||||
size_t value_column_num;
|
||||
size_t version_column_num;
|
||||
|
||||
/// All columns other than 'time', 'value', 'version'. They are unmodified during rollup.
|
||||
ColumnNumbers unmodified_column_numbers;
|
||||
};
|
||||
|
||||
using RowRef = detail::RowRefWithOwnedChunk;
|
||||
|
||||
/// Specialization for SummingSortedTransform.
|
||||
class GraphiteRollupMergedData : public MergedData
|
||||
{
|
||||
public:
|
||||
using MergedData::MergedData;
|
||||
~GraphiteRollupMergedData();
|
||||
|
||||
void startNextGroup(const ColumnRawPtrs & raw_columns, size_t row,
|
||||
Graphite::RollupRule next_rule, ColumnsDefinition & def);
|
||||
void insertRow(time_t time, RowRef & row, ColumnsDefinition & def);
|
||||
void accumulateRow(RowRef & row, ColumnsDefinition & def);
|
||||
bool wasGroupStarted() const { return was_group_started; }
|
||||
|
||||
const Graphite::RollupRule & currentRule() const { return current_rule; }
|
||||
void allocMemForAggregates(size_t size, size_t alignment) { place_for_aggregate_state.reset(size, alignment); }
|
||||
|
||||
private:
|
||||
Graphite::RollupRule current_rule = {nullptr, nullptr};
|
||||
AlignedBuffer place_for_aggregate_state;
|
||||
bool aggregate_state_created = false; /// Invariant: if true then current_rule is not NULL.
|
||||
bool was_group_started = false;
|
||||
};
|
||||
|
||||
private:
|
||||
GraphiteRollupMergedData merged_data;
|
||||
|
||||
const Graphite::Params params;
|
||||
ColumnsDefinition columns_definition;
|
||||
|
||||
time_t time_of_merge;
|
||||
|
||||
/// No data has been read.
|
||||
bool is_first = true;
|
||||
|
||||
/* | path | time | rounded_time | version | value | unmodified |
|
||||
* -----------------------------------------------------------------------------------
|
||||
* | A | 11 | 10 | 1 | 1 | a | |
|
||||
* | A | 11 | 10 | 3 | 2 | b |> subgroup(A, 11) |
|
||||
* | A | 11 | 10 | 2 | 3 | c | |> group(A, 10)
|
||||
* ----------------------------------------------------------------------------------|>
|
||||
* | A | 12 | 10 | 0 | 4 | d | |> Outputs (A, 10, avg(2, 5), a)
|
||||
* | A | 12 | 10 | 1 | 5 | e |> subgroup(A, 12) |
|
||||
* -----------------------------------------------------------------------------------
|
||||
* | A | 21 | 20 | 1 | 6 | f |
|
||||
* | B | 11 | 10 | 1 | 7 | g |
|
||||
* ...
|
||||
*/
|
||||
|
||||
/// Path name of current bucket
|
||||
StringRef current_group_path;
|
||||
|
||||
static constexpr size_t max_row_refs = 2; /// current_subgroup_newest_row, current_row.
|
||||
/// Last row with maximum version for current primary key (time bucket).
|
||||
RowRef current_subgroup_newest_row;
|
||||
|
||||
/// Time of last read row
|
||||
time_t current_time = 0;
|
||||
time_t current_time_rounded = 0;
|
||||
|
||||
const Graphite::Pattern undef_pattern =
|
||||
{ /// temporary empty pattern for selectPatternForPath
|
||||
.regexp = nullptr,
|
||||
.regexp_str = "",
|
||||
.function = nullptr,
|
||||
.retentions = DB::Graphite::Retentions(),
|
||||
.type = undef_pattern.TypeUndef,
|
||||
};
|
||||
|
||||
Graphite::RollupRule selectPatternForPath(StringRef path) const;
|
||||
UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const;
|
||||
|
||||
/// Insert the values into the resulting columns, which will not be changed in the future.
|
||||
void startNextGroup(SortCursor & cursor, Graphite::RollupRule next_rule);
|
||||
|
||||
/// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows.
|
||||
void finishCurrentGroup();
|
||||
|
||||
/// Update the state of the aggregate function with the new `value`.
|
||||
void accumulateRow(RowRef & row);
|
||||
};
|
||||
|
||||
}
|
35
src/Processors/Merges/Algorithms/IMergingAlgorithm.h
Normal file
35
src/Processors/Merges/Algorithms/IMergingAlgorithm.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Chunk.h>
|
||||
#include <variant>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergingAlgorithm
|
||||
{
|
||||
public:
|
||||
struct Status
|
||||
{
|
||||
Chunk chunk;
|
||||
bool is_finished = false;
|
||||
ssize_t required_source = -1;
|
||||
|
||||
explicit Status(Chunk chunk_) : chunk(std::move(chunk_)) {}
|
||||
explicit Status(Chunk chunk_, bool is_finished_) : chunk(std::move(chunk_)), is_finished(is_finished_) {}
|
||||
explicit Status(size_t source) : required_source(source) {}
|
||||
};
|
||||
|
||||
virtual void initialize(Chunks chunks) = 0;
|
||||
virtual void consume(Chunk chunk, size_t source_num) = 0;
|
||||
virtual Status merge() = 0;
|
||||
|
||||
IMergingAlgorithm() = default;
|
||||
virtual ~IMergingAlgorithm() = default;
|
||||
};
|
||||
|
||||
// TODO: use when compile with clang which could support it
|
||||
// template <class T>
|
||||
// concept MergingAlgorithm = std::is_base_of<IMergingAlgorithm, T>::value;
|
||||
|
||||
}
|
@ -0,0 +1,45 @@
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IMergingAlgorithmWithDelayedChunk::IMergingAlgorithmWithDelayedChunk(
|
||||
size_t num_inputs,
|
||||
SortDescription description_)
|
||||
: description(std::move(description_))
|
||||
, source_chunks(num_inputs)
|
||||
, cursors(num_inputs)
|
||||
{
|
||||
}
|
||||
|
||||
void IMergingAlgorithmWithDelayedChunk::initializeQueue(Chunks chunks)
|
||||
{
|
||||
source_chunks = std::move(chunks);
|
||||
|
||||
for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num)
|
||||
{
|
||||
if (!source_chunks[source_num])
|
||||
continue;
|
||||
|
||||
cursors[source_num] = SortCursorImpl(source_chunks[source_num].getColumns(), description, source_num);
|
||||
}
|
||||
|
||||
queue = SortingHeap<SortCursor>(cursors);
|
||||
}
|
||||
|
||||
void IMergingAlgorithmWithDelayedChunk::updateCursor(Chunk chunk, size_t source_num)
|
||||
{
|
||||
auto & source_chunk = source_chunks[source_num];
|
||||
|
||||
/// Extend lifetime of last chunk.
|
||||
last_chunk = std::move(source_chunk);
|
||||
last_chunk_sort_columns = std::move(cursors[source_num].sort_columns);
|
||||
|
||||
source_chunk = std::move(chunk);
|
||||
cursors[source_num].reset(source_chunk.getColumns(), {});
|
||||
|
||||
queue.push(cursors[source_num]);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/RowRef.h>
|
||||
#include <Core/SortDescription.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergingAlgorithmWithDelayedChunk : public IMergingAlgorithm
|
||||
{
|
||||
public:
|
||||
IMergingAlgorithmWithDelayedChunk(
|
||||
size_t num_inputs,
|
||||
SortDescription description_);
|
||||
|
||||
protected:
|
||||
SortingHeap<SortCursor> queue;
|
||||
SortDescription description;
|
||||
|
||||
/// Previous row. May refer to last_chunk_sort_columns or row from source_chunks.
|
||||
detail::RowRef last_key;
|
||||
|
||||
ColumnRawPtrs last_chunk_sort_columns; /// Point to last_chunk if valid.
|
||||
|
||||
void initializeQueue(Chunks chunks);
|
||||
void updateCursor(Chunk chunk, size_t source_num);
|
||||
|
||||
private:
|
||||
/// Chunks currently being merged.
|
||||
std::vector<Chunk> source_chunks;
|
||||
SortCursorImpls cursors;
|
||||
|
||||
/// In merging algorithm, we need to compare current sort key with the last one.
|
||||
/// So, sorting columns for last row needed to be stored.
|
||||
/// In order to do it, we extend lifetime of last chunk and it's sort columns (from corresponding sort cursor).
|
||||
Chunk last_chunk;
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IMergingAlgorithmWithSharedChunks::IMergingAlgorithmWithSharedChunks(
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
size_t max_row_refs)
|
||||
: description(std::move(description_))
|
||||
, chunk_allocator(num_inputs + max_row_refs)
|
||||
, cursors(num_inputs)
|
||||
, source_chunks(num_inputs)
|
||||
, out_row_sources_buf(out_row_sources_buf_)
|
||||
{
|
||||
}
|
||||
|
||||
static void prepareChunk(Chunk & chunk)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
void IMergingAlgorithmWithSharedChunks::initialize(Chunks chunks)
|
||||
{
|
||||
source_chunks.resize(chunks.size());
|
||||
|
||||
for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num)
|
||||
{
|
||||
if (!chunks[source_num])
|
||||
continue;
|
||||
|
||||
prepareChunk(chunks[source_num]);
|
||||
|
||||
auto & source_chunk = source_chunks[source_num];
|
||||
|
||||
source_chunk = chunk_allocator.alloc(std::move(chunks[source_num]));
|
||||
cursors[source_num] = SortCursorImpl(source_chunk->getColumns(), description, source_num);
|
||||
|
||||
source_chunk->all_columns = cursors[source_num].all_columns;
|
||||
source_chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
}
|
||||
|
||||
queue = SortingHeap<SortCursor>(cursors);
|
||||
}
|
||||
|
||||
void IMergingAlgorithmWithSharedChunks::consume(Chunk chunk, size_t source_num)
|
||||
{
|
||||
prepareChunk(chunk);
|
||||
|
||||
auto & source_chunk = source_chunks[source_num];
|
||||
source_chunk = chunk_allocator.alloc(std::move(chunk));
|
||||
cursors[source_num].reset(source_chunk->getColumns(), {});
|
||||
|
||||
source_chunk->all_columns = cursors[source_num].all_columns;
|
||||
source_chunk->sort_columns = cursors[source_num].sort_columns;
|
||||
|
||||
queue.push(cursors[source_num]);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,44 @@
|
||||
#pragma once
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/RowRef.h>
|
||||
#include <Core/SortDescription.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergingAlgorithmWithSharedChunks : public IMergingAlgorithm
|
||||
{
|
||||
public:
|
||||
IMergingAlgorithmWithSharedChunks(
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
size_t max_row_refs);
|
||||
|
||||
void initialize(Chunks chunks) override;
|
||||
void consume(Chunk chunk, size_t source_num) override;
|
||||
|
||||
private:
|
||||
SortDescription description;
|
||||
|
||||
/// Allocator must be destroyed after source_chunks.
|
||||
detail::SharedChunkAllocator chunk_allocator;
|
||||
|
||||
SortCursorImpls cursors;
|
||||
|
||||
protected:
|
||||
/// Chunks currently being merged.
|
||||
using SourceChunks = std::vector<detail::SharedChunkPtr>;
|
||||
SourceChunks source_chunks;
|
||||
|
||||
SortingHeap<SortCursor> queue;
|
||||
|
||||
/// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step)
|
||||
/// If it is not nullptr then it should be populated during execution
|
||||
WriteBuffer * out_row_sources_buf = nullptr;
|
||||
|
||||
using RowRef = detail::RowRefWithOwnedChunk;
|
||||
void setRowRef(RowRef & row, SortCursor & cursor) { row.set(cursor, source_chunks[cursor.impl->order]); }
|
||||
};
|
||||
|
||||
}
|
120
src/Processors/Merges/Algorithms/MergedData.h
Normal file
120
src/Processors/Merges/Algorithms/MergedData.h
Normal file
@ -0,0 +1,120 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// Class which represents current merging chunk of data.
|
||||
/// Also it calculates the number of merged rows and other profile info.
|
||||
class MergedData
|
||||
{
|
||||
public:
|
||||
explicit MergedData(MutableColumns columns_, bool use_average_block_size_, UInt64 max_block_size_)
|
||||
: columns(std::move(columns_)), max_block_size(max_block_size_), use_average_block_size(use_average_block_size_)
|
||||
{
|
||||
}
|
||||
|
||||
/// Pull will be called at next prepare call.
|
||||
void flush() { need_flush = true; }
|
||||
|
||||
void insertRow(const ColumnRawPtrs & raw_columns, size_t row, size_t block_size)
|
||||
{
|
||||
size_t num_columns = raw_columns.size();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
columns[i]->insertFrom(*raw_columns[i], row);
|
||||
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
sum_blocks_granularity += block_size;
|
||||
}
|
||||
|
||||
void insertFromChunk(Chunk && chunk, size_t limit_rows)
|
||||
{
|
||||
if (merged_rows)
|
||||
throw Exception("Cannot insert to MergedData from Chunk because MergedData is not empty.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto num_rows = chunk.getNumRows();
|
||||
columns = chunk.mutateColumns();
|
||||
if (limit_rows && num_rows > limit_rows)
|
||||
{
|
||||
num_rows = limit_rows;
|
||||
for (auto & column : columns)
|
||||
column = (*column->cut(0, num_rows)).mutate();
|
||||
}
|
||||
|
||||
need_flush = true;
|
||||
total_merged_rows += num_rows;
|
||||
merged_rows = num_rows;
|
||||
|
||||
/// We don't cate about granularity here. Because, for fast-forward optimization, chunk will be moved as-is.
|
||||
/// sum_blocks_granularity += block_size * num_rows;
|
||||
}
|
||||
|
||||
Chunk pull()
|
||||
{
|
||||
MutableColumns empty_columns;
|
||||
empty_columns.reserve(columns.size());
|
||||
|
||||
for (const auto & column : columns)
|
||||
empty_columns.emplace_back(column->cloneEmpty());
|
||||
|
||||
empty_columns.swap(columns);
|
||||
Chunk chunk(std::move(empty_columns), merged_rows);
|
||||
|
||||
merged_rows = 0;
|
||||
sum_blocks_granularity = 0;
|
||||
++total_chunks;
|
||||
total_allocated_bytes += chunk.allocatedBytes();
|
||||
need_flush = false;
|
||||
|
||||
return chunk;
|
||||
}
|
||||
|
||||
bool hasEnoughRows() const
|
||||
{
|
||||
/// If full chunk was or is going to be inserted, then we must pull it.
|
||||
/// It is needed for fast-forward optimization.
|
||||
if (need_flush)
|
||||
return true;
|
||||
|
||||
/// Never return more then max_block_size.
|
||||
if (merged_rows >= max_block_size)
|
||||
return true;
|
||||
|
||||
if (!use_average_block_size)
|
||||
return false;
|
||||
|
||||
/// Zero rows always not enough.
|
||||
if (merged_rows == 0)
|
||||
return false;
|
||||
|
||||
size_t average = sum_blocks_granularity / merged_rows;
|
||||
return merged_rows >= average;
|
||||
}
|
||||
|
||||
UInt64 mergedRows() const { return merged_rows; }
|
||||
UInt64 totalMergedRows() const { return total_merged_rows; }
|
||||
UInt64 totalChunks() const { return total_chunks; }
|
||||
UInt64 totalAllocatedBytes() const { return total_allocated_bytes; }
|
||||
|
||||
protected:
|
||||
MutableColumns columns;
|
||||
|
||||
UInt64 sum_blocks_granularity = 0;
|
||||
UInt64 merged_rows = 0;
|
||||
UInt64 total_merged_rows = 0;
|
||||
UInt64 total_chunks = 0;
|
||||
UInt64 total_allocated_bytes = 0;
|
||||
|
||||
const UInt64 max_block_size;
|
||||
const bool use_average_block_size;
|
||||
|
||||
bool need_flush = false;
|
||||
};
|
||||
|
||||
}
|
202
src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp
Normal file
202
src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp
Normal file
@ -0,0 +1,202 @@
|
||||
#include <Processors/Merges/Algorithms/MergingSortedAlgorithm.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, description(std::move(description_))
|
||||
, limit(limit_)
|
||||
, out_row_sources_buf(out_row_sources_buf_)
|
||||
, source_chunks(num_inputs)
|
||||
, cursors(num_inputs)
|
||||
{
|
||||
/// Replace column names in description to positions.
|
||||
for (auto & column_description : description)
|
||||
{
|
||||
has_collation |= column_description.collator != nullptr;
|
||||
if (!column_description.column_name.empty())
|
||||
{
|
||||
column_description.column_number = header.getPositionByName(column_description.column_name);
|
||||
column_description.column_name.clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MergingSortedAlgorithm::addInput()
|
||||
{
|
||||
source_chunks.emplace_back();
|
||||
cursors.emplace_back();
|
||||
}
|
||||
|
||||
static void prepareChunk(Chunk & chunk)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
void MergingSortedAlgorithm::initialize(Chunks chunks)
|
||||
{
|
||||
source_chunks = std::move(chunks);
|
||||
|
||||
for (size_t source_num = 0; source_num < source_chunks.size(); ++source_num)
|
||||
{
|
||||
auto & chunk = source_chunks[source_num];
|
||||
|
||||
if (!chunk)
|
||||
continue;
|
||||
|
||||
prepareChunk(chunk);
|
||||
cursors[source_num] = SortCursorImpl(chunk.getColumns(), description, source_num);
|
||||
}
|
||||
|
||||
if (has_collation)
|
||||
queue_with_collation = SortingHeap<SortCursorWithCollation>(cursors);
|
||||
else
|
||||
queue_without_collation = SortingHeap<SortCursor>(cursors);
|
||||
}
|
||||
|
||||
void MergingSortedAlgorithm::consume(Chunk chunk, size_t source_num)
|
||||
{
|
||||
prepareChunk(chunk);
|
||||
source_chunks[source_num] = std::move(chunk);
|
||||
cursors[source_num].reset(source_chunks[source_num].getColumns(), {});
|
||||
|
||||
if (has_collation)
|
||||
queue_with_collation.push(cursors[source_num]);
|
||||
else
|
||||
queue_without_collation.push(cursors[source_num]);
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status MergingSortedAlgorithm::merge()
|
||||
{
|
||||
if (has_collation)
|
||||
return mergeImpl(queue_with_collation);
|
||||
else
|
||||
return mergeImpl(queue_without_collation);
|
||||
}
|
||||
|
||||
template <typename TSortingHeap>
|
||||
IMergingAlgorithm::Status MergingSortedAlgorithm::mergeImpl(TSortingHeap & queue)
|
||||
{
|
||||
/// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
if (merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
|
||||
auto current = queue.current();
|
||||
|
||||
/** And what if the block is totally less or equal than the rest for the current cursor?
|
||||
* Or is there only one data source left in the queue? Then you can take the entire block on current cursor.
|
||||
*/
|
||||
if (current.impl->isFirst()
|
||||
&& (queue.size() == 1
|
||||
|| (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild()))))
|
||||
{
|
||||
//std::cerr << "current block is totally less or equals\n";
|
||||
|
||||
/// If there are already data in the current block, we first return it.
|
||||
/// We'll get here again the next time we call the merge function.
|
||||
if (merged_data.mergedRows() != 0)
|
||||
{
|
||||
//std::cerr << "merged rows is non-zero\n";
|
||||
// merged_data.flush();
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
/// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl)
|
||||
size_t source_num = current.impl->order;
|
||||
queue.removeTop();
|
||||
return insertFromChunk(source_num);
|
||||
}
|
||||
|
||||
//std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n";
|
||||
//std::cerr << "Inserting row\n";
|
||||
merged_data.insertRow(current->all_columns, current->pos, current->rows);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
/// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl)
|
||||
RowSourcePart row_source(current.impl->order);
|
||||
out_row_sources_buf->write(row_source.data);
|
||||
}
|
||||
|
||||
if (limit && merged_data.totalMergedRows() >= limit)
|
||||
return Status(merged_data.pull(), true);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
//std::cerr << "moving to next row\n";
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We will get the next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
//std::cerr << "It was last row, fetching next block\n";
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status MergingSortedAlgorithm::insertFromChunk(size_t source_num)
|
||||
{
|
||||
if (source_num >= cursors.size())
|
||||
throw Exception("Logical error in MergingSortedTransform", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
//std::cerr << "copied columns\n";
|
||||
|
||||
auto num_rows = source_chunks[source_num].getNumRows();
|
||||
|
||||
UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows;
|
||||
bool is_finished = limit && total_merged_rows_after_insertion >= limit;
|
||||
|
||||
if (limit && total_merged_rows_after_insertion > limit)
|
||||
{
|
||||
num_rows = total_merged_rows_after_insertion - limit;
|
||||
merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows);
|
||||
}
|
||||
else
|
||||
merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0);
|
||||
|
||||
source_chunks[source_num] = Chunk();
|
||||
|
||||
/// Write order of rows for other columns
|
||||
/// this data will be used in gather stream
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
RowSourcePart row_source(source_num);
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
out_row_sources_buf->write(row_source.data);
|
||||
}
|
||||
|
||||
auto status = Status(merged_data.pull(), is_finished);
|
||||
|
||||
if (!is_finished)
|
||||
status.required_source = source_num;
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
}
|
57
src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h
Normal file
57
src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h
Normal file
@ -0,0 +1,57 @@
|
||||
#pragma once
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/SortCursor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Merges several sorted inputs into one sorted output.
|
||||
class MergingSortedAlgorithm final : public IMergingAlgorithm
|
||||
{
|
||||
public:
|
||||
MergingSortedAlgorithm(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes);
|
||||
|
||||
void addInput();
|
||||
|
||||
void initialize(Chunks chunks) override;
|
||||
void consume(Chunk chunk, size_t source_num) override;
|
||||
Status merge() override;
|
||||
|
||||
const MergedData & getMergedData() const { return merged_data; }
|
||||
|
||||
private:
|
||||
MergedData merged_data;
|
||||
|
||||
/// Settings
|
||||
SortDescription description;
|
||||
UInt64 limit;
|
||||
bool has_collation = false;
|
||||
|
||||
/// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step)
|
||||
/// If it is not nullptr then it should be populated during execution
|
||||
WriteBuffer * out_row_sources_buf = nullptr;
|
||||
|
||||
/// Chunks currently being merged.
|
||||
std::vector<Chunk> source_chunks;
|
||||
|
||||
SortCursorImpls cursors;
|
||||
|
||||
SortingHeap<SortCursor> queue_without_collation;
|
||||
SortingHeap<SortCursorWithCollation> queue_with_collation;
|
||||
|
||||
Status insertFromChunk(size_t source_num);
|
||||
|
||||
template <typename TSortingHeap>
|
||||
Status mergeImpl(TSortingHeap & queue);
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,96 @@
|
||||
#include <Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ReplacingSortedAlgorithm::ReplacingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & version_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs)
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
{
|
||||
if (!version_column.empty())
|
||||
version_column_number = header.getPositionByName(version_column);
|
||||
}
|
||||
|
||||
void ReplacingSortedAlgorithm::insertRow()
|
||||
{
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
/// true flag value means "skip row"
|
||||
current_row_sources[max_pos].setSkipFlag(false);
|
||||
|
||||
out_row_sources_buf->write(reinterpret_cast<const char *>(current_row_sources.data()),
|
||||
current_row_sources.size() * sizeof(RowSourcePart));
|
||||
current_row_sources.resize(0);
|
||||
}
|
||||
|
||||
merged_data.insertRow(*selected_row.all_columns, selected_row.row_num, selected_row.owned_chunk->getNumRows());
|
||||
selected_row.clear();
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status ReplacingSortedAlgorithm::merge()
|
||||
{
|
||||
/// Take the rows in needed order and put them into `merged_columns` until rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
|
||||
RowRef current_row;
|
||||
setRowRef(current_row, current);
|
||||
|
||||
bool key_differs = selected_row.empty() || !current_row.hasEqualSortColumnsWith(selected_row);
|
||||
if (key_differs)
|
||||
{
|
||||
/// if there are enough rows and the last one is calculated completely
|
||||
if (merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
|
||||
/// Write the data for the previous primary key.
|
||||
if (!selected_row.empty())
|
||||
insertRow();
|
||||
|
||||
selected_row.clear();
|
||||
}
|
||||
|
||||
/// Initially, skip all rows. Unskip last on insert.
|
||||
size_t current_pos = current_row_sources.size();
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace_back(current.impl->order, true);
|
||||
|
||||
/// A non-strict comparison, since we select the last row for the same version values.
|
||||
if (version_column_number == -1
|
||||
|| selected_row.empty()
|
||||
|| current->all_columns[version_column_number]->compareAt(
|
||||
current->pos, selected_row.row_num,
|
||||
*(*selected_row.all_columns)[version_column_number],
|
||||
/* nan_direction_hint = */ 1) >= 0)
|
||||
{
|
||||
max_pos = current_pos;
|
||||
setRowRef(selected_row, current);
|
||||
}
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
/// We will write the data for the last primary key.
|
||||
if (!selected_row.empty())
|
||||
insertRow();
|
||||
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
}
|
46
src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h
Normal file
46
src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs into one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* keeps row with max `version` value.
|
||||
*/
|
||||
class ReplacingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks
|
||||
{
|
||||
public:
|
||||
ReplacingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & version_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
private:
|
||||
MergedData merged_data;
|
||||
|
||||
ssize_t version_column_number = -1;
|
||||
|
||||
using RowRef = detail::RowRefWithOwnedChunk;
|
||||
static constexpr size_t max_row_refs = 2; /// last, current.
|
||||
RowRef selected_row; /// Last row with maximum version for current primary key.
|
||||
size_t max_pos = 0; /// The position (into current_row_sources) of the row with the highest version.
|
||||
|
||||
/// Sources of rows with the current primary key.
|
||||
PODArray<RowSourcePart> current_row_sources;
|
||||
|
||||
void insertRow();
|
||||
};
|
||||
|
||||
}
|
216
src/Processors/Merges/Algorithms/RowRef.h
Normal file
216
src/Processors/Merges/Algorithms/RowRef.h
Normal file
@ -0,0 +1,216 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Chunk.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/SortCursor.h>
|
||||
#include <Common/StackTrace.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <boost/smart_ptr/intrusive_ptr.hpp>
|
||||
|
||||
namespace DB::ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace DB::detail
|
||||
{
|
||||
|
||||
class SharedChunkAllocator;
|
||||
|
||||
/// Allows you refer to the row in the block and hold the block ownership,
|
||||
/// and thus avoid creating a temporary row object.
|
||||
/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`;
|
||||
/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece;
|
||||
/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr;
|
||||
/// The reference counter is not atomic, since it is used from one thread.
|
||||
struct SharedChunk : Chunk
|
||||
{
|
||||
ColumnRawPtrs all_columns;
|
||||
ColumnRawPtrs sort_columns;
|
||||
|
||||
using Chunk::Chunk;
|
||||
using Chunk::operator=;
|
||||
|
||||
private:
|
||||
int refcount = 0;
|
||||
size_t position = 0;
|
||||
SharedChunkAllocator * allocator = nullptr;
|
||||
|
||||
friend class SharedChunkAllocator;
|
||||
friend void intrusive_ptr_add_ref(SharedChunk * ptr);
|
||||
friend void intrusive_ptr_release(SharedChunk * ptr);
|
||||
};
|
||||
|
||||
using SharedChunkPtr = boost::intrusive_ptr<detail::SharedChunk>;
|
||||
|
||||
/// Custom allocator for shared chunk.
|
||||
/// It helps to avoid explicit new/delete calls if we know maximum required capacity.
|
||||
/// Thanks to that, SharedChunk does not own any memory.
|
||||
/// It improves leaks detection, because memory is allocated only once in constructor.
|
||||
class SharedChunkAllocator
|
||||
{
|
||||
public:
|
||||
explicit SharedChunkAllocator(size_t max_chunks)
|
||||
{
|
||||
if (max_chunks == 0)
|
||||
max_chunks = 1;
|
||||
|
||||
chunks.resize(max_chunks);
|
||||
free_chunks.reserve(max_chunks);
|
||||
|
||||
for (size_t i = 0; i < max_chunks; ++i)
|
||||
free_chunks.push_back(i);
|
||||
}
|
||||
|
||||
SharedChunkPtr alloc(Chunk && chunk)
|
||||
{
|
||||
if (free_chunks.empty())
|
||||
throw Exception("Not enough space in SharedChunkAllocator. "
|
||||
"Chunks allocated: " + std::to_string(chunks.size()), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto pos = free_chunks.back();
|
||||
free_chunks.pop_back();
|
||||
|
||||
chunks[pos] = std::move(chunk);
|
||||
chunks[pos].position = pos;
|
||||
chunks[pos].allocator = this;
|
||||
|
||||
return SharedChunkPtr(&chunks[pos]);
|
||||
}
|
||||
|
||||
~SharedChunkAllocator()
|
||||
{
|
||||
if (free_chunks.size() != chunks.size())
|
||||
{
|
||||
LOG_ERROR(&Logger::get("SharedChunkAllocator"),
|
||||
"SharedChunkAllocator was destroyed before RowRef was released. StackTrace: "
|
||||
<< StackTrace().toString());
|
||||
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<SharedChunk> chunks;
|
||||
std::vector<size_t> free_chunks;
|
||||
|
||||
void release(SharedChunk * ptr) noexcept
|
||||
{
|
||||
if (chunks.empty())
|
||||
{
|
||||
/// This may happen if allocator was removed before chunks.
|
||||
/// Log message and exit, because we don't want to throw exception in destructor.
|
||||
|
||||
LOG_ERROR(&Logger::get("SharedChunkAllocator"),
|
||||
"SharedChunkAllocator was destroyed before RowRef was released. StackTrace: "
|
||||
<< StackTrace().toString());
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
/// Release memory. It is not obligatory.
|
||||
ptr->clear();
|
||||
ptr->all_columns.clear();
|
||||
ptr->sort_columns.clear();
|
||||
|
||||
free_chunks.push_back(ptr->position);
|
||||
}
|
||||
|
||||
friend void intrusive_ptr_release(SharedChunk * ptr);
|
||||
};
|
||||
|
||||
inline void intrusive_ptr_add_ref(SharedChunk * ptr)
|
||||
{
|
||||
++ptr->refcount;
|
||||
}
|
||||
|
||||
inline void intrusive_ptr_release(SharedChunk * ptr)
|
||||
{
|
||||
if (0 == --ptr->refcount)
|
||||
ptr->allocator->release(ptr);
|
||||
}
|
||||
|
||||
/// This class represents a row in a chunk.
|
||||
struct RowRef
|
||||
{
|
||||
const IColumn ** sort_columns = nullptr; /// Point to sort_columns from SortCursor or last_chunk_sort_columns.
|
||||
size_t num_columns = 0;
|
||||
UInt64 row_num = 0;
|
||||
|
||||
bool empty() const { return sort_columns == nullptr; }
|
||||
void reset() { sort_columns = nullptr; }
|
||||
|
||||
void set(SortCursor & cursor)
|
||||
{
|
||||
sort_columns = cursor.impl->sort_columns.data();
|
||||
num_columns = cursor.impl->sort_columns.size();
|
||||
row_num = cursor.impl->pos;
|
||||
}
|
||||
|
||||
static bool checkEquals(size_t size, const IColumn ** lhs, size_t lhs_row, const IColumn ** rhs, size_t rhs_row)
|
||||
{
|
||||
for (size_t col_number = 0; col_number < size; ++col_number)
|
||||
{
|
||||
auto & cur_column = lhs[col_number];
|
||||
auto & other_column = rhs[col_number];
|
||||
|
||||
if (0 != cur_column->compareAt(lhs_row, rhs_row, *other_column, 1))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool hasEqualSortColumnsWith(const RowRef & other)
|
||||
{
|
||||
return checkEquals(num_columns, sort_columns, row_num, other.sort_columns, other.row_num);
|
||||
}
|
||||
};
|
||||
|
||||
/// This class also represents a row in a chunk.
|
||||
/// RowRefWithOwnedChunk hold shared pointer to this chunk, possibly extending its life time.
|
||||
/// It is needed, for example, in CollapsingTransform, where we need to store first negative row for current sort key.
|
||||
/// We do not copy data itself, because it may be potentially changed for each row. Performance for `set` is important.
|
||||
struct RowRefWithOwnedChunk
|
||||
{
|
||||
detail::SharedChunkPtr owned_chunk = nullptr;
|
||||
|
||||
ColumnRawPtrs * all_columns = nullptr;
|
||||
ColumnRawPtrs * sort_columns = nullptr;
|
||||
UInt64 row_num = 0;
|
||||
|
||||
void swap(RowRefWithOwnedChunk & other)
|
||||
{
|
||||
owned_chunk.swap(other.owned_chunk);
|
||||
std::swap(all_columns, other.all_columns);
|
||||
std::swap(sort_columns, other.sort_columns);
|
||||
std::swap(row_num, other.row_num);
|
||||
}
|
||||
|
||||
bool empty() const { return owned_chunk == nullptr; }
|
||||
|
||||
void clear()
|
||||
{
|
||||
owned_chunk.reset();
|
||||
all_columns = nullptr;
|
||||
sort_columns = nullptr;
|
||||
row_num = 0;
|
||||
}
|
||||
|
||||
void set(SortCursor & cursor, SharedChunkPtr chunk)
|
||||
{
|
||||
owned_chunk = std::move(chunk);
|
||||
row_num = cursor.impl->pos;
|
||||
all_columns = &owned_chunk->all_columns;
|
||||
sort_columns = &owned_chunk->sort_columns;
|
||||
}
|
||||
|
||||
bool hasEqualSortColumnsWith(const RowRefWithOwnedChunk & other)
|
||||
{
|
||||
return RowRef::checkEquals(sort_columns->size(), sort_columns->data(), row_num,
|
||||
other.sort_columns->data(), other.row_num);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
698
src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp
Normal file
698
src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp
Normal file
@ -0,0 +1,698 @@
|
||||
#include <Processors/Merges/Algorithms/SummingSortedAlgorithm.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CORRUPTED_DATA;
|
||||
}
|
||||
|
||||
SummingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition() = default;
|
||||
SummingSortedAlgorithm::ColumnsDefinition::ColumnsDefinition(ColumnsDefinition &&) noexcept = default;
|
||||
SummingSortedAlgorithm::ColumnsDefinition::~ColumnsDefinition() = default;
|
||||
|
||||
/// Stores numbers of key-columns and value-columns.
|
||||
struct SummingSortedAlgorithm::MapDescription
|
||||
{
|
||||
std::vector<size_t> key_col_nums;
|
||||
std::vector<size_t> val_col_nums;
|
||||
};
|
||||
|
||||
/// Stores aggregation function, state, and columns to be used as function arguments.
|
||||
struct SummingSortedAlgorithm::AggregateDescription
|
||||
{
|
||||
/// An aggregate function 'sumWithOverflow' or 'sumMapWithOverflow' for summing.
|
||||
AggregateFunctionPtr function;
|
||||
IAggregateFunction::AddFunc add_function = nullptr;
|
||||
std::vector<size_t> column_numbers;
|
||||
IColumn * merged_column = nullptr;
|
||||
AlignedBuffer state;
|
||||
bool created = false;
|
||||
|
||||
/// In case when column has type AggregateFunction:
|
||||
/// use the aggregate function from itself instead of 'function' above.
|
||||
bool is_agg_func_type = false;
|
||||
|
||||
void init(const char * function_name, const DataTypes & argument_types)
|
||||
{
|
||||
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
|
||||
add_function = function->getAddressOfAddFunction();
|
||||
state.reset(function->sizeOfData(), function->alignOfData());
|
||||
}
|
||||
|
||||
void createState()
|
||||
{
|
||||
if (created)
|
||||
return;
|
||||
if (is_agg_func_type)
|
||||
merged_column->insertDefault();
|
||||
else
|
||||
function->create(state.data());
|
||||
created = true;
|
||||
}
|
||||
|
||||
void destroyState()
|
||||
{
|
||||
if (!created)
|
||||
return;
|
||||
if (!is_agg_func_type)
|
||||
function->destroy(state.data());
|
||||
created = false;
|
||||
}
|
||||
|
||||
/// Explicitly destroy aggregation state if the stream is terminated
|
||||
~AggregateDescription()
|
||||
{
|
||||
destroyState();
|
||||
}
|
||||
|
||||
AggregateDescription() = default;
|
||||
AggregateDescription(AggregateDescription &&) = default;
|
||||
AggregateDescription(const AggregateDescription &) = delete;
|
||||
};
|
||||
|
||||
|
||||
static bool isInPrimaryKey(const SortDescription & description, const std::string & name, const size_t number)
|
||||
{
|
||||
for (auto & desc : description)
|
||||
if (desc.column_name == name || (desc.column_name.empty() && desc.column_number == number))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Returns true if merge result is not empty
|
||||
static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc,
|
||||
Row & row, const ColumnRawPtrs & raw_columns, size_t row_number)
|
||||
{
|
||||
/// Strongly non-optimal.
|
||||
|
||||
Row & left = row;
|
||||
Row right(left.size());
|
||||
|
||||
for (size_t col_num : desc.key_col_nums)
|
||||
right[col_num] = (*raw_columns[col_num])[row_number].template get<Array>();
|
||||
|
||||
for (size_t col_num : desc.val_col_nums)
|
||||
right[col_num] = (*raw_columns[col_num])[row_number].template get<Array>();
|
||||
|
||||
auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field &
|
||||
{
|
||||
return matrix[i].get<Array>()[j];
|
||||
};
|
||||
|
||||
auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array
|
||||
{
|
||||
size_t size = col_nums.size();
|
||||
Array res(size);
|
||||
for (size_t col_num_index = 0; col_num_index < size; ++col_num_index)
|
||||
res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j);
|
||||
return res;
|
||||
};
|
||||
|
||||
std::map<Array, Array> merged;
|
||||
|
||||
auto accumulate = [](Array & dst, const Array & src)
|
||||
{
|
||||
bool has_non_zero = false;
|
||||
size_t size = dst.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (applyVisitor(FieldVisitorSum(src[i]), dst[i]))
|
||||
has_non_zero = true;
|
||||
return has_non_zero;
|
||||
};
|
||||
|
||||
auto merge = [&](const Row & matrix)
|
||||
{
|
||||
size_t rows = matrix[desc.key_col_nums[0]].get<Array>().size();
|
||||
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j);
|
||||
Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j);
|
||||
|
||||
auto it = merged.find(key);
|
||||
if (merged.end() == it)
|
||||
merged.emplace(std::move(key), std::move(value));
|
||||
else
|
||||
{
|
||||
if (!accumulate(it->second, value))
|
||||
merged.erase(it);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
merge(left);
|
||||
merge(right);
|
||||
|
||||
for (size_t col_num : desc.key_col_nums)
|
||||
row[col_num] = Array(merged.size());
|
||||
for (size_t col_num : desc.val_col_nums)
|
||||
row[col_num] = Array(merged.size());
|
||||
|
||||
size_t row_num = 0;
|
||||
for (const auto & key_value : merged)
|
||||
{
|
||||
for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index)
|
||||
row[desc.key_col_nums[col_num_index]].get<Array>()[row_num] = key_value.first[col_num_index];
|
||||
|
||||
for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index)
|
||||
row[desc.val_col_nums[col_num_index]].get<Array>()[row_num] = key_value.second[col_num_index];
|
||||
|
||||
++row_num;
|
||||
}
|
||||
|
||||
return row_num != 0;
|
||||
}
|
||||
|
||||
static SummingSortedAlgorithm::ColumnsDefinition defineColumns(
|
||||
const Block & header,
|
||||
const SortDescription & description,
|
||||
const Names & column_names_to_sum)
|
||||
{
|
||||
size_t num_columns = header.columns();
|
||||
SummingSortedAlgorithm::ColumnsDefinition def;
|
||||
def.column_names = header.getNames();
|
||||
|
||||
/// name of nested structure -> the column numbers that refer to it.
|
||||
std::unordered_map<std::string, std::vector<size_t>> discovered_maps;
|
||||
|
||||
/** Fill in the column numbers, which must be summed.
|
||||
* This can only be numeric columns that are not part of the sort key.
|
||||
* If a non-empty column_names_to_sum is specified, then we only take these columns.
|
||||
* Some columns from column_names_to_sum may not be found. This is ignored.
|
||||
*/
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = header.safeGetByPosition(i);
|
||||
|
||||
/// Discover nested Maps and find columns for summation
|
||||
if (typeid_cast<const DataTypeArray *>(column.type.get()))
|
||||
{
|
||||
const auto map_name = Nested::extractTableName(column.name);
|
||||
/// if nested table name ends with `Map` it is a possible candidate for special handling
|
||||
if (map_name == column.name || !endsWith(map_name, "Map"))
|
||||
{
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
discovered_maps[map_name].emplace_back(i);
|
||||
}
|
||||
else
|
||||
{
|
||||
bool is_agg_func = WhichDataType(column.type).isAggregateFunction();
|
||||
|
||||
/// There are special const columns for example after prewhere sections.
|
||||
if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column))
|
||||
{
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Are they inside the PK?
|
||||
if (isInPrimaryKey(description, column.name, i))
|
||||
{
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (column_names_to_sum.empty()
|
||||
|| column_names_to_sum.end() !=
|
||||
std::find(column_names_to_sum.begin(), column_names_to_sum.end(), column.name))
|
||||
{
|
||||
// Create aggregator to sum this column
|
||||
SummingSortedAlgorithm::AggregateDescription desc;
|
||||
desc.is_agg_func_type = is_agg_func;
|
||||
desc.column_numbers = {i};
|
||||
|
||||
if (!is_agg_func)
|
||||
{
|
||||
desc.init("sumWithOverflow", {column.type});
|
||||
}
|
||||
|
||||
def.columns_to_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// Column is not going to be summed, use last value
|
||||
def.column_numbers_not_to_aggregate.push_back(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// select actual nested Maps from list of candidates
|
||||
for (const auto & map : discovered_maps)
|
||||
{
|
||||
/// map should contain at least two elements (key -> value)
|
||||
if (map.second.size() < 2)
|
||||
{
|
||||
for (auto col : map.second)
|
||||
def.column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// no elements of map could be in primary key
|
||||
auto column_num_it = map.second.begin();
|
||||
for (; column_num_it != map.second.end(); ++column_num_it)
|
||||
if (isInPrimaryKey(description, header.safeGetByPosition(*column_num_it).name, *column_num_it))
|
||||
break;
|
||||
if (column_num_it != map.second.end())
|
||||
{
|
||||
for (auto col : map.second)
|
||||
def.column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
DataTypes argument_types;
|
||||
SummingSortedAlgorithm::AggregateDescription desc;
|
||||
SummingSortedAlgorithm::MapDescription map_desc;
|
||||
|
||||
column_num_it = map.second.begin();
|
||||
for (; column_num_it != map.second.end(); ++column_num_it)
|
||||
{
|
||||
const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it);
|
||||
const String & name = key_col.name;
|
||||
const IDataType & nested_type = *assert_cast<const DataTypeArray &>(*key_col.type).getNestedType();
|
||||
|
||||
if (column_num_it == map.second.begin()
|
||||
|| endsWith(name, "ID")
|
||||
|| endsWith(name, "Key")
|
||||
|| endsWith(name, "Type"))
|
||||
{
|
||||
if (!nested_type.isValueRepresentedByInteger() && !isStringOrFixedString(nested_type))
|
||||
break;
|
||||
|
||||
map_desc.key_col_nums.push_back(*column_num_it);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!nested_type.isSummable())
|
||||
break;
|
||||
|
||||
map_desc.val_col_nums.push_back(*column_num_it);
|
||||
}
|
||||
|
||||
// Add column to function arguments
|
||||
desc.column_numbers.push_back(*column_num_it);
|
||||
argument_types.push_back(key_col.type);
|
||||
}
|
||||
|
||||
if (column_num_it != map.second.end())
|
||||
{
|
||||
for (auto col : map.second)
|
||||
def.column_numbers_not_to_aggregate.push_back(col);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (map_desc.key_col_nums.size() == 1)
|
||||
{
|
||||
// Create summation for all value columns in the map
|
||||
desc.init("sumMapWithOverflow", argument_types);
|
||||
def.columns_to_aggregate.emplace_back(std::move(desc));
|
||||
}
|
||||
else
|
||||
{
|
||||
// Fall back to legacy mergeMaps for composite keys
|
||||
for (auto col : map.second)
|
||||
def.column_numbers_not_to_aggregate.push_back(col);
|
||||
def.maps_to_sum.emplace_back(std::move(map_desc));
|
||||
}
|
||||
}
|
||||
|
||||
return def;
|
||||
}
|
||||
|
||||
static MutableColumns getMergedDataColumns(
|
||||
const Block & header,
|
||||
const SummingSortedAlgorithm::ColumnsDefinition & def)
|
||||
{
|
||||
MutableColumns columns;
|
||||
size_t num_columns = def.column_numbers_not_to_aggregate.size() + def.columns_to_aggregate.size();
|
||||
columns.reserve(num_columns);
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
{
|
||||
// Wrap aggregated columns in a tuple to match function signature
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
MutableColumns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->cloneEmpty();
|
||||
|
||||
columns.emplace_back(ColumnTuple::create(std::move(tuple_columns)));
|
||||
}
|
||||
else
|
||||
columns.emplace_back(header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty());
|
||||
}
|
||||
|
||||
for (auto & column_number : def.column_numbers_not_to_aggregate)
|
||||
columns.emplace_back(header.safeGetByPosition(column_number).type->createColumn());
|
||||
|
||||
return columns;
|
||||
}
|
||||
|
||||
static void preprocessChunk(Chunk & chunk)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
static void postprocessChunk(
|
||||
Chunk & chunk, size_t num_result_columns,
|
||||
const SummingSortedAlgorithm::ColumnsDefinition & def)
|
||||
{
|
||||
size_t num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
|
||||
Columns res_columns(num_result_columns);
|
||||
size_t next_column = 0;
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
{
|
||||
auto column = std::move(columns[next_column]);
|
||||
++next_column;
|
||||
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
/// Unpack tuple into block.
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
res_columns[desc.column_numbers[i]] = assert_cast<const ColumnTuple &>(*column).getColumnPtr(i);
|
||||
}
|
||||
else
|
||||
res_columns[desc.column_numbers[0]] = std::move(column);
|
||||
}
|
||||
|
||||
for (auto column_number : def.column_numbers_not_to_aggregate)
|
||||
{
|
||||
auto column = std::move(columns[next_column]);
|
||||
++next_column;
|
||||
|
||||
res_columns[column_number] = std::move(column);
|
||||
}
|
||||
|
||||
chunk.setColumns(std::move(res_columns), num_rows);
|
||||
}
|
||||
|
||||
static void setRow(Row & row, const ColumnRawPtrs & raw_columns, size_t row_num, const Names & column_names)
|
||||
{
|
||||
size_t num_columns = row.size();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
try
|
||||
{
|
||||
raw_columns[i]->get(row_num, row[i]);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
/// Find out the name of the column and throw more informative exception.
|
||||
|
||||
String column_name;
|
||||
if (i < column_names.size())
|
||||
column_name = column_names[i];
|
||||
|
||||
throw Exception("MergingSortedBlockInputStream failed to read row " + toString(row_num)
|
||||
+ " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"),
|
||||
ErrorCodes::CORRUPTED_DATA);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
SummingSortedAlgorithm::SummingMergedData::SummingMergedData(
|
||||
MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_)
|
||||
, def(def_)
|
||||
{
|
||||
current_row.resize(def.column_names.size());
|
||||
initAggregateDescription();
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::SummingMergedData::startGroup(ColumnRawPtrs & raw_columns, size_t row)
|
||||
{
|
||||
is_group_started = true;
|
||||
|
||||
setRow(current_row, raw_columns, row, def.column_names);
|
||||
|
||||
/// Reset aggregation states for next row
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
desc.createState();
|
||||
|
||||
if (def.maps_to_sum.empty())
|
||||
{
|
||||
/// We have only columns_to_aggregate. The status of current row will be determined
|
||||
/// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions.
|
||||
current_row_is_zero = true; // NOLINT
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have complex maps that will be summed with 'mergeMap' method.
|
||||
/// The single row is considered non zero, and the status after merging with other rows
|
||||
/// will be determined in the branch below (when key_differs == false).
|
||||
current_row_is_zero = false; // NOLINT
|
||||
}
|
||||
|
||||
addRowImpl(raw_columns, row);
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::SummingMergedData::finishGroup()
|
||||
{
|
||||
is_group_started = false;
|
||||
|
||||
/// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate.
|
||||
if (def.columns_to_aggregate.empty())
|
||||
current_row_is_zero = false;
|
||||
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
{
|
||||
// Do not insert if the aggregation state hasn't been created
|
||||
if (desc.created)
|
||||
{
|
||||
if (desc.is_agg_func_type)
|
||||
{
|
||||
current_row_is_zero = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
try
|
||||
{
|
||||
desc.function->insertResultInto(desc.state.data(), *desc.merged_column);
|
||||
|
||||
/// Update zero status of current row
|
||||
if (desc.column_numbers.size() == 1)
|
||||
{
|
||||
// Flag row as non-empty if at least one column number if non-zero
|
||||
current_row_is_zero = current_row_is_zero
|
||||
&& desc.merged_column->isDefaultAt(desc.merged_column->size() - 1);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// It is sumMapWithOverflow aggregate function.
|
||||
/// Assume that the row isn't empty in this case
|
||||
/// (just because it is compatible with previous version)
|
||||
current_row_is_zero = false;
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
desc.destroyState();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
desc.destroyState();
|
||||
}
|
||||
else
|
||||
desc.merged_column->insertDefault();
|
||||
}
|
||||
|
||||
/// If it is "zero" row, then rollback the insertion
|
||||
/// (at this moment we need rollback only cols from columns_to_aggregate)
|
||||
if (current_row_is_zero)
|
||||
{
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
desc.merged_column->popBack(1);
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
size_t next_column = columns.size() - def.column_numbers_not_to_aggregate.size();
|
||||
for (auto column_number : def.column_numbers_not_to_aggregate)
|
||||
{
|
||||
columns[next_column]->insert(current_row[column_number]);
|
||||
++next_column;
|
||||
}
|
||||
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
/// TODO: sum_blocks_granularity += block_size;
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::SummingMergedData::addRow(ColumnRawPtrs & raw_columns, size_t row)
|
||||
{
|
||||
// Merge maps only for same rows
|
||||
for (const auto & desc : def.maps_to_sum)
|
||||
if (mergeMap(desc, current_row, raw_columns, row))
|
||||
current_row_is_zero = false;
|
||||
|
||||
addRowImpl(raw_columns, row);
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::SummingMergedData::addRowImpl(ColumnRawPtrs & raw_columns, size_t row)
|
||||
{
|
||||
for (auto & desc : def.columns_to_aggregate)
|
||||
{
|
||||
if (!desc.created)
|
||||
throw Exception("Logical error in SummingSortedBlockInputStream, there are no description",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (desc.is_agg_func_type)
|
||||
{
|
||||
// desc.state is not used for AggregateFunction types
|
||||
auto & col = raw_columns[desc.column_numbers[0]];
|
||||
assert_cast<ColumnAggregateFunction &>(*desc.merged_column).insertMergeFrom(*col, row);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Specialized case for unary functions
|
||||
if (desc.column_numbers.size() == 1)
|
||||
{
|
||||
auto & col = raw_columns[desc.column_numbers[0]];
|
||||
desc.add_function(desc.function.get(), desc.state.data(), &col, row, nullptr);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Gather all source columns into a vector
|
||||
ColumnRawPtrs column_ptrs(desc.column_numbers.size());
|
||||
for (size_t i = 0; i < desc.column_numbers.size(); ++i)
|
||||
column_ptrs[i] = raw_columns[desc.column_numbers[i]];
|
||||
|
||||
desc.add_function(desc.function.get(), desc.state.data(), column_ptrs.data(), row, nullptr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::SummingMergedData::initAggregateDescription()
|
||||
{
|
||||
size_t num_columns = def.columns_to_aggregate.size();
|
||||
for (size_t column_number = 0; column_number < num_columns; ++column_number)
|
||||
def.columns_to_aggregate[column_number].merged_column = columns[column_number].get();
|
||||
}
|
||||
|
||||
|
||||
Chunk SummingSortedAlgorithm::SummingMergedData::pull()
|
||||
{
|
||||
auto chunk = MergedData::pull();
|
||||
postprocessChunk(chunk, def.column_names.size(), def);
|
||||
|
||||
initAggregateDescription();
|
||||
|
||||
return chunk;
|
||||
}
|
||||
|
||||
|
||||
SummingSortedAlgorithm::SummingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const Names & column_names_to_sum,
|
||||
size_t max_block_size)
|
||||
: IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_))
|
||||
, columns_definition(defineColumns(header, description, column_names_to_sum))
|
||||
, merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition)
|
||||
{
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::initialize(Chunks chunks)
|
||||
{
|
||||
for (auto & chunk : chunks)
|
||||
if (chunk)
|
||||
preprocessChunk(chunk);
|
||||
|
||||
initializeQueue(std::move(chunks));
|
||||
}
|
||||
|
||||
void SummingSortedAlgorithm::consume(Chunk chunk, size_t source_num)
|
||||
{
|
||||
preprocessChunk(chunk);
|
||||
updateCursor(std::move(chunk), source_num);
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status SummingSortedAlgorithm::merge()
|
||||
{
|
||||
/// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
bool key_differs;
|
||||
|
||||
SortCursor current = queue.current();
|
||||
|
||||
{
|
||||
detail::RowRef current_key;
|
||||
current_key.set(current);
|
||||
|
||||
key_differs = last_key.empty() || !last_key.hasEqualSortColumnsWith(current_key);
|
||||
|
||||
last_key = current_key;
|
||||
last_chunk_sort_columns.clear();
|
||||
}
|
||||
|
||||
if (key_differs)
|
||||
{
|
||||
if (merged_data.isGroupStarted())
|
||||
/// Write the data for the previous group.
|
||||
merged_data.finishGroup();
|
||||
|
||||
if (merged_data.hasEnoughRows())
|
||||
{
|
||||
/// The block is now full and the last row is calculated completely.
|
||||
last_key.reset();
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
merged_data.startGroup(current->all_columns, current->pos);
|
||||
}
|
||||
else
|
||||
merged_data.addRow(current->all_columns, current->pos);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We get the next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
/// We will write the data for the last group, if it is non-zero.
|
||||
if (merged_data.isGroupStarted())
|
||||
merged_data.finishGroup();
|
||||
|
||||
last_chunk_sort_columns.clear();
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
}
|
91
src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h
Normal file
91
src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h
Normal file
@ -0,0 +1,91 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <Core/Row.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs into one.
|
||||
* For each group of consecutive identical values of the primary key (the columns by which the data is sorted),
|
||||
* collapses them into one row, summing all the numeric columns except the primary key.
|
||||
* If in all numeric columns, except for the primary key, the result is zero, it deletes the row.
|
||||
*/
|
||||
class SummingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk
|
||||
{
|
||||
public:
|
||||
SummingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_,
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum,
|
||||
size_t max_block_size);
|
||||
|
||||
void initialize(Chunks chunks) override;
|
||||
void consume(Chunk chunk, size_t source_num) override;
|
||||
Status merge() override;
|
||||
|
||||
struct AggregateDescription;
|
||||
struct MapDescription;
|
||||
|
||||
/// This structure define columns into one of three types:
|
||||
/// * columns which values not needed to be aggregated
|
||||
/// * aggregate functions and columns which needed to be summed
|
||||
/// * mapping for nested columns
|
||||
struct ColumnsDefinition
|
||||
{
|
||||
ColumnsDefinition(); /// Is needed because destructor is defined.
|
||||
ColumnsDefinition(ColumnsDefinition &&) noexcept; /// Is needed because destructor is defined.
|
||||
~ColumnsDefinition(); /// Is needed because otherwise std::vector's destructor uses incomplete types.
|
||||
|
||||
/// Columns with which values should not be aggregated.
|
||||
ColumnNumbers column_numbers_not_to_aggregate;
|
||||
/// Columns which should be aggregated.
|
||||
std::vector<AggregateDescription> columns_to_aggregate;
|
||||
/// Mapping for nested columns.
|
||||
std::vector<MapDescription> maps_to_sum;
|
||||
|
||||
/// Names of columns from header.
|
||||
Names column_names;
|
||||
};
|
||||
|
||||
/// Specialization for SummingSortedTransform. Inserts only data for non-aggregated columns.
|
||||
class SummingMergedData : public MergedData
|
||||
{
|
||||
private:
|
||||
using MergedData::pull;
|
||||
using MergedData::insertRow;
|
||||
|
||||
public:
|
||||
SummingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_);
|
||||
|
||||
void startGroup(ColumnRawPtrs & raw_columns, size_t row);
|
||||
void finishGroup();
|
||||
|
||||
bool isGroupStarted() const { return is_group_started; }
|
||||
void addRow(ColumnRawPtrs & raw_columns, size_t row); /// Possible only when group was started.
|
||||
|
||||
Chunk pull();
|
||||
|
||||
private:
|
||||
ColumnsDefinition & def;
|
||||
|
||||
bool is_group_started = false;
|
||||
|
||||
Row current_row;
|
||||
bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally.
|
||||
|
||||
void addRowImpl(ColumnRawPtrs & raw_columns, size_t row);
|
||||
|
||||
/// Initialize aggregate descriptions with columns.
|
||||
void initAggregateDescription();
|
||||
};
|
||||
|
||||
private:
|
||||
/// Order between members is important because merged_data has reference to columns_definition.
|
||||
ColumnsDefinition columns_definition;
|
||||
SummingMergedData merged_data;
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,150 @@
|
||||
#include <Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static const size_t MAX_ROWS_IN_MULTIVERSION_QUEUE = 8192;
|
||||
|
||||
VersionedCollapsingAlgorithm::VersionedCollapsingAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: IMergingAlgorithmWithSharedChunks(
|
||||
num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE)
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
/// -1 for +1 in FixedSizeDequeWithGaps's internal buffer. 3 is a reasonable minimum size to collapse anything.
|
||||
, max_rows_in_queue(std::min(std::max<size_t>(3, max_block_size), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 1)
|
||||
, current_keys(max_rows_in_queue)
|
||||
{
|
||||
sign_column_number = header.getPositionByName(sign_column_);
|
||||
}
|
||||
|
||||
inline ALWAYS_INLINE static void writeRowSourcePart(WriteBuffer & buffer, RowSourcePart row_source)
|
||||
{
|
||||
if constexpr (sizeof(RowSourcePart) == 1)
|
||||
buffer.write(*reinterpret_cast<const char *>(&row_source));
|
||||
else
|
||||
buffer.write(reinterpret_cast<const char *>(&row_source), sizeof(RowSourcePart));
|
||||
}
|
||||
|
||||
void VersionedCollapsingAlgorithm::insertGap(size_t gap_size)
|
||||
{
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
for (size_t i = 0; i < gap_size; ++i)
|
||||
{
|
||||
writeRowSourcePart(*out_row_sources_buf, current_row_sources.front());
|
||||
current_row_sources.pop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void VersionedCollapsingAlgorithm::insertRow(size_t skip_rows, const RowRef & row)
|
||||
{
|
||||
merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows());
|
||||
|
||||
insertGap(skip_rows);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
current_row_sources.front().setSkipFlag(false);
|
||||
writeRowSourcePart(*out_row_sources_buf, current_row_sources.front());
|
||||
current_row_sources.pop();
|
||||
}
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status VersionedCollapsingAlgorithm::merge()
|
||||
{
|
||||
/// Take rows in correct order and put them into `merged_columns` until the rows no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
SortCursor current = queue.current();
|
||||
|
||||
RowRef current_row;
|
||||
|
||||
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];
|
||||
|
||||
setRowRef(current_row, current);
|
||||
|
||||
/// At first, let's decide the number of rows needed to insert right now.
|
||||
size_t num_rows_to_insert = 0;
|
||||
if (!current_keys.empty())
|
||||
{
|
||||
auto key_differs = !current_row.hasEqualSortColumnsWith(current_keys.back());
|
||||
|
||||
if (key_differs) /// Flush whole queue
|
||||
num_rows_to_insert = current_keys.size();
|
||||
else if (current_keys.size() >= max_rows_in_queue) /// Flush single row if queue is big
|
||||
num_rows_to_insert = 1;
|
||||
}
|
||||
|
||||
/// Insert ready roes if any.
|
||||
while (num_rows_to_insert)
|
||||
{
|
||||
const auto & row = current_keys.front();
|
||||
auto gap = current_keys.frontGap();
|
||||
|
||||
insertRow(gap, row);
|
||||
|
||||
current_keys.popFront();
|
||||
|
||||
--num_rows_to_insert;
|
||||
|
||||
/// It's ok to return here, because we didn't affect queue.
|
||||
if (merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
if (current_keys.empty())
|
||||
{
|
||||
sign_in_queue = sign;
|
||||
current_keys.pushBack(current_row);
|
||||
}
|
||||
else /// If queue is not empty, then current_row has the same key as in current_keys queue
|
||||
{
|
||||
if (sign == sign_in_queue)
|
||||
current_keys.pushBack(current_row);
|
||||
else
|
||||
{
|
||||
current_keys.popBack();
|
||||
current_keys.pushGap(2);
|
||||
}
|
||||
}
|
||||
|
||||
if (out_row_sources_buf)
|
||||
current_row_sources.emplace(current->order, true);
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We take next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
return Status(current.impl->order);
|
||||
}
|
||||
}
|
||||
|
||||
while (!current_keys.empty())
|
||||
{
|
||||
const auto & row = current_keys.front();
|
||||
auto gap = current_keys.frontGap();
|
||||
|
||||
insertRow(gap, row);
|
||||
current_keys.popFront();
|
||||
|
||||
if (merged_data.hasEnoughRows())
|
||||
return Status(merged_data.pull());
|
||||
}
|
||||
|
||||
/// Write information about last collapsed rows.
|
||||
insertGap(current_keys.frontGap());
|
||||
return Status(merged_data.pull(), true);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,45 @@
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithSharedChunks.h>
|
||||
#include <Processors/Merges/Algorithms/MergedData.h>
|
||||
#include <Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <queue>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Merges several sorted inputs to one.
|
||||
* For each group of consecutive identical values of the sorting key
|
||||
* (the columns by which the data is sorted, including specially specified version column),
|
||||
* merges any pair of consecutive rows with opposite sign.
|
||||
*/
|
||||
class VersionedCollapsingAlgorithm final : public IMergingAlgorithmWithSharedChunks
|
||||
{
|
||||
public:
|
||||
/// Don't need version column. It's in primary key.
|
||||
VersionedCollapsingAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
private:
|
||||
MergedData merged_data;
|
||||
|
||||
size_t sign_column_number = 0;
|
||||
|
||||
const size_t max_rows_in_queue;
|
||||
|
||||
/// Rows with the same primary key and sign.
|
||||
FixedSizeDequeWithGaps<RowRef> current_keys;
|
||||
Int8 sign_in_queue = 0;
|
||||
|
||||
std::queue<RowSourcePart> current_row_sources; /// Sources of rows with the current primary key
|
||||
|
||||
void insertGap(size_t gap_size);
|
||||
void insertRow(size_t skip_rows, const RowRef & row);
|
||||
};
|
||||
|
||||
}
|
37
src/Processors/Merges/CollapsingSortedTransform.h
Normal file
37
src/Processors/Merges/CollapsingSortedTransform.h
Normal file
@ -0,0 +1,37 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via CollapsingSortedAlgorithm.
|
||||
class CollapsingSortedTransform final : public IMergingTransform<CollapsingSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
CollapsingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
sign_column,
|
||||
max_block_size,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes,
|
||||
&Logger::get("CollapsingSortedTransform"))
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "CollapsingSortedTransform"; }
|
||||
};
|
||||
|
||||
}
|
31
src/Processors/Merges/GraphiteRollupSortedTransform.h
Normal file
31
src/Processors/Merges/GraphiteRollupSortedTransform.h
Normal file
@ -0,0 +1,31 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via GraphiteRollupSortedAlgorithm.
|
||||
class GraphiteRollupSortedTransform final : public IMergingTransform<GraphiteRollupSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
GraphiteRollupSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
max_block_size,
|
||||
std::move(params_),
|
||||
time_of_merge_)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "GraphiteRollupSortedTransform"; }
|
||||
};
|
||||
|
||||
}
|
177
src/Processors/Merges/IMergingTransform.cpp
Normal file
177
src/Processors/Merges/IMergingTransform.cpp
Normal file
@ -0,0 +1,177 @@
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
IMergingTransformBase::IMergingTransformBase(
|
||||
size_t num_inputs,
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_)
|
||||
: IProcessor(InputPorts(num_inputs, input_header), {output_header})
|
||||
, have_all_inputs(have_all_inputs_)
|
||||
{
|
||||
}
|
||||
|
||||
void IMergingTransformBase::onNewInput()
|
||||
{
|
||||
throw Exception("onNewInput is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void IMergingTransformBase::addInput()
|
||||
{
|
||||
if (have_all_inputs)
|
||||
throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
inputs.emplace_back(outputs.front().getHeader(), this);
|
||||
onNewInput();
|
||||
}
|
||||
|
||||
void IMergingTransformBase::setHaveAllInputs()
|
||||
{
|
||||
if (have_all_inputs)
|
||||
throw Exception("IMergingTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
have_all_inputs = true;
|
||||
}
|
||||
|
||||
IProcessor::Status IMergingTransformBase::prepareInitializeInputs()
|
||||
{
|
||||
/// Add information about inputs.
|
||||
if (input_states.empty())
|
||||
{
|
||||
input_states.reserve(inputs.size());
|
||||
for (auto & input : inputs)
|
||||
input_states.emplace_back(input);
|
||||
|
||||
state.init_chunks.resize(inputs.size());
|
||||
}
|
||||
|
||||
/// Check for inputs we need.
|
||||
bool all_inputs_has_data = true;
|
||||
auto it = inputs.begin();
|
||||
for (size_t i = 0; it != inputs.end(); ++i, ++it)
|
||||
{
|
||||
auto & input = *it;
|
||||
if (input.isFinished())
|
||||
continue;
|
||||
|
||||
if (input_states[i].is_initialized)
|
||||
{
|
||||
// input.setNotNeeded();
|
||||
continue;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
{
|
||||
all_inputs_has_data = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
auto chunk = input.pull();
|
||||
if (!chunk.hasRows())
|
||||
{
|
||||
|
||||
if (!input.isFinished())
|
||||
all_inputs_has_data = false;
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
state.init_chunks[i] = std::move(chunk);
|
||||
input_states[i].is_initialized = true;
|
||||
}
|
||||
|
||||
if (!all_inputs_has_data)
|
||||
return Status::NeedData;
|
||||
|
||||
is_initialized = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
IProcessor::Status IMergingTransformBase::prepare()
|
||||
{
|
||||
if (!have_all_inputs)
|
||||
return Status::NeedData;
|
||||
|
||||
auto & output = outputs.front();
|
||||
|
||||
/// Special case for no inputs.
|
||||
if (inputs.empty())
|
||||
{
|
||||
output.finish();
|
||||
onFinish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
/// Check can output.
|
||||
|
||||
if (output.isFinished())
|
||||
{
|
||||
for (auto & in : inputs)
|
||||
in.close();
|
||||
|
||||
onFinish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
/// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before.
|
||||
bool is_port_full = !output.canPush();
|
||||
|
||||
/// Push if has data.
|
||||
if (state.output_chunk && !is_port_full)
|
||||
output.push(std::move(state.output_chunk));
|
||||
|
||||
if (!is_initialized)
|
||||
return prepareInitializeInputs();
|
||||
|
||||
if (state.is_finished)
|
||||
{
|
||||
|
||||
if (is_port_full)
|
||||
return Status::PortFull;
|
||||
|
||||
for (auto & input : inputs)
|
||||
input.close();
|
||||
|
||||
outputs.front().finish();
|
||||
|
||||
onFinish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (state.need_data)
|
||||
{
|
||||
auto & input = input_states[state.next_input_to_read].port;
|
||||
if (!input.isFinished())
|
||||
{
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
auto chunk = input.pull();
|
||||
if (!chunk.hasRows() && !input.isFinished())
|
||||
return Status::NeedData;
|
||||
|
||||
state.input_chunk = std::move(chunk);
|
||||
}
|
||||
|
||||
state.need_data = false;
|
||||
}
|
||||
|
||||
if (is_port_full)
|
||||
return Status::PortFull;
|
||||
|
||||
return Status::Ready;
|
||||
}
|
||||
|
||||
}
|
123
src/Processors/Merges/IMergingTransform.h
Normal file
123
src/Processors/Merges/IMergingTransform.h
Normal file
@ -0,0 +1,123 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/Algorithms/IMergingAlgorithm.h>
|
||||
#include <Processors/IProcessor.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Base class for IMergingTransform.
|
||||
/// It is needed to extract all non-template methods in single translation unit.
|
||||
class IMergingTransformBase : public IProcessor
|
||||
{
|
||||
public:
|
||||
IMergingTransformBase(
|
||||
size_t num_inputs,
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_);
|
||||
|
||||
/// Methods to add additional input port. It is possible to do only before the first call of `prepare`.
|
||||
void addInput();
|
||||
/// Need to be called after all inputs are added. (only if have_all_inputs was not specified).
|
||||
void setHaveAllInputs();
|
||||
|
||||
Status prepare() override;
|
||||
|
||||
protected:
|
||||
virtual void onNewInput(); /// Is called when new input is added. Only if have_all_inputs = false.
|
||||
virtual void onFinish() {} /// Is called when all data is processed.
|
||||
|
||||
/// Processor state.
|
||||
struct State
|
||||
{
|
||||
Chunk output_chunk;
|
||||
Chunk input_chunk;
|
||||
bool is_finished = false;
|
||||
bool need_data = false;
|
||||
size_t next_input_to_read = 0;
|
||||
|
||||
Chunks init_chunks;
|
||||
};
|
||||
|
||||
State state;
|
||||
|
||||
private:
|
||||
struct InputState
|
||||
{
|
||||
explicit InputState(InputPort & port_) : port(port_) {}
|
||||
|
||||
InputPort & port;
|
||||
bool is_initialized = false;
|
||||
};
|
||||
|
||||
std::vector<InputState> input_states;
|
||||
std::atomic<bool> have_all_inputs;
|
||||
bool is_initialized = false;
|
||||
|
||||
IProcessor::Status prepareInitializeInputs();
|
||||
};
|
||||
|
||||
/// Implementation of MergingTransform using IMergingAlgorithm.
|
||||
template <typename Algorithm>
|
||||
class IMergingTransform : public IMergingTransformBase
|
||||
{
|
||||
public:
|
||||
template <typename ... Args>
|
||||
IMergingTransform(
|
||||
size_t num_inputs,
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_,
|
||||
Args && ... args)
|
||||
: IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_)
|
||||
, algorithm(std::forward<Args>(args) ...)
|
||||
{
|
||||
}
|
||||
|
||||
void work() override
|
||||
{
|
||||
if (!state.init_chunks.empty())
|
||||
algorithm.initialize(std::move(state.init_chunks));
|
||||
|
||||
if (state.input_chunk)
|
||||
{
|
||||
// std::cerr << "Consume chunk with " << state.input_chunk.getNumRows()
|
||||
// << " for input " << state.next_input_to_read << std::endl;
|
||||
algorithm.consume(std::move(state.input_chunk), state.next_input_to_read);
|
||||
}
|
||||
|
||||
IMergingAlgorithm::Status status = algorithm.merge();
|
||||
|
||||
if (status.chunk && status.chunk.hasRows())
|
||||
{
|
||||
// std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl;
|
||||
state.output_chunk = std::move(status.chunk);
|
||||
}
|
||||
|
||||
if (status.required_source >= 0)
|
||||
{
|
||||
// std::cerr << "Required data for input " << status.required_source << std::endl;
|
||||
state.next_input_to_read = status.required_source;
|
||||
state.need_data = true;
|
||||
}
|
||||
|
||||
if (status.is_finished)
|
||||
{
|
||||
// std::cerr << "Finished" << std::endl;
|
||||
state.is_finished = true;
|
||||
}
|
||||
}
|
||||
|
||||
protected:
|
||||
Algorithm algorithm;
|
||||
|
||||
/// Profile info.
|
||||
Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE};
|
||||
|
||||
private:
|
||||
using IMergingTransformBase::state;
|
||||
};
|
||||
|
||||
}
|
63
src/Processors/Merges/MergingSortedTransform.cpp
Normal file
63
src/Processors/Merges/MergingSortedTransform.cpp
Normal file
@ -0,0 +1,63 @@
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MergingSortedTransform::MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool quiet_,
|
||||
bool use_average_block_sizes,
|
||||
bool have_all_inputs_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, have_all_inputs_,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
max_block_size,
|
||||
limit_,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes)
|
||||
, quiet(quiet_)
|
||||
{
|
||||
}
|
||||
|
||||
void MergingSortedTransform::onNewInput()
|
||||
{
|
||||
algorithm.addInput();
|
||||
}
|
||||
|
||||
void MergingSortedTransform::onFinish()
|
||||
{
|
||||
if (quiet)
|
||||
return;
|
||||
|
||||
auto & merged_data = algorithm.getMergedData();
|
||||
|
||||
auto * log = &Logger::get("MergingSortedTransform");
|
||||
|
||||
double seconds = total_stopwatch.elapsedSeconds();
|
||||
|
||||
std::stringstream message;
|
||||
message << std::fixed << std::setprecision(2)
|
||||
<< "Merge sorted " << merged_data.totalChunks() << " blocks, " << merged_data.totalMergedRows() << " rows"
|
||||
<< " in " << seconds << " sec.";
|
||||
|
||||
if (seconds != 0)
|
||||
message << ", "
|
||||
<< merged_data.totalMergedRows() / seconds << " rows/sec., "
|
||||
<< merged_data.totalAllocatedBytes() / 1000000.0 / seconds << " MB/sec.";
|
||||
|
||||
LOG_DEBUG(log, message.str());
|
||||
}
|
||||
|
||||
}
|
35
src/Processors/Merges/MergingSortedTransform.h
Normal file
35
src/Processors/Merges/MergingSortedTransform.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/MergingSortedAlgorithm.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via MergingSortedAlgorithm.
|
||||
class MergingSortedTransform final : public IMergingTransform<MergingSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_ = 0,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool quiet_ = false,
|
||||
bool use_average_block_sizes = false,
|
||||
bool have_all_inputs_ = true);
|
||||
|
||||
String getName() const override { return "MergingSortedTransform"; }
|
||||
|
||||
protected:
|
||||
void onNewInput() override;
|
||||
void onFinish() override;
|
||||
|
||||
private:
|
||||
bool quiet = false;
|
||||
};
|
||||
|
||||
}
|
35
src/Processors/Merges/ReplacingSortedTransform.h
Normal file
35
src/Processors/Merges/ReplacingSortedTransform.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via ReplacingSortedAlgorithm.
|
||||
class ReplacingSortedTransform final : public IMergingTransform<ReplacingSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
ReplacingSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & version_column,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
version_column,
|
||||
max_block_size,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "ReplacingSorted"; }
|
||||
};
|
||||
|
||||
}
|
33
src/Processors/Merges/SummingSortedTransform.h
Normal file
33
src/Processors/Merges/SummingSortedTransform.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/SummingSortedAlgorithm.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via SummingSortedAlgorithm.
|
||||
class SummingSortedTransform final : public IMergingTransform<SummingSortedAlgorithm>
|
||||
{
|
||||
public:
|
||||
|
||||
SummingSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_,
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum,
|
||||
size_t max_block_size)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
column_names_to_sum,
|
||||
max_block_size)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "SummingSortedTransform"; }
|
||||
};
|
||||
|
||||
}
|
36
src/Processors/Merges/VersionedCollapsingTransform.h
Normal file
36
src/Processors/Merges/VersionedCollapsingTransform.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/Merges/IMergingTransform.h>
|
||||
#include <Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Implementation of IMergingTransform via VersionedCollapsingAlgorithm.
|
||||
class VersionedCollapsingTransform final : public IMergingTransform<VersionedCollapsingAlgorithm>
|
||||
{
|
||||
public:
|
||||
/// Don't need version column. It's in primary key.
|
||||
VersionedCollapsingTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, true,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
sign_column_,
|
||||
max_block_size,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "VersionedCollapsingTransform"; }
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,6 @@
|
||||
#include <Processors/Transforms/MergeSortingTransform.h>
|
||||
#include <Processors/IAccumulatingTransform.h>
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
@ -191,6 +191,7 @@ void MergeSortingTransform::consume(Chunk chunk)
|
||||
{
|
||||
bool quiet = false;
|
||||
bool have_all_inputs = false;
|
||||
bool use_average_block_sizes = false;
|
||||
|
||||
external_merging_sorted = std::make_shared<MergingSortedTransform>(
|
||||
header_without_constants,
|
||||
@ -198,7 +199,9 @@ void MergeSortingTransform::consume(Chunk chunk)
|
||||
description,
|
||||
max_merged_block_size,
|
||||
limit,
|
||||
nullptr,
|
||||
quiet,
|
||||
use_average_block_sizes,
|
||||
have_all_inputs);
|
||||
|
||||
processors.emplace_back(external_merging_sorted);
|
||||
|
@ -1,331 +0,0 @@
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergingSortedTransform::MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size_,
|
||||
UInt64 limit_,
|
||||
bool quiet_,
|
||||
bool have_all_inputs_)
|
||||
: IProcessor(InputPorts(num_inputs, header), {header})
|
||||
, description(description_), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, have_all_inputs(have_all_inputs_)
|
||||
, merged_data(header), source_chunks(num_inputs), cursors(num_inputs)
|
||||
{
|
||||
auto & sample = outputs.front().getHeader();
|
||||
/// Replace column names in description to positions.
|
||||
for (auto & column_description : description)
|
||||
{
|
||||
has_collation |= column_description.collator != nullptr;
|
||||
if (!column_description.column_name.empty())
|
||||
{
|
||||
column_description.column_number = sample.getPositionByName(column_description.column_name);
|
||||
column_description.column_name.clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MergingSortedTransform::addInput()
|
||||
{
|
||||
if (have_all_inputs)
|
||||
throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
inputs.emplace_back(outputs.front().getHeader(), this);
|
||||
source_chunks.emplace_back();
|
||||
cursors.emplace_back();
|
||||
}
|
||||
|
||||
void MergingSortedTransform::setHaveAllInputs()
|
||||
{
|
||||
if (have_all_inputs)
|
||||
throw Exception("MergingSortedTransform already have all inputs.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
have_all_inputs = true;
|
||||
}
|
||||
|
||||
IProcessor::Status MergingSortedTransform::prepare()
|
||||
{
|
||||
if (!have_all_inputs)
|
||||
return Status::NeedData;
|
||||
|
||||
auto & output = outputs.front();
|
||||
|
||||
/// Special case for no inputs.
|
||||
if (inputs.empty())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
/// Check can output.
|
||||
|
||||
if (output.isFinished())
|
||||
{
|
||||
for (auto & in : inputs)
|
||||
in.close();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
/// Do not disable inputs, so it will work in the same way as with AsynchronousBlockInputStream, like before.
|
||||
bool is_port_full = !output.canPush();
|
||||
|
||||
/// Special case for single input.
|
||||
if (inputs.size() == 1)
|
||||
{
|
||||
auto & input = inputs.front();
|
||||
if (input.isFinished())
|
||||
{
|
||||
output.finish();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (input.hasData())
|
||||
{
|
||||
if (!is_port_full)
|
||||
output.push(input.pull());
|
||||
|
||||
return Status::PortFull;
|
||||
}
|
||||
|
||||
return Status::NeedData;
|
||||
}
|
||||
|
||||
/// Push if has data.
|
||||
if (merged_data.mergedRows() && !is_port_full)
|
||||
output.push(merged_data.pull());
|
||||
|
||||
if (!is_initialized)
|
||||
{
|
||||
/// Check for inputs we need.
|
||||
bool all_inputs_has_data = true;
|
||||
auto it = inputs.begin();
|
||||
for (size_t i = 0; it != inputs.end(); ++i, ++it)
|
||||
{
|
||||
auto & input = *it;
|
||||
if (input.isFinished())
|
||||
continue;
|
||||
|
||||
if (!cursors[i].empty())
|
||||
{
|
||||
// input.setNotNeeded();
|
||||
continue;
|
||||
}
|
||||
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
{
|
||||
all_inputs_has_data = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
auto chunk = input.pull();
|
||||
if (!chunk.hasRows())
|
||||
{
|
||||
|
||||
if (!input.isFinished())
|
||||
all_inputs_has_data = false;
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
updateCursor(std::move(chunk), i);
|
||||
}
|
||||
|
||||
if (!all_inputs_has_data)
|
||||
return Status::NeedData;
|
||||
|
||||
if (has_collation)
|
||||
queue_with_collation = SortingHeap<SortCursorWithCollation>(cursors);
|
||||
else
|
||||
queue_without_collation = SortingHeap<SortCursor>(cursors);
|
||||
|
||||
is_initialized = true;
|
||||
return Status::Ready;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_finished)
|
||||
{
|
||||
|
||||
if (is_port_full)
|
||||
return Status::PortFull;
|
||||
|
||||
for (auto & input : inputs)
|
||||
input.close();
|
||||
|
||||
outputs.front().finish();
|
||||
|
||||
return Status::Finished;
|
||||
}
|
||||
|
||||
if (need_data)
|
||||
{
|
||||
auto & input = *std::next(inputs.begin(), next_input_to_read);
|
||||
if (!input.isFinished())
|
||||
{
|
||||
input.setNeeded();
|
||||
|
||||
if (!input.hasData())
|
||||
return Status::NeedData;
|
||||
|
||||
auto chunk = input.pull();
|
||||
if (!chunk.hasRows() && !input.isFinished())
|
||||
return Status::NeedData;
|
||||
|
||||
updateCursor(std::move(chunk), next_input_to_read);
|
||||
|
||||
if (has_collation)
|
||||
queue_with_collation.push(cursors[next_input_to_read]);
|
||||
else
|
||||
queue_without_collation.push(cursors[next_input_to_read]);
|
||||
}
|
||||
|
||||
need_data = false;
|
||||
}
|
||||
|
||||
if (is_port_full)
|
||||
return Status::PortFull;
|
||||
|
||||
return Status::Ready;
|
||||
}
|
||||
}
|
||||
|
||||
void MergingSortedTransform::work()
|
||||
{
|
||||
if (has_collation)
|
||||
merge(queue_with_collation);
|
||||
else
|
||||
merge(queue_without_collation);
|
||||
}
|
||||
|
||||
template <typename TSortingHeap>
|
||||
void MergingSortedTransform::merge(TSortingHeap & queue)
|
||||
{
|
||||
/// Returns MergeStatus which we should return if we are going to finish now.
|
||||
auto can_read_another_row = [&, this]()
|
||||
{
|
||||
if (limit && merged_data.totalMergedRows() >= limit)
|
||||
{
|
||||
//std::cerr << "Limit reached\n";
|
||||
is_finished = true;
|
||||
return false;
|
||||
}
|
||||
|
||||
return merged_data.mergedRows() < max_block_size;
|
||||
};
|
||||
|
||||
/// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size`
|
||||
while (queue.isValid())
|
||||
{
|
||||
/// Shouldn't happen at first iteration, but check just in case.
|
||||
if (!can_read_another_row())
|
||||
return;
|
||||
|
||||
auto current = queue.current();
|
||||
|
||||
/** And what if the block is totally less or equal than the rest for the current cursor?
|
||||
* Or is there only one data source left in the queue? Then you can take the entire block on current cursor.
|
||||
*/
|
||||
if (current.impl->isFirst()
|
||||
&& (queue.size() == 1
|
||||
|| (queue.size() >= 2 && current.totallyLessOrEquals(queue.nextChild()))))
|
||||
{
|
||||
//std::cerr << "current block is totally less or equals\n";
|
||||
|
||||
/// If there are already data in the current block, we first return it. We'll get here again the next time we call the merge function.
|
||||
if (merged_data.mergedRows() != 0)
|
||||
{
|
||||
//std::cerr << "merged rows is non-zero\n";
|
||||
return;
|
||||
}
|
||||
|
||||
/// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl)
|
||||
size_t source_num = current.impl->order;
|
||||
insertFromChunk(source_num);
|
||||
queue.removeTop();
|
||||
return;
|
||||
}
|
||||
|
||||
//std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n";
|
||||
//std::cerr << "Inserting row\n";
|
||||
merged_data.insertRow(current->all_columns, current->pos);
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
/// Actually, current.impl->order stores source number (i.e. cursors[current.impl->order] == current.impl)
|
||||
RowSourcePart row_source(current.impl->order);
|
||||
out_row_sources_buf->write(row_source.data);
|
||||
}
|
||||
|
||||
if (!current->isLast())
|
||||
{
|
||||
// std::cerr << "moving to next row\n";
|
||||
queue.next();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We will get the next block from the corresponding source, if there is one.
|
||||
queue.removeTop();
|
||||
|
||||
// std::cerr << "It was last row, fetching next block\n";
|
||||
need_data = true;
|
||||
next_input_to_read = current.impl->order;
|
||||
|
||||
if (limit && merged_data.totalMergedRows() >= limit)
|
||||
is_finished = true;
|
||||
|
||||
return;
|
||||
}
|
||||
}
|
||||
is_finished = true;
|
||||
}
|
||||
|
||||
void MergingSortedTransform::insertFromChunk(size_t source_num)
|
||||
{
|
||||
if (source_num >= cursors.size())
|
||||
throw Exception("Logical error in MergingSortedTrandform", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
//std::cerr << "copied columns\n";
|
||||
|
||||
auto num_rows = source_chunks[source_num].getNumRows();
|
||||
|
||||
UInt64 total_merged_rows_after_insertion = merged_data.mergedRows() + num_rows;
|
||||
if (limit && total_merged_rows_after_insertion > limit)
|
||||
{
|
||||
num_rows = total_merged_rows_after_insertion - limit;
|
||||
merged_data.insertFromChunk(std::move(source_chunks[source_num]), num_rows);
|
||||
is_finished = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
merged_data.insertFromChunk(std::move(source_chunks[source_num]), 0);
|
||||
need_data = true;
|
||||
next_input_to_read = source_num;
|
||||
}
|
||||
source_chunks[source_num] = Chunk();
|
||||
|
||||
if (out_row_sources_buf)
|
||||
{
|
||||
RowSourcePart row_source(source_num);
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
out_row_sources_buf->write(row_source.data);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -1,160 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Processors/IProcessor.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/SortCursor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class MergingSortedTransform : public IProcessor
|
||||
{
|
||||
public:
|
||||
MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit = 0,
|
||||
bool quiet = false,
|
||||
bool have_all_inputs = true);
|
||||
|
||||
String getName() const override { return "MergingSortedTransform"; }
|
||||
Status prepare() override;
|
||||
void work() override;
|
||||
|
||||
void addInput();
|
||||
void setHaveAllInputs();
|
||||
|
||||
protected:
|
||||
|
||||
class MergedData
|
||||
{
|
||||
public:
|
||||
explicit MergedData(const Block & header)
|
||||
{
|
||||
columns.reserve(header.columns());
|
||||
for (const auto & column : header)
|
||||
columns.emplace_back(column.type->createColumn());
|
||||
}
|
||||
|
||||
void insertRow(const ColumnRawPtrs & raw_columns, size_t row)
|
||||
{
|
||||
size_t num_columns = raw_columns.size();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
columns[i]->insertFrom(*raw_columns[i], row);
|
||||
|
||||
++total_merged_rows;
|
||||
++merged_rows;
|
||||
}
|
||||
|
||||
void insertFromChunk(Chunk && chunk, size_t limit_rows)
|
||||
{
|
||||
if (merged_rows)
|
||||
throw Exception("Cannot insert to MergedData from Chunk because MergedData is not empty.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto num_rows = chunk.getNumRows();
|
||||
columns = chunk.mutateColumns();
|
||||
if (limit_rows && num_rows > limit_rows)
|
||||
{
|
||||
num_rows = limit_rows;
|
||||
for (auto & column : columns)
|
||||
column = (*column->cut(0, num_rows)).mutate();
|
||||
}
|
||||
|
||||
total_merged_rows += num_rows;
|
||||
merged_rows = num_rows;
|
||||
}
|
||||
|
||||
Chunk pull()
|
||||
{
|
||||
MutableColumns empty_columns;
|
||||
empty_columns.reserve(columns.size());
|
||||
|
||||
for (const auto & column : columns)
|
||||
empty_columns.emplace_back(column->cloneEmpty());
|
||||
|
||||
empty_columns.swap(columns);
|
||||
Chunk chunk(std::move(empty_columns), merged_rows);
|
||||
merged_rows = 0;
|
||||
|
||||
return chunk;
|
||||
}
|
||||
|
||||
UInt64 totalMergedRows() const { return total_merged_rows; }
|
||||
UInt64 mergedRows() const { return merged_rows; }
|
||||
|
||||
private:
|
||||
UInt64 total_merged_rows = 0;
|
||||
UInt64 merged_rows = 0;
|
||||
MutableColumns columns;
|
||||
};
|
||||
|
||||
/// Settings
|
||||
SortDescription description;
|
||||
const size_t max_block_size;
|
||||
UInt64 limit;
|
||||
bool has_collation = false;
|
||||
bool quiet = false;
|
||||
|
||||
std::atomic<bool> have_all_inputs;
|
||||
|
||||
MergedData merged_data;
|
||||
|
||||
/// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step)
|
||||
/// If it is not nullptr then it should be populated during execution
|
||||
WriteBuffer * out_row_sources_buf = nullptr;
|
||||
|
||||
/// Chunks currently being merged.
|
||||
std::vector<Chunk> source_chunks;
|
||||
|
||||
SortCursorImpls cursors;
|
||||
|
||||
SortingHeap<SortCursor> queue_without_collation;
|
||||
SortingHeap<SortCursorWithCollation> queue_with_collation;
|
||||
|
||||
private:
|
||||
|
||||
/// Processor state.
|
||||
bool is_initialized = false;
|
||||
bool is_finished = false;
|
||||
bool need_data = false;
|
||||
size_t next_input_to_read = 0;
|
||||
|
||||
template <typename TSortingHeap>
|
||||
void merge(TSortingHeap & queue);
|
||||
|
||||
void insertFromChunk(size_t source_num);
|
||||
|
||||
void updateCursor(Chunk chunk, size_t source_num)
|
||||
{
|
||||
auto num_rows = chunk.getNumRows();
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = column->convertToFullColumnIfConst();
|
||||
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
|
||||
auto & source_chunk = source_chunks[source_num];
|
||||
|
||||
if (source_chunk.empty())
|
||||
{
|
||||
source_chunk = std::move(chunk);
|
||||
cursors[source_num] = SortCursorImpl(source_chunk.getColumns(), description, source_num);
|
||||
has_collation |= cursors[source_num].has_collation;
|
||||
}
|
||||
else
|
||||
{
|
||||
source_chunk = std::move(chunk);
|
||||
cursors[source_num].reset(source_chunk.getColumns(), {});
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -8,7 +8,7 @@
|
||||
#include <Processors/ISimpleTransform.h>
|
||||
#include <Processors/LimitTransform.h>
|
||||
#include <Processors/printPipeline.h>
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Executors/PipelineExecutor.h>
|
||||
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
|
@ -30,7 +30,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialBlockInputStream.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
|
@ -14,7 +14,7 @@
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataStreams/GraphiteRollupSortedBlockInputStream.h>
|
||||
#include <Processors/Merges/Algorithms/Graphite.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/MergeTree/MergeTreePartsMover.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "MergeTreeDataMergerMutator.h"
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeSequentialBlockInputStream.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
|
||||
#include <Disks/DiskSpaceMonitor.h>
|
||||
@ -12,16 +12,19 @@
|
||||
#include <DataStreams/TTLBlockInputStream.h>
|
||||
#include <DataStreams/DistinctSortedBlockInputStream.h>
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/CollapsingSortedBlockInputStream.h>
|
||||
#include <DataStreams/SummingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ReplacingSortedBlockInputStream.h>
|
||||
#include <DataStreams/GraphiteRollupSortedBlockInputStream.h>
|
||||
#include <DataStreams/AggregatingSortedBlockInputStream.h>
|
||||
#include <DataStreams/VersionedCollapsingSortedBlockInputStream.h>
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DataStreams/ConcatBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/CollapsingSortedTransform.h>
|
||||
#include <Processors/Merges/SummingSortedTransform.h>
|
||||
#include <Processors/Merges/ReplacingSortedTransform.h>
|
||||
#include <Processors/Merges/GraphiteRollupSortedTransform.h>
|
||||
#include <Processors/Merges/AggregatingSortedTransform.h>
|
||||
#include <Processors/Merges/VersionedCollapsingTransform.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/interpolate.h>
|
||||
@ -663,7 +666,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
/** Read from all parts, merge and write into a new one.
|
||||
* In passing, we calculate expression for sorting.
|
||||
*/
|
||||
BlockInputStreams src_streams;
|
||||
Pipes pipes;
|
||||
UInt64 watch_prev_elapsed = 0;
|
||||
|
||||
/// We count total amount of bytes in parts
|
||||
@ -690,18 +693,24 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto input = std::make_unique<MergeTreeSequentialBlockInputStream>(
|
||||
auto input = std::make_unique<MergeTreeSequentialSource>(
|
||||
data, part, merging_column_names, read_with_direct_io, true);
|
||||
|
||||
input->setProgressCallback(
|
||||
MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress));
|
||||
|
||||
BlockInputStreamPtr stream = std::move(input);
|
||||
if (data.hasPrimaryKey() || data.hasSkipIndices())
|
||||
stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
std::make_shared<ExpressionBlockInputStream>(stream, data.sorting_key_and_skip_indices_expr));
|
||||
Pipe pipe(std::move(input));
|
||||
|
||||
src_streams.emplace_back(stream);
|
||||
if (data.hasPrimaryKey() || data.hasSkipIndices())
|
||||
{
|
||||
auto expr = std::make_shared<ExpressionTransform>(pipe.getHeader(), data.sorting_key_and_skip_indices_expr);
|
||||
pipe.addSimpleTransform(std::move(expr));
|
||||
|
||||
auto materializing = std::make_shared<MaterializingTransform>(pipe.getHeader());
|
||||
pipe.addSimpleTransform(std::move(materializing));
|
||||
}
|
||||
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
Names sort_columns = data.sorting_key_columns;
|
||||
@ -709,14 +718,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
||||
Block header = src_streams.at(0)->getHeader();
|
||||
Block header = pipes.at(0).getHeader();
|
||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||
sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1);
|
||||
|
||||
/// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number.
|
||||
/// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part,
|
||||
/// that is going in insertion order.
|
||||
std::shared_ptr<IBlockInputStream> merged_stream;
|
||||
ProcessorPtr merged_transform;
|
||||
|
||||
/// If merge is vertical we cannot calculate it
|
||||
bool blocks_are_granules_size = (merge_alg == MergeAlgorithm::Vertical);
|
||||
@ -725,45 +734,48 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
switch (data.merging_params.mode)
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
merged_stream = std::make_unique<MergingSortedBlockInputStream>(
|
||||
src_streams, sort_description, merge_block_size, 0, rows_sources_write_buf.get(), true, blocks_are_granules_size);
|
||||
merged_transform = std::make_unique<MergingSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size, 0, rows_sources_write_buf.get(), true, blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
merged_stream = std::make_unique<CollapsingSortedBlockInputStream>(
|
||||
src_streams, sort_description, data.merging_params.sign_column,
|
||||
merged_transform = std::make_unique<CollapsingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.sign_column,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
merged_stream = std::make_unique<SummingSortedBlockInputStream>(
|
||||
src_streams, sort_description, data.merging_params.columns_to_sum, merge_block_size);
|
||||
merged_transform = std::make_unique<SummingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.columns_to_sum, merge_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
merged_stream = std::make_unique<AggregatingSortedBlockInputStream>(
|
||||
src_streams, sort_description, merge_block_size);
|
||||
merged_transform = std::make_unique<AggregatingSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
merged_stream = std::make_unique<ReplacingSortedBlockInputStream>(
|
||||
src_streams, sort_description, data.merging_params.version_column,
|
||||
merged_transform = std::make_unique<ReplacingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.version_column,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
merged_stream = std::make_unique<GraphiteRollupSortedBlockInputStream>(
|
||||
src_streams, sort_description, merge_block_size,
|
||||
merged_transform = std::make_unique<GraphiteRollupSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size,
|
||||
data.merging_params.graphite_params, time_of_merge);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
merged_stream = std::make_unique<VersionedCollapsingSortedBlockInputStream>(
|
||||
src_streams, sort_description, data.merging_params.sign_column,
|
||||
merged_transform = std::make_unique<VersionedCollapsingTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.sign_column,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
}
|
||||
|
||||
Pipe merged_pipe(std::move(pipes), std::move(merged_transform));
|
||||
BlockInputStreamPtr merged_stream = std::make_shared<TreeExecutorBlockInputStream>(std::move(merged_pipe));
|
||||
|
||||
if (deduplicate)
|
||||
merged_stream = std::make_shared<DistinctSortedBlockInputStream>(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names());
|
||||
|
||||
@ -861,13 +873,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
MergeStageProgress column_progress(progress_before, column_sizes->columnWeight(column_name));
|
||||
for (size_t part_num = 0; part_num < parts.size(); ++part_num)
|
||||
{
|
||||
auto column_part_stream = std::make_shared<MergeTreeSequentialBlockInputStream>(
|
||||
auto column_part_source = std::make_shared<MergeTreeSequentialSource>(
|
||||
data, parts[part_num], column_names, read_with_direct_io, true);
|
||||
|
||||
column_part_stream->setProgressCallback(
|
||||
column_part_source->setProgressCallback(
|
||||
MergeProgressCallback(merge_entry, watch_prev_elapsed, column_progress));
|
||||
|
||||
column_part_streams[part_num] = std::move(column_part_stream);
|
||||
column_part_streams[part_num] = std::make_shared<TreeExecutorBlockInputStream>(
|
||||
Pipe(std::move(column_part_source)));
|
||||
}
|
||||
|
||||
rows_sources_read_buf.seek(0, 0);
|
||||
|
@ -38,18 +38,9 @@ namespace std
|
||||
}
|
||||
#endif
|
||||
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
#include <DataStreams/FilterBlockInputStream.h>
|
||||
#include <DataStreams/CollapsingFinalBlockInputStream.h>
|
||||
#include <DataStreams/AddingConstColumnBlockInputStream.h>
|
||||
#include <DataStreams/CreatingSetsBlockInputStream.h>
|
||||
#include <DataStreams/MergingSortedBlockInputStream.h>
|
||||
#include <DataStreams/NullBlockInputStream.h>
|
||||
#include <DataStreams/SummingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ReplacingSortedBlockInputStream.h>
|
||||
#include <DataStreams/ReverseBlockInputStream.h>
|
||||
#include <DataStreams/AggregatingSortedBlockInputStream.h>
|
||||
#include <DataStreams/VersionedCollapsingSortedBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -58,7 +49,11 @@ namespace std
|
||||
#include <Processors/Transforms/AddingConstColumnTransform.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/ReverseTransform.h>
|
||||
#include <Processors/Transforms/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/SummingSortedTransform.h>
|
||||
#include <Processors/Merges/AggregatingSortedTransform.h>
|
||||
#include <Processors/Merges/ReplacingSortedTransform.h>
|
||||
#include <Processors/Merges/VersionedCollapsingTransform.h>
|
||||
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include <Processors/ConcatProcessor.h>
|
||||
@ -1096,16 +1091,14 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
};
|
||||
|
||||
BlockInputStreamPtr merged;
|
||||
ProcessorPtr merged_processor;
|
||||
switch (data.merging_params.mode)
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
{
|
||||
auto merged_processor =
|
||||
std::make_shared<MergingSortedTransform>(header, pipes.size(), sort_description, max_block_size);
|
||||
Pipe pipe(std::move(pipes), std::move(merged_processor));
|
||||
pipes = Pipes();
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
return pipes;
|
||||
merged_processor = std::make_shared<MergingSortedTransform>(header, pipes.size(),
|
||||
sort_description, max_block_size);
|
||||
break;
|
||||
}
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
@ -1114,28 +1107,36 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
merged = std::make_shared<SummingSortedBlockInputStream>(streams_to_merge(),
|
||||
merged_processor = std::make_shared<SummingSortedTransform>(header, pipes.size(),
|
||||
sort_description, data.merging_params.columns_to_sum, max_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
merged = std::make_shared<AggregatingSortedBlockInputStream>(streams_to_merge(), sort_description, max_block_size);
|
||||
merged_processor = std::make_shared<AggregatingSortedTransform>(header, pipes.size(),
|
||||
sort_description, max_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing: /// TODO Make ReplacingFinalBlockInputStream
|
||||
merged = std::make_shared<ReplacingSortedBlockInputStream>(streams_to_merge(),
|
||||
merged_processor = std::make_shared<ReplacingSortedTransform>(header, pipes.size(),
|
||||
sort_description, data.merging_params.version_column, max_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing: /// TODO Make VersionedCollapsingFinalBlockInputStream
|
||||
merged = std::make_shared<VersionedCollapsingSortedBlockInputStream>(
|
||||
streams_to_merge(), sort_description, data.merging_params.sign_column, max_block_size);
|
||||
merged_processor = std::make_shared<VersionedCollapsingTransform>(header, pipes.size(),
|
||||
sort_description, data.merging_params.sign_column, max_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (merged_processor)
|
||||
{
|
||||
Pipe pipe(std::move(pipes), std::move(merged_processor));
|
||||
pipes = Pipes();
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
if (merged)
|
||||
pipes.emplace_back(std::make_shared<SourceFromInputStream>(merged));
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Storages/MergeTree/MergeTreeSequentialBlockInputStream.h>
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
|
||||
namespace DB
|
||||
@ -8,16 +8,17 @@ namespace ErrorCodes
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream(
|
||||
MergeTreeSequentialSource::MergeTreeSequentialSource(
|
||||
const MergeTreeData & storage_,
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
MergeTreeData::DataPartPtr data_part_,
|
||||
Names columns_to_read_,
|
||||
bool read_with_direct_io_,
|
||||
bool take_column_types_from_storage,
|
||||
bool quiet)
|
||||
: storage(storage_)
|
||||
, data_part(data_part_)
|
||||
, columns_to_read(columns_to_read_)
|
||||
: SourceWithProgress(storage_.getSampleBlockForColumns(columns_to_read_))
|
||||
, storage(storage_)
|
||||
, data_part(std::move(data_part_))
|
||||
, columns_to_read(std::move(columns_to_read_))
|
||||
, read_with_direct_io(read_with_direct_io_)
|
||||
, mark_cache(storage.global_context.getMarkCache())
|
||||
{
|
||||
@ -35,8 +36,6 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream(
|
||||
|
||||
addTotalRowsApprox(data_part->rows_count);
|
||||
|
||||
header = storage.getSampleBlockForColumns(columns_to_read);
|
||||
|
||||
/// Add columns because we don't want to read empty blocks
|
||||
injectRequiredColumns(storage, data_part, columns_to_read);
|
||||
NamesAndTypesList columns_for_reader;
|
||||
@ -64,33 +63,11 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream(
|
||||
/* uncompressed_cache = */ nullptr, mark_cache.get(), reader_settings);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeSequentialBlockInputStream::fixHeader(Block & header_block) const
|
||||
{
|
||||
/// Types may be different during ALTER (when this stream is used to perform an ALTER).
|
||||
for (const auto & name_type : data_part->getColumns())
|
||||
{
|
||||
if (header_block.has(name_type.name))
|
||||
{
|
||||
auto & elem = header_block.getByName(name_type.name);
|
||||
if (!elem.type->equals(*name_type.type))
|
||||
{
|
||||
elem.type = name_type.type;
|
||||
elem.column = elem.type->createColumn();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Block MergeTreeSequentialBlockInputStream::getHeader() const
|
||||
{
|
||||
return header;
|
||||
}
|
||||
|
||||
Block MergeTreeSequentialBlockInputStream::readImpl()
|
||||
Chunk MergeTreeSequentialSource::generate()
|
||||
try
|
||||
{
|
||||
Block res;
|
||||
auto & header = getPort().getHeader();
|
||||
|
||||
if (!isCancelled() && current_row < data_part->rows_count)
|
||||
{
|
||||
size_t rows_to_read = data_part->index_granularity.getMarkRows(current_mark);
|
||||
@ -98,15 +75,15 @@ try
|
||||
|
||||
auto & sample = reader->getColumns();
|
||||
Columns columns(sample.size());
|
||||
size_t rows_readed = reader->readRows(current_mark, continue_reading, rows_to_read, columns);
|
||||
size_t rows_read = reader->readRows(current_mark, continue_reading, rows_to_read, columns);
|
||||
|
||||
if (rows_readed)
|
||||
if (rows_read)
|
||||
{
|
||||
current_row += rows_readed;
|
||||
current_mark += (rows_to_read == rows_readed);
|
||||
current_row += rows_read;
|
||||
current_mark += (rows_to_read == rows_read);
|
||||
|
||||
bool should_evaluate_missing_defaults = false;
|
||||
reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_readed);
|
||||
reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_read);
|
||||
|
||||
if (should_evaluate_missing_defaults)
|
||||
{
|
||||
@ -115,20 +92,21 @@ try
|
||||
|
||||
reader->performRequiredConversions(columns);
|
||||
|
||||
res = header.cloneEmpty();
|
||||
|
||||
/// Reorder columns and fill result block.
|
||||
size_t num_columns = sample.size();
|
||||
Columns res_columns;
|
||||
res_columns.reserve(num_columns);
|
||||
|
||||
auto it = sample.begin();
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
if (res.has(it->name))
|
||||
res.getByName(it->name).column = std::move(columns[i]);
|
||||
if (header.has(it->name))
|
||||
res_columns.emplace_back(std::move(columns[i]));
|
||||
|
||||
++it;
|
||||
}
|
||||
|
||||
res.checkNumberOfRows();
|
||||
return Chunk(std::move(res_columns), rows_read);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -136,7 +114,7 @@ try
|
||||
finish();
|
||||
}
|
||||
|
||||
return res;
|
||||
return {};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -146,8 +124,7 @@ catch (...)
|
||||
throw;
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeSequentialBlockInputStream::finish()
|
||||
void MergeTreeSequentialSource::finish()
|
||||
{
|
||||
/** Close the files (before destroying the object).
|
||||
* When many sources are created, but simultaneously reading only a few of them,
|
||||
@ -157,7 +134,6 @@ void MergeTreeSequentialBlockInputStream::finish()
|
||||
data_part.reset();
|
||||
}
|
||||
|
||||
|
||||
MergeTreeSequentialBlockInputStream::~MergeTreeSequentialBlockInputStream() = default;
|
||||
MergeTreeSequentialSource::~MergeTreeSequentialSource() = default;
|
||||
|
||||
}
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
@ -9,40 +9,33 @@ namespace DB
|
||||
{
|
||||
|
||||
/// Lightweight (in terms of logic) stream for reading single part from MergeTree
|
||||
class MergeTreeSequentialBlockInputStream : public IBlockInputStream
|
||||
class MergeTreeSequentialSource : public SourceWithProgress
|
||||
{
|
||||
public:
|
||||
MergeTreeSequentialBlockInputStream(
|
||||
MergeTreeSequentialSource(
|
||||
const MergeTreeData & storage_,
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
MergeTreeData::DataPartPtr data_part_,
|
||||
Names columns_to_read_,
|
||||
bool read_with_direct_io_,
|
||||
bool take_column_types_from_storage,
|
||||
bool quiet = false
|
||||
);
|
||||
|
||||
~MergeTreeSequentialBlockInputStream() override;
|
||||
~MergeTreeSequentialSource() override;
|
||||
|
||||
String getName() const override { return "MergeTreeSequentialBlockInputStream"; }
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
/// Closes readers and unlock part locks
|
||||
void finish();
|
||||
String getName() const override { return "MergeTreeSequentialSource"; }
|
||||
|
||||
size_t getCurrentMark() const { return current_mark; }
|
||||
|
||||
size_t getCurrentRow() const { return current_row; }
|
||||
|
||||
protected:
|
||||
Block readImpl() override;
|
||||
Chunk generate() override;
|
||||
|
||||
private:
|
||||
|
||||
const MergeTreeData & storage;
|
||||
|
||||
Block header;
|
||||
|
||||
/// Data part will not be removed if the pointer owns it
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
|
||||
@ -52,7 +45,7 @@ private:
|
||||
/// Should read using direct IO
|
||||
bool read_with_direct_io;
|
||||
|
||||
Logger * log = &Logger::get("MergeTreeSequentialBlockInputStream");
|
||||
Logger * log = &Logger::get("MergeTreeSequentialSource");
|
||||
|
||||
std::shared_ptr<MarkCache> mark_cache;
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
@ -65,8 +58,8 @@ private:
|
||||
size_t current_row = 0;
|
||||
|
||||
private:
|
||||
void fixHeader(Block & header_block) const;
|
||||
|
||||
/// Closes readers and unlock part locks
|
||||
void finish();
|
||||
};
|
||||
|
||||
}
|
@ -14,6 +14,8 @@
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
Loading…
Reference in New Issue
Block a user