ClickHouse/dbms/include/DB/DataStreams/MergingSortedBlockInputStream.h

101 lines
3.0 KiB
C
Raw Normal View History

2012-07-25 19:53:43 +00:00
#pragma once
#include <queue>
#include <Yandex/logger_useful.h>
2012-07-25 19:53:43 +00:00
#include <DB/Core/SortDescription.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
/** Соединяет несколько сортированных потоков в один.
*/
class MergingSortedBlockInputStream : public IProfilingBlockInputStream
{
public:
/// limit - если не 0, то можно выдать только первые limit строк в сортированном порядке.
MergingSortedBlockInputStream(BlockInputStreams inputs_, SortDescription & description_, size_t max_block_size_, size_t limit_ = 0)
: description(description_), max_block_size(max_block_size_), limit(limit_), total_merged_rows(0), first(true), has_collation(false),
2013-05-04 04:05:15 +00:00
num_columns(0), source_blocks(inputs_.size()), cursors(inputs_.size()), log(&Logger::get("MergingSortedBlockInputStream"))
2012-07-25 19:53:43 +00:00
{
2013-05-04 04:05:15 +00:00
children.insert(children.end(), inputs_.begin(), inputs_.end());
2012-07-25 19:53:43 +00:00
}
String getName() const { return "MergingSortedBlockInputStream"; }
String getID() const
{
std::stringstream res;
res << "MergingSorted(";
Strings children_ids(children.size());
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Порядок не имеет значения.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
res << (i == 0 ? "" : ", ") << children_ids[i];
for (size_t i = 0; i < description.size(); ++i)
res << ", " << description[i].getID();
res << ")";
return res.str();
}
2012-08-14 20:33:37 +00:00
protected:
2012-10-20 02:10:47 +00:00
Block readImpl();
2013-09-13 20:33:09 +00:00
void readSuffixImpl();
2012-10-20 02:10:47 +00:00
2012-08-14 20:33:37 +00:00
/// Инициализирует очередь и следующий блок результата.
void init(Block & merged_block, ColumnPlainPtrs & merged_columns);
/// Достаёт из источника, соответствующего current следующий блок.
template <typename TSortCursor>
2013-05-28 16:56:05 +00:00
void fetchNextBlock(const TSortCursor & current, std::priority_queue<TSortCursor> & queue);
2012-08-14 20:33:37 +00:00
2012-07-25 19:53:43 +00:00
SortDescription description;
size_t max_block_size;
size_t limit;
size_t total_merged_rows;
2012-07-25 19:53:43 +00:00
bool first;
2013-05-28 16:56:05 +00:00
bool has_collation;
2012-07-25 19:53:43 +00:00
/// Текущие сливаемые блоки.
size_t num_columns;
Blocks source_blocks;
typedef std::vector<SortCursorImpl> CursorImpls;
CursorImpls cursors;
2012-07-25 19:53:43 +00:00
typedef std::priority_queue<SortCursor> Queue;
Queue queue;
2013-05-28 16:56:05 +00:00
typedef std::priority_queue<SortCursorWithCollation> QueueWithCollation;
QueueWithCollation queue_with_collation;
2012-08-01 19:55:05 +00:00
2012-08-14 20:33:37 +00:00
private:
2013-05-28 16:56:05 +00:00
/** Делаем поддержку двух разных курсоров - с Collation и без.
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
*/
template <typename TSortCursor>
2013-05-28 16:56:05 +00:00
void initQueue(std::priority_queue<TSortCursor> & queue);
template <typename TSortCursor>
2013-05-28 16:56:05 +00:00
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
2012-08-01 19:55:05 +00:00
Logger * log;
2012-07-25 19:53:43 +00:00
};
}