ClickHouse/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp

276 lines
9.7 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/MergeTreeBaseBlockInputStream.h>
2018-02-13 19:34:15 +00:00
#include <Storages/MergeTree/MergeTreeRangeReader.h>
#include <Storages/MergeTree/MergeTreeReader.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
#include <Columns/FilterDescription.h>
#include <Columns/ColumnArray.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
2017-06-06 17:18:32 +00:00
#include <ext/range.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
extern const int LOGICAL_ERROR;
}
MergeTreeBaseBlockInputStream::MergeTreeBaseBlockInputStream(
MergeTreeData & storage,
const ExpressionActionsPtr & prewhere_actions,
const String & prewhere_column_name,
size_t max_block_size_rows,
size_t preferred_block_size_bytes,
size_t preferred_max_column_in_block_size_bytes,
size_t min_bytes_to_use_direct_io,
size_t max_read_buffer_size,
bool use_uncompressed_cache,
bool save_marks_in_cache,
const Names & virt_column_names)
:
storage(storage),
prewhere_actions(prewhere_actions),
prewhere_column_name(prewhere_column_name),
max_block_size_rows(max_block_size_rows),
preferred_block_size_bytes(preferred_block_size_bytes),
preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes),
min_bytes_to_use_direct_io(min_bytes_to_use_direct_io),
max_read_buffer_size(max_read_buffer_size),
use_uncompressed_cache(use_uncompressed_cache),
save_marks_in_cache(save_marks_in_cache),
virt_column_names(virt_column_names),
max_block_size_marks(max_block_size_rows / storage.index_granularity)
{
}
Block MergeTreeBaseBlockInputStream::readImpl()
{
Block res;
while (!res && !isCancelled())
{
if (!task && !getNewTask())
break;
res = readFromPart();
if (res)
injectVirtualColumns(res);
2017-06-15 17:01:13 +00:00
if (task->isFinished())
task.reset();
}
return res;
}
Block MergeTreeBaseBlockInputStream::readFromPart()
{
Block res;
if (task->size_predictor)
task->size_predictor->startBlock();
const auto max_block_size_rows = this->max_block_size_rows;
const auto preferred_block_size_bytes = this->preferred_block_size_bytes;
const auto preferred_max_column_in_block_size_bytes = this->preferred_max_column_in_block_size_bytes;
const auto index_granularity = storage.index_granularity;
const double min_filtration_ratio = 0.00001;
auto estimateNumRows = [preferred_block_size_bytes, max_block_size_rows,
index_granularity, preferred_max_column_in_block_size_bytes, min_filtration_ratio](
2018-02-13 19:34:15 +00:00
MergeTreeReadTask & task, MergeTreePrewhereRangeReader & reader)
{
if (!task.size_predictor)
2017-07-12 11:40:00 +00:00
return max_block_size_rows;
2017-07-24 15:06:32 +00:00
/// Calculates number of rows will be read using preferred_block_size_bytes.
/// Can't be less than index_granularity.
size_t rows_to_read = task.size_predictor->estimateNumRows(preferred_block_size_bytes);
if (!rows_to_read)
return rows_to_read;
rows_to_read = std::max(index_granularity, rows_to_read);
if (preferred_max_column_in_block_size_bytes)
{
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
size_t rows_to_read_for_max_size_column
= task.size_predictor->estimateNumRowsForMaxSizeColumn(preferred_max_column_in_block_size_bytes);
double filtration_ratio = std::max(min_filtration_ratio, 1.0 - task.size_predictor->filtered_rows_ratio);
size_t rows_to_read_for_max_size_column_with_filtration
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
2017-07-21 18:02:02 +00:00
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than index_granularity.
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
}
size_t unread_rows_in_current_granule = reader.numPendingRowsInCurrentGranule();
if (unread_rows_in_current_granule >= rows_to_read)
return rows_to_read;
size_t granule_to_read = (rows_to_read + reader.numReadRowsInCurrentGranule() + index_granularity / 2) / index_granularity;
return index_granularity * granule_to_read - reader.numReadRowsInCurrentGranule();
};
if (prewhere_actions)
{
do
{
2018-02-13 19:34:15 +00:00
auto processNextRange = [this]()
{
2018-02-13 19:34:15 +00:00
const auto & range = task->mark_ranges.back();
task->pre_range_reader = pre_reader->readRange(
range.begin, range.end, nullptr, prewhere_actions,
&prewhere_column_name, &task->ordered_names, task->should_reorder);
task->range_reader = reader->readRange(
range.begin, range.end, &task->pre_range_reader,
nullptr, nullptr, &task->ordered_names, true);
task->mark_ranges.pop_back();
};
2018-02-13 19:34:15 +00:00
auto resetRangeReaders = [this]()
{
2018-02-13 19:34:15 +00:00
task->range_reader.reset();
task->pre_range_reader.reset();
};
if (!task->range_reader)
processNextRange();
/// FIXME: size prediction model is updated by filtered rows, but it predicts size of unfiltered rows also
2018-02-13 19:34:15 +00:00
size_t recommended_rows = estimateNumRows(*task, task->range_reader);
if (res && recommended_rows < 1)
break;
2018-02-13 19:34:15 +00:00
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), std::min(max_block_size_rows, recommended_rows));
2018-02-13 19:34:15 +00:00
size_t total_filtered_rows = 0;
2018-02-13 19:34:15 +00:00
while (!task->isFinished() && space_left && !isCancelled())
{
2018-02-13 19:34:15 +00:00
if (!task->range_reader)
processNextRange();
2018-02-13 19:34:15 +00:00
size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left);
size_t filtered_rows = 0;
2018-02-13 19:34:15 +00:00
auto read_result = task->range_reader.read(res, rows_to_read);
if (task->size_predictor)
{
task->size_predictor->updateFilteredRowsRation(
read_result.getNumAddedRows() + read_result.getNumFilteredRows(),
read_result.getNumFilteredRows());
}
2018-02-13 19:34:15 +00:00
total_filtered_rows += filtered_rows;
2018-02-13 19:34:15 +00:00
if (task->range_reader.isReadingFinished())
resetRangeReaders();
2018-02-13 19:34:15 +00:00
space_left -= rows_to_read;
}
2018-02-13 19:34:15 +00:00
if (res.rows() == 0)
{
res.clear();
return res;
}
2018-02-13 19:34:15 +00:00
progressImpl({ res.rows(), res.bytes() });
2018-02-13 19:34:15 +00:00
if (task->remove_prewhere_column && res.has(prewhere_column_name))
res.erase(prewhere_column_name);
2018-02-13 19:34:15 +00:00
if (task->size_predictor && res)
task->size_predictor->update(res);
2018-02-13 19:34:15 +00:00
res.checkNumberOfRows();
}
2017-06-15 17:01:13 +00:00
while (!task->isFinished() && !res && !isCancelled());
}
else
{
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), max_block_size_rows);
2017-06-15 17:01:13 +00:00
while (!task->isFinished() && space_left && !isCancelled())
{
2018-02-13 19:34:15 +00:00
if (!task->range_reader)
{
2017-06-14 10:50:22 +00:00
auto & range = task->mark_ranges.back();
2018-02-13 19:34:15 +00:00
task->range_reader = reader->readRange(range.begin, range.end, nullptr,
nullptr, nullptr, &task->ordered_names, task->should_reorder);
2017-06-14 10:50:22 +00:00
task->mark_ranges.pop_back();
}
2018-02-13 19:34:15 +00:00
size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left);
size_t recommended_rows = estimateNumRows(*task, task->range_reader);
if (res && recommended_rows < 1)
break;
2018-02-13 19:34:15 +00:00
rows_to_read = std::min(rows_to_read, std::max(static_cast<decltype(recommended_rows)>(1), recommended_rows));
2018-02-13 19:34:15 +00:00
auto read_result = task->range_reader.read(res, rows_to_read);
if (task->size_predictor)
{
2018-02-13 19:34:15 +00:00
task->size_predictor->updateFilteredRowsRation(
read_result.getNumAddedRows() + read_result.getNumFilteredRows(),
read_result.getNumFilteredRows());
}
2018-02-13 19:34:15 +00:00
if (task->range_reader.isReadingFinished())
task->range_reader.reset();
2018-02-13 19:34:15 +00:00
if (task->size_predictor && res)
task->size_predictor->update(res);
2018-02-13 19:34:15 +00:00
space_left -= rows_to_read;
}
progressImpl({ res.rows(), res.bytes() });
}
return res;
}
void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block)
{
const auto rows = block.rows();
/// add virtual columns
/// Except _sample_factor, which is added from the outside.
if (!virt_column_names.empty())
{
for (const auto & virt_column_name : virt_column_names)
{
if (virt_column_name == "_part")
{
block.insert(ColumnWithTypeAndName{
2017-12-10 22:44:04 +00:00
DataTypeString().createColumnConst(rows, task->data_part->name)->convertToFullColumnIfConst(),
std::make_shared<DataTypeString>(),
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
virt_column_name});
}
else if (virt_column_name == "_part_index")
{
block.insert(ColumnWithTypeAndName{
2017-12-10 22:44:04 +00:00
DataTypeUInt64().createColumnConst(rows, static_cast<UInt64>(task->part_index_in_query))->convertToFullColumnIfConst(),
std::make_shared<DataTypeUInt64>(),
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
virt_column_name});
}
}
}
}
MergeTreeBaseBlockInputStream::~MergeTreeBaseBlockInputStream() = default;
}