2013-10-01 18:09:31 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Row.h>
|
|
|
|
#include <Core/ColumnNumbers.h>
|
|
|
|
#include <DataStreams/MergingSortedBlockInputStream.h>
|
2017-10-09 05:56:22 +00:00
|
|
|
#include <AggregateFunctions/IAggregateFunction.h>
|
2013-10-01 18:09:31 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** 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.
|
2013-10-01 18:09:31 +00:00
|
|
|
*/
|
|
|
|
class SummingSortedBlockInputStream : public MergingSortedBlockInputStream
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
SummingSortedBlockInputStream(BlockInputStreams inputs_,
|
|
|
|
const SortDescription & description_,
|
2017-05-13 22:19:04 +00:00
|
|
|
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names_to_sum_,
|
|
|
|
size_t max_block_size_)
|
|
|
|
: MergingSortedBlockInputStream(inputs_, description_, max_block_size_), column_names_to_sum(column_names_to_sum_)
|
|
|
|
{
|
|
|
|
}
|
2013-10-01 18:09:31 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return "SummingSorted"; }
|
2013-10-01 18:09:31 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getID() const override;
|
2013-10-01 18:09:31 +00:00
|
|
|
|
|
|
|
protected:
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Can return 1 more records than max_block_size.
|
2017-04-01 07:20:54 +00:00
|
|
|
Block readImpl() override;
|
2013-10-01 18:09:31 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
Logger * log = &Logger::get("SummingSortedBlockInputStream");
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Read up to the end.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool finished = false;
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Columns with which numbers should be summed.
|
|
|
|
Names column_names_to_sum; /// If set, it is converted to column_numbers_to_sum when initialized.
|
2017-10-09 05:56:22 +00:00
|
|
|
ColumnNumbers column_numbers_not_to_aggregate;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** 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...).
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Example:
|
2017-04-01 07:20:54 +00:00
|
|
|
* [(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)]
|
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
|
2017-10-12 21:54:49 +00:00
|
|
|
/// Stores aggregation function, state, and columns to be used as function arguments
|
2017-10-09 05:56:22 +00:00
|
|
|
struct AggregateDescription
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-10-09 05:56:22 +00:00
|
|
|
AggregateFunctionPtr function;
|
2017-11-16 19:03:32 +00:00
|
|
|
IAggregateFunction::AddFunc add_function = nullptr;
|
2017-10-09 05:56:22 +00:00
|
|
|
std::vector<size_t> column_numbers;
|
|
|
|
ColumnPtr merged_column;
|
|
|
|
std::vector<char> state;
|
|
|
|
bool created = false;
|
2017-10-12 21:54:49 +00:00
|
|
|
|
|
|
|
/// Explicitly destroy aggregation state if the stream is terminated
|
|
|
|
~AggregateDescription()
|
|
|
|
{
|
|
|
|
if (created)
|
|
|
|
function->destroy(state.data());
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2017-10-12 22:09:06 +00:00
|
|
|
/// Stores numbers of key-columns and value-columns.
|
|
|
|
struct MapDescription
|
|
|
|
{
|
|
|
|
std::vector<size_t> key_col_nums;
|
|
|
|
std::vector<size_t> val_col_nums;
|
|
|
|
};
|
|
|
|
|
2017-10-09 05:56:22 +00:00
|
|
|
std::vector<AggregateDescription> columns_to_aggregate;
|
2017-10-12 22:09:06 +00:00
|
|
|
std::vector<MapDescription> maps_to_sum;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
RowRef current_key; /// The current primary key.
|
|
|
|
RowRef next_key; /// The primary key of the next row.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Row current_row;
|
2017-11-16 19:03:32 +00:00
|
|
|
bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-16 19:03:32 +00:00
|
|
|
bool output_is_non_empty = false; /// Have we given out at least one row as a result.
|
|
|
|
size_t merged_rows = 0; /// Number of rows merged into current result block
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** We support two different cursors - with Collation and without.
|
|
|
|
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename TSortCursor>
|
2017-04-01 07:20:54 +00:00
|
|
|
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
|
|
|
|
|
2017-11-16 19:03:32 +00:00
|
|
|
/// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero".
|
|
|
|
/// If force_insertion=true, then the row will be inserted even if it is "zero"
|
|
|
|
void insertCurrentRowIfNeeded(ColumnPlainPtrs & merged_columns, bool force_insertion);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-16 19:03:32 +00:00
|
|
|
/// Returns true is merge result is not empty
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename TSortCursor>
|
2017-10-12 22:09:06 +00:00
|
|
|
bool mergeMap(const MapDescription & map, Row & row, TSortCursor & cursor);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-16 19:03:32 +00:00
|
|
|
// Add the row under the cursor to the `row`.
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename TSortCursor>
|
2017-11-16 19:03:32 +00:00
|
|
|
void addRow(Row & row, TSortCursor & cursor);
|
2013-10-01 18:09:31 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|