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

212 lines
7.2 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeBaseBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeReader.h>
#include <Core/Defines.h>
2016-11-20 12:43:20 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int NOT_IMPLEMENTED;
2016-11-20 12:43:20 +00:00
}
MergeTreeBlockInputStream::MergeTreeBlockInputStream(
MergeTreeData & storage_,
const MergeTreeData::DataPartPtr & owned_data_part_,
size_t max_block_size_rows_,
size_t preferred_block_size_bytes_,
size_t preferred_max_column_in_block_size_bytes_,
Names column_names,
const MarkRanges & mark_ranges_,
bool use_uncompressed_cache_,
const PrewhereInfoPtr & prewhere_info,
bool check_columns,
size_t min_bytes_to_use_direct_io_,
size_t max_read_buffer_size_,
bool save_marks_in_cache_,
2017-04-05 20:34:19 +00:00
const Names & virt_column_names,
size_t part_index_in_query_,
bool quiet)
:
MergeTreeBaseBlockInputStream{storage_, prewhere_info, max_block_size_rows_,
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, min_bytes_to_use_direct_io_,
max_read_buffer_size_, use_uncompressed_cache_, save_marks_in_cache_, virt_column_names},
2017-04-05 20:34:19 +00:00
ordered_names{column_names},
data_part{owned_data_part_},
part_columns_lock(data_part->columns_lock),
all_mark_ranges(mark_ranges_),
2017-04-05 20:34:19 +00:00
part_index_in_query(part_index_in_query_),
check_columns(check_columns),
path(data_part->getFullPath())
{
/// Let's estimate total number of rows for progress bar.
size_t total_rows = 0;
for (const auto & range : all_mark_ranges)
total_rows += range.end - range.begin;
total_rows *= storage.index_granularity;
if (!quiet)
2017-04-05 20:34:19 +00:00
LOG_TRACE(log, "Reading " << all_mark_ranges.size() << " ranges from part " << data_part->name
<< ", approx. " << total_rows
<< (all_mark_ranges.size() > 1
? ", up to " + toString((all_mark_ranges.back().end - all_mark_ranges.front().begin) * storage.index_granularity)
: "")
<< " rows starting from " << all_mark_ranges.front().begin * storage.index_granularity);
2018-02-23 10:02:29 +00:00
addTotalRowsApprox(total_rows);
}
2018-02-21 05:11:53 +00:00
Block MergeTreeBlockInputStream::getHeader() const
{
if (!header)
2018-02-21 05:11:53 +00:00
{
header = storage.getSampleBlockForColumns(ordered_names);
executePrewhereActions(header);
/// Types may be different during ALTER (when this stream is used to perform an ALTER).
/// NOTE: We may use similar code to implement non blocking ALTERs.
for (const auto & name_type : data_part->columns)
2018-02-21 05:11:53 +00:00
{
if (header.has(name_type.name))
2018-02-21 05:11:53 +00:00
{
auto & elem = header.getByName(name_type.name);
if (!elem.type->equals(*name_type.type))
{
elem.type = name_type.type;
elem.column = elem.type->createColumn();
}
2018-02-21 05:11:53 +00:00
}
}
2016-11-20 12:43:20 +00:00
injectVirtualColumns(header);
}
2018-02-21 05:11:53 +00:00
return header;
}
2017-04-05 20:34:19 +00:00
bool MergeTreeBlockInputStream::getNewTask()
try
2016-11-20 12:43:20 +00:00
{
2017-04-05 20:34:19 +00:00
/// Produce only one task
if (!is_first_task)
{
finish();
2017-04-05 20:34:19 +00:00
return false;
}
is_first_task = false;
Names pre_column_names, column_names = ordered_names;
/// inject columns required for defaults evaluation
bool should_reorder = !injectRequiredColumns(storage, data_part, column_names).empty();
if (prewhere_info)
2017-04-05 20:34:19 +00:00
{
pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns();
2017-04-05 20:34:19 +00:00
if (pre_column_names.empty())
pre_column_names.push_back(column_names[0]);
const auto injected_pre_columns = injectRequiredColumns(storage, data_part, pre_column_names);
if (!injected_pre_columns.empty())
should_reorder = true;
const NameSet pre_name_set(pre_column_names.begin(), pre_column_names.end());
Names post_column_names;
for (const auto & name : column_names)
if (!pre_name_set.count(name))
post_column_names.push_back(name);
column_names = post_column_names;
}
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
column_name_set = NameSet{column_names.begin(), column_names.end()};
if (check_columns)
{
/// Under owned_data_part->columns_lock we check that all requested columns are of the same type as in the table.
/// This may be not true in case of ALTER MODIFY.
if (!pre_column_names.empty())
storage.check(data_part->columns, pre_column_names);
if (!column_names.empty())
storage.check(data_part->columns, column_names);
2018-03-13 15:00:28 +00:00
const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical();
pre_columns = physical_columns.addTypes(pre_column_names);
columns = physical_columns.addTypes(column_names);
2017-04-05 20:34:19 +00:00
}
else
{
pre_columns = data_part->columns.addTypes(pre_column_names);
columns = data_part->columns.addTypes(column_names);
}
/** @note you could simply swap `reverse` in if and else branches of MergeTreeDataSelectExecutor,
* and remove this reverse. */
MarkRanges remaining_mark_ranges = all_mark_ranges;
std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end());
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
: std::make_unique<MergeTreeBlockSizePredictor>(data_part, ordered_names, data_part->storage.getSampleBlock());
2017-04-05 20:34:19 +00:00
task = std::make_unique<MergeTreeReadTask>(
data_part, remaining_mark_ranges, part_index_in_query, ordered_names, column_name_set, columns, pre_columns,
prewhere_info && prewhere_info->remove_prewhere_column, should_reorder, std::move(size_predictor));
if (!reader)
{
if (use_uncompressed_cache)
owned_uncompressed_cache = storage.context.getUncompressedCache();
owned_mark_cache = storage.context.getMarkCache();
reader = std::make_unique<MergeTreeReader>(
2017-04-05 20:34:19 +00:00
path, data_part, columns, owned_uncompressed_cache.get(),
owned_mark_cache.get(), save_marks_in_cache, storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size);
if (prewhere_info)
pre_reader = std::make_unique<MergeTreeReader>(
2017-04-05 20:34:19 +00:00
path, data_part, pre_columns, owned_uncompressed_cache.get(),
owned_mark_cache.get(), save_marks_in_cache, storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size);
}
2017-04-05 20:34:19 +00:00
return true;
}
catch (...)
2017-04-05 20:34:19 +00:00
{
/// Suspicion of the broken part. A part is added to the queue for verification.
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
storage.reportBrokenPart(data_part->name);
throw;
}
void MergeTreeBlockInputStream::finish()
{
/** Close the files (before destroying the object).
* When many sources are created, but simultaneously reading only a few of them,
* buffers don't waste memory.
*/
reader.reset();
pre_reader.reset();
part_columns_lock.unlock();
data_part.reset();
2016-11-20 12:43:20 +00:00
}
MergeTreeBlockInputStream::~MergeTreeBlockInputStream() = default;
2016-11-20 12:43:20 +00:00
}