2017-12-15 20:48:46 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeBaseBlockInputStream.h>
|
2018-02-13 19:34:15 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
2017-04-06 17:21:45 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeReader.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
2017-12-15 20:48:46 +00:00
|
|
|
#include <Columns/FilterDescription.h>
|
2017-12-18 05:37:20 +00:00
|
|
|
#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>
|
2018-04-06 13:58:06 +00:00
|
|
|
#include <DataTypes/DataTypeNothing.h>
|
2017-04-06 17:21:45 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
|
2017-08-01 13:04:48 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-04-06 17:21:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
MergeTreeBaseBlockInputStream::MergeTreeBaseBlockInputStream(
|
|
|
|
MergeTreeData & storage,
|
2018-04-11 14:31:54 +00:00
|
|
|
const PrewhereInfoPtr & prewhere_info,
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 max_block_size_rows,
|
|
|
|
UInt64 preferred_block_size_bytes,
|
|
|
|
UInt64 preferred_max_column_in_block_size_bytes,
|
|
|
|
UInt64 min_bytes_to_use_direct_io,
|
|
|
|
UInt64 max_read_buffer_size,
|
2017-04-06 17:21:45 +00:00
|
|
|
bool use_uncompressed_cache,
|
|
|
|
bool save_marks_in_cache,
|
|
|
|
const Names & virt_column_names)
|
|
|
|
:
|
|
|
|
storage(storage),
|
2018-04-11 14:31:54 +00:00
|
|
|
prewhere_info(prewhere_info),
|
2017-04-06 17:21:45 +00:00
|
|
|
max_block_size_rows(max_block_size_rows),
|
|
|
|
preferred_block_size_bytes(preferred_block_size_bytes),
|
2017-06-30 16:28:27 +00:00
|
|
|
preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes),
|
2017-04-06 17:21:45 +00:00
|
|
|
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())
|
2017-04-10 17:10:33 +00:00
|
|
|
task.reset();
|
2017-04-10 14:06:44 +00:00
|
|
|
}
|
|
|
|
|
2017-04-06 17:21:45 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Block MergeTreeBaseBlockInputStream::readFromPart()
|
|
|
|
{
|
2017-04-07 11:43:24 +00:00
|
|
|
if (task->size_predictor)
|
|
|
|
task->size_predictor->startBlock();
|
|
|
|
|
2017-07-11 09:32:39 +00:00
|
|
|
const auto max_block_size_rows = this->max_block_size_rows;
|
2017-06-20 10:12:20 +00:00
|
|
|
const auto preferred_block_size_bytes = this->preferred_block_size_bytes;
|
2017-07-29 21:52:35 +00:00
|
|
|
const auto preferred_max_column_in_block_size_bytes = this->preferred_max_column_in_block_size_bytes;
|
2017-06-21 17:19:35 +00:00
|
|
|
const auto index_granularity = storage.index_granularity;
|
2017-06-30 16:28:27 +00:00
|
|
|
const double min_filtration_ratio = 0.00001;
|
2017-06-21 17:19:35 +00:00
|
|
|
|
2017-07-11 09:32:39 +00:00
|
|
|
auto estimateNumRows = [preferred_block_size_bytes, max_block_size_rows,
|
2017-06-30 16:28:27 +00:00
|
|
|
index_granularity, preferred_max_column_in_block_size_bytes, min_filtration_ratio](
|
2018-02-20 11:45:58 +00:00
|
|
|
MergeTreeReadTask & task, MergeTreeRangeReader & reader)
|
2017-06-21 17:19:35 +00:00
|
|
|
{
|
|
|
|
if (!task.size_predictor)
|
2017-07-12 11:40:00 +00:00
|
|
|
return max_block_size_rows;
|
2017-06-30 16:28:27 +00:00
|
|
|
|
2017-07-24 15:06:32 +00:00
|
|
|
/// Calculates number of rows will be read using preferred_block_size_bytes.
|
2017-07-24 13:59:03 +00:00
|
|
|
/// Can't be less than index_granularity.
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 rows_to_read = task.size_predictor->estimateNumRows(preferred_block_size_bytes);
|
2017-07-24 13:59:03 +00:00
|
|
|
if (!rows_to_read)
|
|
|
|
return rows_to_read;
|
|
|
|
rows_to_read = std::max(index_granularity, rows_to_read);
|
2017-07-21 17:45:51 +00:00
|
|
|
|
|
|
|
if (preferred_max_column_in_block_size_bytes)
|
|
|
|
{
|
2017-07-24 13:59:03 +00:00
|
|
|
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 rows_to_read_for_max_size_column
|
2017-07-21 17:45:51 +00:00
|
|
|
= 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);
|
2018-02-20 11:45:58 +00:00
|
|
|
auto rows_to_read_for_max_size_column_with_filtration
|
2018-06-06 17:16:05 +00:00
|
|
|
= static_cast<UInt64>(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.
|
2017-07-21 17:45:51 +00:00
|
|
|
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
|
|
|
|
}
|
2017-07-11 14:16:00 +00:00
|
|
|
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 unread_rows_in_current_granule = reader.numPendingRowsInCurrentGranule();
|
2017-07-11 14:16:00 +00:00
|
|
|
if (unread_rows_in_current_granule >= rows_to_read)
|
|
|
|
return rows_to_read;
|
|
|
|
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 granule_to_read = (rows_to_read + reader.numReadRowsInCurrentGranule() + index_granularity / 2) / index_granularity;
|
2017-11-21 02:23:41 +00:00
|
|
|
return index_granularity * granule_to_read - reader.numReadRowsInCurrentGranule();
|
2017-06-21 17:19:35 +00:00
|
|
|
};
|
2017-06-20 10:12:20 +00:00
|
|
|
|
2018-02-20 11:45:58 +00:00
|
|
|
if (!task->range_reader.isInitialized())
|
2017-04-06 17:21:45 +00:00
|
|
|
{
|
2018-04-11 14:31:54 +00:00
|
|
|
if (prewhere_info)
|
2017-04-06 17:21:45 +00:00
|
|
|
{
|
2018-03-05 14:41:43 +00:00
|
|
|
if (reader->getColumns().empty())
|
|
|
|
{
|
|
|
|
task->range_reader = MergeTreeRangeReader(
|
2018-04-11 14:31:54 +00:00
|
|
|
pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions,
|
|
|
|
&prewhere_info->prewhere_column_name, &task->ordered_names,
|
2018-03-05 14:41:43 +00:00
|
|
|
task->should_reorder, task->remove_prewhere_column, true);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
task->pre_range_reader = MergeTreeRangeReader(
|
2018-04-11 14:31:54 +00:00
|
|
|
pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions,
|
|
|
|
&prewhere_info->prewhere_column_name, &task->ordered_names,
|
2018-03-05 14:41:43 +00:00
|
|
|
task->should_reorder, task->remove_prewhere_column, false);
|
|
|
|
|
|
|
|
task->range_reader = MergeTreeRangeReader(
|
|
|
|
reader.get(), index_granularity, &task->pre_range_reader, nullptr,
|
|
|
|
nullptr, &task->ordered_names, true, false, true);
|
|
|
|
}
|
2018-02-20 11:45:58 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
task->range_reader = MergeTreeRangeReader(
|
2018-04-11 14:31:54 +00:00
|
|
|
reader.get(), index_granularity, nullptr, nullptr,
|
2018-03-05 14:41:43 +00:00
|
|
|
nullptr, &task->ordered_names, task->should_reorder, false, true);
|
2018-02-20 11:45:58 +00:00
|
|
|
}
|
|
|
|
}
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 recommended_rows = estimateNumRows(*task, task->range_reader);
|
|
|
|
UInt64 rows_to_read = std::max(UInt64(1), std::min(max_block_size_rows, recommended_rows));
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-02-20 11:45:58 +00:00
|
|
|
auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges);
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-02-20 13:59:19 +00:00
|
|
|
/// All rows were filtered. Repeat.
|
|
|
|
if (read_result.block.rows() == 0)
|
|
|
|
read_result.block.clear();
|
|
|
|
|
2018-06-06 17:16:05 +00:00
|
|
|
UInt64 num_filtered_rows = read_result.numReadRows() - read_result.block.rows();
|
2018-02-20 14:26:22 +00:00
|
|
|
|
2018-03-05 14:41:43 +00:00
|
|
|
progressImpl({ read_result.numReadRows(), read_result.numBytesRead() });
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-02-20 11:45:58 +00:00
|
|
|
if (task->size_predictor)
|
|
|
|
{
|
2018-03-05 14:41:43 +00:00
|
|
|
task->size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows);
|
2017-08-01 13:04:48 +00:00
|
|
|
|
2018-02-20 11:45:58 +00:00
|
|
|
if (read_result.block)
|
|
|
|
task->size_predictor->update(read_result.block);
|
|
|
|
}
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-04-11 14:31:54 +00:00
|
|
|
if (read_result.block && prewhere_info && !task->remove_prewhere_column)
|
2018-02-22 11:54:26 +00:00
|
|
|
{
|
2018-03-05 14:41:43 +00:00
|
|
|
/// Convert const column to full here because it's cheaper to filter const column than full.
|
2018-04-11 14:31:54 +00:00
|
|
|
auto & column = read_result.block.getByName(prewhere_info->prewhere_column_name);
|
2018-03-05 14:41:43 +00:00
|
|
|
column.column = column.column->convertToFullColumnIfConst();
|
2017-04-06 17:21:45 +00:00
|
|
|
}
|
2017-04-10 17:10:33 +00:00
|
|
|
|
2018-02-22 12:43:57 +00:00
|
|
|
read_result.block.checkNumberOfRows();
|
2017-04-06 17:21:45 +00:00
|
|
|
|
2018-02-20 11:45:58 +00:00
|
|
|
return read_result.block;
|
2017-04-06 17:21:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-02-19 03:56:08 +00:00
|
|
|
void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const
|
2017-04-06 17:21:45 +00:00
|
|
|
{
|
|
|
|
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")
|
|
|
|
{
|
2018-02-19 03:56:08 +00:00
|
|
|
ColumnPtr column;
|
|
|
|
if (rows)
|
|
|
|
column = DataTypeString().createColumnConst(rows, task->data_part->name)->convertToFullColumnIfConst();
|
|
|
|
else
|
|
|
|
column = DataTypeString().createColumn();
|
|
|
|
|
|
|
|
block.insert({ column, std::make_shared<DataTypeString>(), virt_column_name});
|
2017-04-06 17:21:45 +00:00
|
|
|
}
|
|
|
|
else if (virt_column_name == "_part_index")
|
|
|
|
{
|
2018-02-19 03:56:08 +00:00
|
|
|
ColumnPtr column;
|
|
|
|
if (rows)
|
|
|
|
column = DataTypeUInt64().createColumnConst(rows, static_cast<UInt64>(task->part_index_in_query))->convertToFullColumnIfConst();
|
|
|
|
else
|
|
|
|
column = DataTypeUInt64().createColumn();
|
|
|
|
|
|
|
|
block.insert({ column, std::make_shared<DataTypeUInt64>(), virt_column_name});
|
2017-04-06 17:21:45 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-04-16 12:21:36 +00:00
|
|
|
void MergeTreeBaseBlockInputStream::executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info)
|
2018-04-06 13:58:06 +00:00
|
|
|
{
|
2018-04-11 14:31:54 +00:00
|
|
|
if (prewhere_info)
|
2018-04-06 13:58:06 +00:00
|
|
|
{
|
2018-04-11 14:31:54 +00:00
|
|
|
prewhere_info->prewhere_actions->execute(block);
|
|
|
|
if (prewhere_info->remove_prewhere_column)
|
|
|
|
block.erase(prewhere_info->prewhere_column_name);
|
2018-04-06 13:58:06 +00:00
|
|
|
|
2018-04-11 14:31:54 +00:00
|
|
|
if (!block)
|
2018-04-06 13:58:06 +00:00
|
|
|
block.insert({nullptr, std::make_shared<DataTypeNothing>(), "_nothing"});
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-04-06 17:21:45 +00:00
|
|
|
MergeTreeBaseBlockInputStream::~MergeTreeBaseBlockInputStream() = default;
|
|
|
|
|
|
|
|
}
|