ClickHouse/dbms/src/DataStreams/MergingSortedBlockInputStream.h

194 lines
5.9 KiB
C++
Raw Normal View History

2012-07-25 19:53:43 +00:00
#pragma once
#include <queue>
#include <boost/smart_ptr/intrusive_ptr.hpp>
2015-09-29 19:19:54 +00:00
#include <common/logger_useful.h>
2019-04-19 13:38:25 +00:00
#include <Common/SharedBlockRowRef.h>
#include <Core/Row.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>
2012-07-25 19:53:43 +00:00
2018-06-05 19:46:49 +00:00
#include <IO/WriteHelpers.h>
#include <DataStreams/IBlockInputStream.h>
2012-07-25 19:53:43 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int CORRUPTED_DATA;
}
/** Merges several sorted streams into one sorted stream.
2012-07-25 19:53:43 +00:00
*/
class MergingSortedBlockInputStream : public IBlockInputStream
2012-07-25 19:53:43 +00:00
{
public:
/** limit - if isn't 0, then we can produce only first limit rows in sorted order.
* out_row_sources - if isn't nullptr, then at the end of execution it should contain part numbers of each readed row (and needed flag)
* quiet - don't log profiling info
*/
MergingSortedBlockInputStream(
const BlockInputStreams & inputs_, const SortDescription & description_, size_t max_block_size_,
2019-03-20 16:18:13 +00:00
UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool average_block_sizes_ = false);
2012-07-25 19:53:43 +00:00
String getName() const override { return "MergingSorted"; }
2012-07-25 19:53:43 +00:00
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() const override { return header; }
2012-08-14 20:33:37 +00:00
protected:
2019-04-01 11:29:31 +00:00
/// 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
{
2019-03-28 11:12:59 +00:00
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_)
{}
2019-04-01 11:29:31 +00:00
/// add single row from block size `granularity`
2019-03-28 11:12:59 +00:00
void addRowWithGranularity(size_t granularity)
{
2019-03-28 11:12:59 +00:00
sum_blocks_granularity += granularity;
sum_rows_count++;
}
2019-04-01 11:29:31 +00:00
/// check that sum_rows_count is enough
bool checkStop() const;
2019-04-01 11:29:31 +00:00
bool empty() const
{
2019-03-28 11:12:59 +00:00
return sum_blocks_granularity == 0;
}
};
Block readImpl() override;
void readSuffixImpl() override;
/// Initializes the queue and the columns of next result block.
void init(MutableColumns & merged_columns);
2017-05-13 22:19:04 +00:00
/// Gets the next block from the source corresponding to the `current`.
template <typename TSortCursor>
void fetchNextBlock(const TSortCursor & current, std::priority_queue<TSortCursor> & queue);
Block header;
const SortDescription description;
const size_t max_block_size;
2019-02-10 15:17:45 +00:00
UInt64 limit;
UInt64 total_merged_rows = 0;
bool first = true;
bool has_collation = false;
bool quiet = false;
2019-03-20 16:18:13 +00:00
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;
2017-05-13 22:19:04 +00:00
/// Blocks currently being merged.
size_t num_columns = 0;
std::vector<SharedBlockPtr> source_blocks;
SortCursorImpls cursors;
using Queue = std::priority_queue<SortCursor>;
Queue queue_without_collation;
using QueueWithCollation = std::priority_queue<SortCursorWithCollation>;
QueueWithCollation queue_with_collation;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
/// 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;
2017-05-13 22:19:04 +00:00
/// These methods are used in Collapsing/Summing/Aggregating... SortedBlockInputStream-s.
2017-05-13 22:19:04 +00:00
/// Save the row pointed to by cursor in `row`.
2017-09-15 12:16:12 +00:00
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__);
2017-05-13 22:19:04 +00:00
/// 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);
}
}
}
2017-09-15 12:16:12 +00:00
template <typename TSortCursor>
2019-04-19 13:38:25 +00:00
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;
}
2016-04-13 03:56:22 +00:00
2017-09-15 12:16:12 +00:00
template <typename TSortCursor>
2019-04-19 13:38:25 +00:00
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;
}
2016-04-13 03:56:22 +00:00
2012-08-14 20:33:37 +00:00
private:
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.
*/
template <typename TSortCursor>
void initQueue(std::priority_queue<TSortCursor> & queue);
template <typename TSortCursor>
void merge(MutableColumns & merged_columns, std::priority_queue<TSortCursor> & queue);
Logger * log = &Logger::get("MergingSortedBlockInputStream");
2015-01-18 08:25:56 +00:00
2017-05-13 22:19:04 +00:00
/// Read is finished.
bool finished = false;
2012-07-25 19:53:43 +00:00
};
}