Merge pull request #9885 from ClickHouse/sorting-processors

Sorting processors
This commit is contained in:
alexey-milovidov 2020-04-20 09:30:03 +03:00 committed by GitHub
commit 83e17ae327
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
62 changed files with 3787 additions and 2856 deletions

View File

@ -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)

View File

@ -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();
}
}
}

View File

@ -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;
};
};
}

View File

@ -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;
}
}

View File

@ -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();
};
}

View File

@ -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);
};
}

View File

@ -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;
}

View File

@ -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.

View File

@ -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;
}
}

View File

@ -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);
};
}

View File

@ -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);
}
}
}
}
}

View File

@ -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);
};
}

View File

@ -3,6 +3,7 @@
#include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Columns/ColumnConst.h>
namespace DB
{

View File

@ -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;
}
}

View File

@ -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>

View File

@ -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);

View File

@ -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>

View 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"; }
};
}

View 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);
}
}

View File

@ -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;
};
}

View 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);
}
}

View 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();
};
}

View File

@ -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);
};
}

View 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 *>;
}

View File

@ -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());
}
}

View 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);
};
}

View 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;
}

View File

@ -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]);
}
}

View File

@ -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;
};
}

View File

@ -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]);
}
}

View File

@ -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]); }
};
}

View 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;
};
}

View 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;
}
}

View 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);
};
}

View File

@ -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);
}
}

View 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();
};
}

View 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);
}
};
}

View 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);
}
}

View 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;
};
}

View File

@ -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);
}
}

View File

@ -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);
};
}

View 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"; }
};
}

View 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"; }
};
}

View 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;
}
}

View 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;
};
}

View 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());
}
}

View 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;
};
}

View 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"; }
};
}

View 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"; }
};
}

View 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"; }
};
}

View File

@ -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);

View File

@ -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);
}
}
}

View File

@ -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(), {});
}
}
};
}

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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);

View File

@ -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));

View File

@ -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>

View File

@ -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;
}

View File

@ -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();
};
}

View File

@ -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>