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

154 lines
4.4 KiB
C
Raw Normal View History

2012-07-25 19:53:43 +00:00
#pragma once
#include <queue>
2015-09-29 19:19:54 +00:00
#include <common/logger_useful.h>
2013-10-01 20:38:01 +00:00
#include <DB/Core/Row.h>
2012-07-25 19:53:43 +00:00
#include <DB/Core/SortDescription.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CORRUPTED_DATA;
}
2012-07-25 19:53:43 +00:00
/** Соединяет несколько сортированных потоков в один.
*/
class MergingSortedBlockInputStream : public IProfilingBlockInputStream
{
public:
/// limit - если не 0, то можно выдать только первые limit строк в сортированном порядке.
2014-03-14 17:03:52 +00:00
MergingSortedBlockInputStream(BlockInputStreams inputs_, const SortDescription & description_, size_t max_block_size_, size_t limit_ = 0)
: description(description_), max_block_size(max_block_size_), limit(limit_),
source_blocks(inputs_.size()), cursors(inputs_.size())
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 override { return "MergingSorted"; }
2012-07-25 19:53:43 +00:00
String getID() const override
{
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:
Block readImpl() override;
void readSuffixImpl() override;
2012-08-14 20:33:37 +00:00
/// Инициализирует очередь и следующий блок результата.
void init(Block & merged_block, ColumnPlainPtrs & merged_columns);
2012-08-14 20:33:37 +00:00
/// Достаёт из источника, соответствующего current следующий блок.
template <typename TSortCursor>
2013-05-28 16:56:05 +00:00
void fetchNextBlock(const TSortCursor & current, std::priority_queue<TSortCursor> & queue);
2012-07-25 19:53:43 +00:00
SortDescription description;
size_t max_block_size;
size_t limit;
size_t total_merged_rows = 0;
2012-07-25 19:53:43 +00:00
bool first = true;
bool has_collation = false;
2012-07-25 19:53:43 +00:00
/// Текущие сливаемые блоки.
size_t num_columns = 0;
2012-07-25 19:53:43 +00:00
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
2013-10-01 20:38:01 +00:00
2014-05-26 16:11:20 +00:00
/// Эти методы используются в Collapsing/Summing/Aggregating SortedBlockInputStream-ах.
2013-10-01 20:38:01 +00:00
2014-04-11 16:56:49 +00:00
/// Сохранить строчку, на которую указывает cursor, в row.
2013-10-01 20:38:01 +00:00
template<class 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__);
/// Узнаем имя столбца и бросим исключение поинформативней.
String column_name;
for (const Block & block : source_blocks)
{
if (i < block.columns())
{
column_name = block.getByPosition(i).name;
break;
}
}
throw DB::Exception("MergingSortedBlockInputStream failed to read row " + toString(cursor->pos)
+ " of column " + toString(i) + (column_name.empty() ? "" : " (" + column_name + ")"),
ErrorCodes::CORRUPTED_DATA);
}
}
2013-10-01 20:38:01 +00:00
}
/// Сохранить первичный ключ, на который указывает cursor в row.
template<class TSortCursor>
void setPrimaryKey(Row & row, TSortCursor & cursor)
{
for (size_t i = 0; i < cursor->sort_columns_size; ++i)
cursor->sort_columns[i]->get(cursor->pos, row[i]);
}
2012-08-14 20:33:37 +00:00
private:
2013-05-28 16:56:05 +00:00
/** Делаем поддержку двух разных курсоров - с Collation и без.
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
*/
template <typename TSortCursor>
void initQueue(std::priority_queue<TSortCursor> & queue);
template <typename TSortCursor>
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
Logger * log = &Logger::get("MergingSortedBlockInputStream");
2015-01-18 08:25:56 +00:00
/// Прочитали до конца.
bool finished = false;
2012-07-25 19:53:43 +00:00
};
}