mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #53931 from CurtizJ/refactoring-of-merge-tree-reading
Refactoring of reading from `MergeTree` tables
This commit is contained in:
commit
7589a3dd23
@ -23,12 +23,12 @@
|
||||
#include <Processors/Transforms/ReverseTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolInOrder.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h>
|
||||
#include <Storages/MergeTree/MergeTreeSource.h>
|
||||
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
@ -251,7 +251,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
Poco::Logger * log_,
|
||||
MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_,
|
||||
bool enable_parallel_reading)
|
||||
: SourceStepWithFilter(DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader(
|
||||
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||
storage_snapshot_->getSampleBlockForColumns(real_column_names_),
|
||||
getPrewhereInfoFromQueryInfo(query_info_),
|
||||
data_.getPartitionValueType(),
|
||||
@ -268,10 +268,11 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
, storage_snapshot(std::move(storage_snapshot_))
|
||||
, metadata_for_reading(storage_snapshot->getMetadataForQuery())
|
||||
, context(std::move(context_))
|
||||
, max_block_size(max_block_size_)
|
||||
, block_size{
|
||||
.max_block_size_rows = max_block_size_,
|
||||
.preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes,
|
||||
.preferred_max_column_in_block_size_bytes = context->getSettingsRef().preferred_max_column_in_block_size_bytes}
|
||||
, requested_num_streams(num_streams_)
|
||||
, preferred_block_size_bytes(context->getSettingsRef().preferred_block_size_bytes)
|
||||
, preferred_max_column_in_block_size_bytes(context->getSettingsRef().preferred_max_column_in_block_size_bytes)
|
||||
, sample_factor_column_queried(sample_factor_column_queried_)
|
||||
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
|
||||
, log(log_)
|
||||
@ -281,7 +282,7 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
if (sample_factor_column_queried)
|
||||
{
|
||||
/// Only _sample_factor virtual column is added by ReadFromMergeTree
|
||||
/// Other virtual columns are added by MergeTreeBaseSelectProcessor.
|
||||
/// Other virtual columns are added by MergeTreeSelectProcessor.
|
||||
auto type = std::make_shared<DataTypeFloat64>();
|
||||
output_stream->header.insert({type->createColumn(), type, "_sample_factor"});
|
||||
}
|
||||
@ -325,50 +326,50 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache
|
||||
)
|
||||
PoolSettings pool_settings)
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
|
||||
auto extension = ParallelReadingExtension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = required_columns
|
||||
.columns_to_read = required_columns,
|
||||
};
|
||||
|
||||
/// We have a special logic for local replica. It has to read less data, because in some cases it should
|
||||
/// merge states of aggregate functions or do some other important stuff other than reading from Disk.
|
||||
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
pool_settings.min_marks_for_concurrent_read = static_cast<size_t>(pool_settings.min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
auto pool = std::make_shared<MergeTreeReadPoolParallelReplicas>(
|
||||
std::move(extension),
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
max_streams,
|
||||
extension,
|
||||
parts_with_range,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
min_marks_for_concurrent_read);
|
||||
pool_settings,
|
||||
context);
|
||||
|
||||
auto block_size_copy = block_size;
|
||||
block_size_copy.min_marks_to_read = pool_settings.min_marks_for_concurrent_read;
|
||||
|
||||
Pipes pipes;
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
for (size_t i = 0; i < max_streams; ++i)
|
||||
for (size_t i = 0; i < pool_settings.threads; ++i)
|
||||
{
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size,
|
||||
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
|
||||
data, storage_snapshot, use_uncompressed_cache,
|
||||
prewhere_info, actions_settings, reader_settings, virt_column_names);
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(i);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size_copy, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
|
||||
/// Set the approximate number of rows for the first source only
|
||||
/// In case of parallel processing on replicas do not set approximate rows at all.
|
||||
@ -387,12 +388,8 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
Pipe ReadFromMergeTree::readFromPool(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache)
|
||||
PoolSettings pool_settings)
|
||||
{
|
||||
Pipes pipes;
|
||||
size_t sum_marks = parts_with_range.getMarksCountAllParts();
|
||||
size_t total_rows = parts_with_range.getRowsCountAllParts();
|
||||
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
@ -403,11 +400,11 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
/// round min_marks_to_read up to nearest multiple of block_size expressed in marks
|
||||
/// If granularity is adaptive it doesn't make sense
|
||||
/// Maybe it will make sense to add settings `max_block_size_bytes`
|
||||
if (max_block_size && !data.canUseAdaptiveGranularity())
|
||||
if (block_size.max_block_size_rows && !data.canUseAdaptiveGranularity())
|
||||
{
|
||||
size_t fixed_index_granularity = data.getSettings()->index_granularity;
|
||||
min_marks_for_concurrent_read = (min_marks_for_concurrent_read * fixed_index_granularity + max_block_size - 1)
|
||||
/ max_block_size * max_block_size / fixed_index_granularity;
|
||||
pool_settings.min_marks_for_concurrent_read = (pool_settings.min_marks_for_concurrent_read * fixed_index_granularity + block_size.max_block_size_rows - 1)
|
||||
/ block_size.max_block_size_rows * block_size.max_block_size_rows / fixed_index_granularity;
|
||||
}
|
||||
|
||||
bool all_parts_are_remote = true;
|
||||
@ -421,34 +418,30 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
|
||||
MergeTreeReadPoolPtr pool;
|
||||
|
||||
if ((all_parts_are_remote && settings.allow_prefetched_read_pool_for_remote_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method))
|
||||
|| (all_parts_are_local && settings.allow_prefetched_read_pool_for_local_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method)))
|
||||
bool allow_prefetched_remote = all_parts_are_remote
|
||||
&& settings.allow_prefetched_read_pool_for_remote_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method);
|
||||
|
||||
bool allow_prefetched_local = all_parts_are_local
|
||||
&& settings.allow_prefetched_read_pool_for_local_filesystem
|
||||
&& MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method);
|
||||
|
||||
if (allow_prefetched_remote || allow_prefetched_local)
|
||||
{
|
||||
pool = std::make_shared<MergeTreePrefetchedReadPool>(
|
||||
max_streams,
|
||||
sum_marks,
|
||||
min_marks_for_concurrent_read,
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
settings.preferred_block_size_bytes,
|
||||
reader_settings,
|
||||
context,
|
||||
use_uncompressed_cache,
|
||||
all_parts_are_remote,
|
||||
*data.getSettings());
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
else
|
||||
{
|
||||
pool = std::make_shared<MergeTreeReadPool>(
|
||||
max_streams,
|
||||
sum_marks,
|
||||
min_marks_for_concurrent_read,
|
||||
std::move(parts_with_range),
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
@ -456,22 +449,28 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
context,
|
||||
false);
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
|
||||
auto * logger = &Poco::Logger::get(data.getLogName() + " (SelectExecutor)");
|
||||
LOG_DEBUG(logger, "Reading approx. {} rows with {} streams", total_rows, max_streams);
|
||||
LOG_DEBUG(log, "Reading approx. {} rows with {} streams", total_rows, pool_settings.threads);
|
||||
|
||||
for (size_t i = 0; i < max_streams; ++i)
|
||||
/// The reason why we change this setting is because MergeTreeReadPool takes the full task
|
||||
/// ignoring min_marks_to_read setting in case of remote disk (see MergeTreeReadPool::getTask).
|
||||
/// In this case, we won't limit the number of rows to read based on adaptive granularity settings.
|
||||
auto block_size_copy = block_size;
|
||||
block_size_copy.min_marks_to_read = pool_settings.min_marks_for_concurrent_read;
|
||||
|
||||
Pipes pipes;
|
||||
for (size_t i = 0; i < pool_settings.threads; ++i)
|
||||
{
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size,
|
||||
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
|
||||
data, storage_snapshot, use_uncompressed_cache,
|
||||
prewhere_info, actions_settings, reader_settings, virt_column_names);
|
||||
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(i);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size_copy, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
|
||||
if (i == 0)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
@ -485,17 +484,65 @@ Pipe ReadFromMergeTree::readFromPool(
|
||||
return pipe;
|
||||
}
|
||||
|
||||
template<typename Algorithm>
|
||||
ProcessorPtr ReadFromMergeTree::createSource(
|
||||
const RangesInDataPart & part,
|
||||
const Names & required_columns,
|
||||
bool use_uncompressed_cache,
|
||||
bool has_limit_below_one_block,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
|
||||
Pipe ReadFromMergeTree::readInOrder(
|
||||
RangesInDataParts parts_with_ranges,
|
||||
Names required_columns,
|
||||
PoolSettings pool_settings,
|
||||
ReadType read_type,
|
||||
UInt64 limit)
|
||||
{
|
||||
auto total_rows = part.getRowsCount();
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
total_rows = query_info.limit;
|
||||
/// For reading in order it makes sense to read only
|
||||
/// one range per task to reduce number of read rows.
|
||||
bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < block_size.max_block_size_rows;
|
||||
MergeTreeReadPoolPtr pool;
|
||||
|
||||
if (is_parallel_reading_from_replicas)
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
ParallelReadingExtension extension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = required_columns,
|
||||
};
|
||||
|
||||
pool_settings.min_marks_for_concurrent_read = static_cast<size_t>(
|
||||
pool_settings.min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
|
||||
|
||||
CoordinationMode mode = read_type == ReadType::InOrder
|
||||
? CoordinationMode::WithOrder
|
||||
: CoordinationMode::ReverseOrder;
|
||||
|
||||
pool = std::make_shared<MergeTreeReadPoolParallelReplicasInOrder>(
|
||||
std::move(extension),
|
||||
mode,
|
||||
parts_with_ranges,
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
else
|
||||
{
|
||||
pool = std::make_shared<MergeTreeReadPoolInOrder>(
|
||||
has_limit_below_one_block,
|
||||
read_type,
|
||||
parts_with_ranges,
|
||||
storage_snapshot,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
required_columns,
|
||||
virt_column_names,
|
||||
pool_settings,
|
||||
context);
|
||||
}
|
||||
|
||||
/// Actually it means that parallel reading from replicas enabled
|
||||
/// and we have to collaborate with initiator.
|
||||
@ -504,37 +551,34 @@ ProcessorPtr ReadFromMergeTree::createSource(
|
||||
/// because we don't know actual amount of read rows in case when limit is set.
|
||||
bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order;
|
||||
|
||||
auto algorithm = std::make_unique<Algorithm>(
|
||||
data, storage_snapshot, part.data_part, part.alter_conversions, max_block_size, preferred_block_size_bytes,
|
||||
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
|
||||
actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
|
||||
|
||||
if (set_rows_approx)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
|
||||
return source;
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::readInOrder(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
ReadType read_type,
|
||||
bool use_uncompressed_cache,
|
||||
UInt64 limit,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
|
||||
{
|
||||
Pipes pipes;
|
||||
/// For reading in order it makes sense to read only
|
||||
/// one range per task to reduce number of read rows.
|
||||
bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < max_block_size;
|
||||
|
||||
for (const auto & part : parts_with_range)
|
||||
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
|
||||
{
|
||||
auto source = read_type == ReadType::InReverseOrder
|
||||
? createSource<MergeTreeReverseSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool)
|
||||
: createSource<MergeTreeInOrderSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool);
|
||||
const auto & part_with_ranges = parts_with_ranges[i];
|
||||
|
||||
UInt64 total_rows = part_with_ranges.getRowsCount();
|
||||
if (query_info.limit > 0 && query_info.limit < total_rows)
|
||||
total_rows = query_info.limit;
|
||||
|
||||
LOG_TRACE(log, "Reading {} ranges in{}order from part {}, approx. {} rows starting from {}",
|
||||
part_with_ranges.ranges.size(),
|
||||
read_type == ReadType::InReverseOrder ? " reverse " : " ",
|
||||
part_with_ranges.data_part->name, total_rows,
|
||||
part_with_ranges.data_part->index_granularity.getMarkStartingRow(part_with_ranges.ranges.front().begin));
|
||||
|
||||
MergeTreeSelectAlgorithmPtr algorithm;
|
||||
if (read_type == ReadType::InReverseOrder)
|
||||
algorithm = std::make_unique<MergeTreeInReverseOrderSelectAlgorithm>(i);
|
||||
else
|
||||
algorithm = std::make_unique<MergeTreeInOrderSelectAlgorithm>(i);
|
||||
|
||||
auto processor = std::make_unique<MergeTreeSelectProcessor>(
|
||||
pool, std::move(algorithm), data, prewhere_info,
|
||||
actions_settings, block_size, reader_settings, virt_column_names);
|
||||
|
||||
auto source = std::make_shared<MergeTreeSource>(std::move(processor));
|
||||
if (set_rows_approx)
|
||||
source->addTotalRowsApprox(total_rows);
|
||||
|
||||
pipes.emplace_back(std::move(source));
|
||||
}
|
||||
@ -553,16 +597,33 @@ Pipe ReadFromMergeTree::readInOrder(
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::read(
|
||||
RangesInDataParts parts_with_range, Names required_columns, ReadType read_type,
|
||||
size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache)
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
ReadType read_type,
|
||||
size_t max_streams,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
bool use_uncompressed_cache)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
size_t sum_marks = parts_with_range.getMarksCountAllParts();
|
||||
|
||||
PoolSettings pool_settings
|
||||
{
|
||||
.threads = max_streams,
|
||||
.sum_marks = sum_marks,
|
||||
.min_marks_for_concurrent_read = min_marks_for_concurrent_read,
|
||||
.preferred_block_size_bytes = settings.preferred_block_size_bytes,
|
||||
.use_uncompressed_cache = use_uncompressed_cache,
|
||||
.use_const_size_tasks_for_remote_reading = settings.merge_tree_use_const_size_tasks_for_remote_reading,
|
||||
};
|
||||
|
||||
if (read_type == ReadType::ParallelReplicas)
|
||||
return readFromPoolParallelReplicas(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache);
|
||||
return readFromPoolParallelReplicas(std::move(parts_with_range), std::move(required_columns), std::move(pool_settings));
|
||||
|
||||
if (read_type == ReadType::Default && max_streams > 1)
|
||||
return readFromPool(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache);
|
||||
return readFromPool(std::move(parts_with_range), std::move(required_columns), std::move(pool_settings));
|
||||
|
||||
auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, /*limit */0, /*pool*/nullptr);
|
||||
auto pipe = readInOrder(parts_with_range, required_columns, pool_settings, read_type, /*limit=*/ 0);
|
||||
|
||||
/// Use ConcatProcessor to concat sources together.
|
||||
/// It is needed to read in parts order (and so in PK order) if single thread is used.
|
||||
@ -585,7 +646,6 @@ struct PartRangesReadInfo
|
||||
size_t index_granularity_bytes = 0;
|
||||
size_t max_marks_to_use_cache = 0;
|
||||
size_t min_marks_for_concurrent_read = 0;
|
||||
|
||||
bool use_uncompressed_cache = false;
|
||||
|
||||
PartRangesReadInfo(
|
||||
@ -663,8 +723,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
|
||||
auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default;
|
||||
|
||||
return read(std::move(parts_with_ranges), column_names, read_type,
|
||||
num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache);
|
||||
return read(std::move(parts_with_ranges),
|
||||
column_names,
|
||||
read_type,
|
||||
num_streams,
|
||||
info.min_marks_for_concurrent_read,
|
||||
info.use_uncompressed_cache);
|
||||
}
|
||||
|
||||
static ActionsDAGPtr createProjection(const Block & header)
|
||||
@ -715,7 +779,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
|
||||
/// Let's split ranges to avoid reading much data.
|
||||
auto split_ranges
|
||||
= [rows_granularity = data_settings->index_granularity, my_max_block_size = max_block_size](const auto & ranges, int direction)
|
||||
= [rows_granularity = data_settings->index_granularity, my_max_block_size = block_size.max_block_size_rows]
|
||||
(const auto & ranges, int direction)
|
||||
{
|
||||
MarkRanges new_ranges;
|
||||
const size_t max_marks_in_range = (my_max_block_size + rows_granularity - 1) / rows_granularity;
|
||||
@ -762,109 +827,94 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
const size_t min_marks_per_stream = (info.sum_marks - 1) / num_streams + 1;
|
||||
bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold);
|
||||
|
||||
std::vector<RangesInDataParts> splitted_parts_and_ranges;
|
||||
splitted_parts_and_ranges.reserve(num_streams);
|
||||
const auto read_type = input_order_info->direction == 1 ? ReadType::InOrder : ReadType::InReverseOrder;
|
||||
|
||||
const auto read_type = input_order_info->direction == 1
|
||||
? ReadFromMergeTree::ReadType::InOrder
|
||||
: ReadFromMergeTree::ReadType::InReverseOrder;
|
||||
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool;
|
||||
|
||||
if (is_parallel_reading_from_replicas)
|
||||
PoolSettings pool_settings
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
auto extension = ParallelReadingExtension
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.count_participating_replicas = client_info.count_participating_replicas,
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.columns_to_read = column_names
|
||||
};
|
||||
|
||||
auto min_marks_for_concurrent_read = info.min_marks_for_concurrent_read;
|
||||
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * settings.parallel_replicas_single_task_marks_count_multiplier);
|
||||
|
||||
pool = std::make_shared<MergeTreeInOrderReadPoolParallelReplicas>(
|
||||
parts_with_ranges,
|
||||
extension,
|
||||
read_type == ReadFromMergeTree::ReadType::InOrder ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder,
|
||||
min_marks_for_concurrent_read);
|
||||
}
|
||||
|
||||
|
||||
for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i)
|
||||
{
|
||||
size_t need_marks = min_marks_per_stream;
|
||||
RangesInDataParts new_parts;
|
||||
|
||||
/// Loop over parts.
|
||||
/// We will iteratively take part or some subrange of a part from the back
|
||||
/// and assign a stream to read from it.
|
||||
while (need_marks > 0 && !parts_with_ranges.empty())
|
||||
{
|
||||
RangesInDataPart part = parts_with_ranges.back();
|
||||
parts_with_ranges.pop_back();
|
||||
size_t & marks_in_part = info.sum_marks_in_parts.back();
|
||||
|
||||
/// We will not take too few rows from a part.
|
||||
if (marks_in_part >= info.min_marks_for_concurrent_read && need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = info.min_marks_for_concurrent_read;
|
||||
|
||||
/// Do not leave too few rows in the part.
|
||||
if (marks_in_part > need_marks && marks_in_part - need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
|
||||
/// We take full part if it contains enough marks or
|
||||
/// if we know limit and part contains less than 'limit' rows.
|
||||
bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount());
|
||||
|
||||
/// We take the whole part if it is small enough.
|
||||
if (take_full_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
info.sum_marks_in_parts.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Loop through ranges in part. Take enough ranges to cover "need_marks".
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected end of ranges while spreading marks among streams");
|
||||
|
||||
MarkRange & range = part.ranges.front();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_front();
|
||||
}
|
||||
parts_with_ranges.emplace_back(part);
|
||||
}
|
||||
|
||||
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
|
||||
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
|
||||
}
|
||||
.min_marks_for_concurrent_read = info.min_marks_for_concurrent_read,
|
||||
.preferred_block_size_bytes = settings.preferred_block_size_bytes,
|
||||
.use_uncompressed_cache = info.use_uncompressed_cache,
|
||||
};
|
||||
|
||||
Pipes pipes;
|
||||
for (auto & item : splitted_parts_and_ranges)
|
||||
/// For parallel replicas the split will be performed on the initiator side.
|
||||
if (is_parallel_reading_from_replicas)
|
||||
{
|
||||
pipes.emplace_back(readInOrder(std::move(item), column_names, read_type,
|
||||
info.use_uncompressed_cache, input_order_info->limit, pool));
|
||||
pipes.emplace_back(readInOrder(std::move(parts_with_ranges), column_names, pool_settings, read_type, input_order_info->limit));
|
||||
}
|
||||
else
|
||||
{
|
||||
std::vector<RangesInDataParts> splitted_parts_and_ranges;
|
||||
splitted_parts_and_ranges.reserve(num_streams);
|
||||
|
||||
for (size_t i = 0; i < num_streams && !parts_with_ranges.empty(); ++i)
|
||||
{
|
||||
size_t need_marks = min_marks_per_stream;
|
||||
RangesInDataParts new_parts;
|
||||
|
||||
/// Loop over parts.
|
||||
/// We will iteratively take part or some subrange of a part from the back
|
||||
/// and assign a stream to read from it.
|
||||
while (need_marks > 0 && !parts_with_ranges.empty())
|
||||
{
|
||||
RangesInDataPart part = parts_with_ranges.back();
|
||||
parts_with_ranges.pop_back();
|
||||
size_t & marks_in_part = info.sum_marks_in_parts.back();
|
||||
|
||||
/// We will not take too few rows from a part.
|
||||
if (marks_in_part >= info.min_marks_for_concurrent_read && need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = info.min_marks_for_concurrent_read;
|
||||
|
||||
/// Do not leave too few rows in the part.
|
||||
if (marks_in_part > need_marks && marks_in_part - need_marks < info.min_marks_for_concurrent_read)
|
||||
need_marks = marks_in_part;
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
|
||||
/// We take full part if it contains enough marks or
|
||||
/// if we know limit and part contains less than 'limit' rows.
|
||||
bool take_full_part = marks_in_part <= need_marks || (input_order_info->limit && input_order_info->limit < part.getRowsCount());
|
||||
|
||||
/// We take the whole part if it is small enough.
|
||||
if (take_full_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
info.sum_marks_in_parts.pop_back();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Loop through ranges in part. Take enough ranges to cover "need_marks".
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected end of ranges while spreading marks among streams");
|
||||
|
||||
MarkRange & range = part.ranges.front();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_front();
|
||||
}
|
||||
parts_with_ranges.emplace_back(part);
|
||||
}
|
||||
|
||||
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
|
||||
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
|
||||
}
|
||||
|
||||
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
|
||||
}
|
||||
|
||||
for (auto && item : splitted_parts_and_ranges)
|
||||
pipes.emplace_back(readInOrder(std::move(item), column_names, pool_settings, read_type, input_order_info->limit));
|
||||
}
|
||||
|
||||
Block pipe_header;
|
||||
@ -898,7 +948,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
if (pipe.numOutputPorts() > 1)
|
||||
{
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, block_size.max_block_size_rows, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
}
|
||||
@ -931,7 +981,7 @@ static void addMergingFinal(
|
||||
const SortDescription & sort_description,
|
||||
MergeTreeData::MergingParams merging_params,
|
||||
Names partition_key_columns,
|
||||
size_t max_block_size)
|
||||
size_t max_block_size_rows)
|
||||
{
|
||||
const auto & header = pipe.getHeader();
|
||||
size_t num_outputs = pipe.numOutputPorts();
|
||||
@ -944,31 +994,31 @@ static void addMergingFinal(
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
return std::make_shared<MergingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
return std::make_shared<CollapsingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, true, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.sign_column, true, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
return std::make_shared<SummingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
return std::make_shared<AggregatingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
return std::make_shared<ReplacingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size_rows, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
return std::make_shared<VersionedCollapsingTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, max_block_size, /*max_block_size_bytes=*/0);
|
||||
sort_description, merging_params.sign_column, max_block_size_rows, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
return std::make_shared<GraphiteRollupSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
|
||||
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
@ -1064,11 +1114,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
return this->read(
|
||||
std::move(parts),
|
||||
column_names,
|
||||
ReadFromMergeTree::ReadType::InOrder,
|
||||
ReadType::InOrder,
|
||||
1 /* num_streams */,
|
||||
0 /* min_marks_for_concurrent_read */,
|
||||
info.use_uncompressed_cache);
|
||||
};
|
||||
|
||||
pipes = buildPipesForReadingByPKRanges(
|
||||
metadata_for_reading->getPrimaryKey(),
|
||||
sorting_expr,
|
||||
@ -1080,7 +1131,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
else
|
||||
{
|
||||
pipes.emplace_back(read(
|
||||
std::move(new_parts), column_names, ReadFromMergeTree::ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
|
||||
std::move(new_parts), column_names, ReadType::InOrder, num_streams, 0, info.use_uncompressed_cache));
|
||||
|
||||
pipes.back().addSimpleTransform([sorting_expr](const Block & header)
|
||||
{ return std::make_shared<ExpressionTransform>(header, sorting_expr); });
|
||||
@ -1121,7 +1172,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
sort_description,
|
||||
data.merging_params,
|
||||
partition_key_columns,
|
||||
max_block_size);
|
||||
block_size.max_block_size_rows);
|
||||
|
||||
partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes)));
|
||||
}
|
||||
@ -1141,7 +1192,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts)
|
||||
num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size());
|
||||
|
||||
auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default,
|
||||
auto pipe = read(std::move(lonely_parts), column_names, ReadType::Default,
|
||||
num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache);
|
||||
|
||||
/// Drop temporary columns, added by 'sorting_key_expr'
|
||||
@ -1605,11 +1656,13 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info
|
||||
{
|
||||
query_info.prewhere_info = prewhere_info_value;
|
||||
prewhere_info = prewhere_info_value;
|
||||
output_stream = DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader(
|
||||
|
||||
output_stream = DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||
storage_snapshot->getSampleBlockForColumns(real_column_names),
|
||||
prewhere_info_value,
|
||||
data.getPartitionValueType(),
|
||||
virt_column_names)};
|
||||
|
||||
updateSortDescriptionForOutputStream(
|
||||
*output_stream,
|
||||
storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(),
|
||||
|
@ -87,25 +87,7 @@ public:
|
||||
};
|
||||
|
||||
using IndexStats = std::vector<IndexStat>;
|
||||
|
||||
enum class ReadType
|
||||
{
|
||||
/// By default, read will use MergeTreeReadPool and return pipe with num_streams outputs.
|
||||
/// If num_streams == 1, will read without pool, in order specified in parts.
|
||||
Default,
|
||||
/// Read in sorting key order.
|
||||
/// Returned pipe will have the number of ports equals to parts.size().
|
||||
/// Parameter num_streams_ is ignored in this case.
|
||||
/// User should add MergingSorted itself if needed.
|
||||
InOrder,
|
||||
/// The same as InOrder, but in reverse order.
|
||||
/// For every part, read ranges and granules from end to begin. Also add ReverseTransform.
|
||||
InReverseOrder,
|
||||
/// A special type of reading where every replica
|
||||
/// talks to a remote coordinator (which is located on the initiator node)
|
||||
/// and who spreads marks and parts across them.
|
||||
ParallelReplicas,
|
||||
};
|
||||
using ReadType = MergeTreeReadType;
|
||||
|
||||
struct AnalysisResult
|
||||
{
|
||||
@ -113,7 +95,7 @@ public:
|
||||
MergeTreeDataSelectSamplingData sampling;
|
||||
IndexStats index_stats;
|
||||
Names column_names_to_read;
|
||||
ReadFromMergeTree::ReadType read_type = ReadFromMergeTree::ReadType::Default;
|
||||
ReadType read_type = ReadType::Default;
|
||||
UInt64 total_parts = 0;
|
||||
UInt64 parts_before_pk = 0;
|
||||
UInt64 selected_parts = 0;
|
||||
@ -223,7 +205,7 @@ public:
|
||||
|
||||
const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; }
|
||||
const MergeTreeData & getMergeTreeData() const { return data; }
|
||||
size_t getMaxBlockSize() const { return max_block_size; }
|
||||
size_t getMaxBlockSize() const { return block_size.max_block_size_rows; }
|
||||
size_t getNumStreams() const { return requested_num_streams; }
|
||||
bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; }
|
||||
|
||||
@ -271,12 +253,10 @@ private:
|
||||
StorageMetadataPtr metadata_for_reading;
|
||||
|
||||
ContextPtr context;
|
||||
const MergeTreeReadTask::BlockSizeParams block_size;
|
||||
|
||||
const size_t max_block_size;
|
||||
size_t requested_num_streams;
|
||||
size_t output_streams_limit = 0;
|
||||
const size_t preferred_block_size_bytes;
|
||||
const size_t preferred_max_column_in_block_size_bytes;
|
||||
const bool sample_factor_column_queried;
|
||||
|
||||
/// Used for aggregation optimisation (see DB::QueryPlanOptimizations::tryAggregateEachPartitionIndependently).
|
||||
@ -292,16 +272,14 @@ private:
|
||||
UInt64 selected_rows = 0;
|
||||
UInt64 selected_marks = 0;
|
||||
|
||||
using PoolSettings = MergeTreeReadPoolBase::PoolSettings;
|
||||
|
||||
Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
|
||||
Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
|
||||
Pipe readFromPool(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings);
|
||||
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings);
|
||||
Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit);
|
||||
|
||||
template<typename TSource>
|
||||
ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
|
||||
|
||||
Pipe spreadMarkRanges(
|
||||
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
|
||||
Pipe groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection);
|
||||
|
||||
|
@ -3,24 +3,27 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct MergeTreeReadTask;
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
|
||||
|
||||
/// The interface that determines how tasks for reading (MergeTreeReadTask)
|
||||
/// are distributed among data parts with ranges.
|
||||
class IMergeTreeReadPool : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
virtual ~IMergeTreeReadPool() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
virtual Block getHeader() const = 0;
|
||||
|
||||
virtual MergeTreeReadTaskPtr getTask(size_t thread) = 0;
|
||||
/// Returns true if tasks are returned in the same order as the order of ranges passed to pool
|
||||
virtual bool preservesOrderOfRanges() const = 0;
|
||||
|
||||
/// task_idx is an implementation defined identifier that helps
|
||||
/// to get required task. E.g. it may be number of thread in case of Default reading type or an index of a part in case of InOrder/InReverseOrder reading type.
|
||||
virtual MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) = 0;
|
||||
virtual void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) = 0;
|
||||
};
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
|
@ -1,696 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
#include <city.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event WaitPrefetchTaskMicroseconds;
|
||||
};
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int QUERY_WAS_CANCELLED;
|
||||
}
|
||||
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns);
|
||||
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns);
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
UInt64 preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
bool use_uncompressed_cache_,
|
||||
const Names & virt_column_names_)
|
||||
: storage(storage_)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps))
|
||||
, 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_)
|
||||
, reader_settings(reader_settings_)
|
||||
, use_uncompressed_cache(use_uncompressed_cache_)
|
||||
, virt_column_names(virt_column_names_)
|
||||
, partition_value_type(storage.getPartitionValueType())
|
||||
, owned_uncompressed_cache(use_uncompressed_cache ? storage.getContext()->getUncompressedCache() : nullptr)
|
||||
, owned_mark_cache(storage.getContext()->getMarkCache())
|
||||
{
|
||||
header_without_const_virtual_columns = applyPrewhereActions(std::move(header), prewhere_info);
|
||||
size_t non_const_columns_offset = header_without_const_virtual_columns.columns();
|
||||
injectNonConstVirtualColumns(0, header_without_const_virtual_columns, virt_column_names);
|
||||
|
||||
for (size_t col_num = non_const_columns_offset; col_num < header_without_const_virtual_columns.columns(); ++col_num)
|
||||
non_const_virtual_column_names.emplace_back(header_without_const_virtual_columns.getByPosition(col_num).name);
|
||||
|
||||
result_header = header_without_const_virtual_columns;
|
||||
injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names);
|
||||
|
||||
if (!prewhere_actions.steps.empty())
|
||||
LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions());
|
||||
|
||||
if (prewhere_info)
|
||||
LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}",
|
||||
(prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("<nullptr>")),
|
||||
(!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string("<nullptr>")));
|
||||
}
|
||||
|
||||
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere);
|
||||
|
||||
PrewhereExprInfo IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
|
||||
{
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
PrewhereExprStep row_level_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter, actions_settings),
|
||||
.filter_column_name = prewhere_info->row_level_column_name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(row_level_filter_step)));
|
||||
}
|
||||
|
||||
if (!enable_multiple_prewhere_read_steps ||
|
||||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions))
|
||||
{
|
||||
PrewhereExprStep prewhere_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings),
|
||||
.filter_column_name = prewhere_info->prewhere_column_name,
|
||||
.remove_filter_column = prewhere_info->remove_prewhere_column,
|
||||
.need_filter = prewhere_info->need_filter,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(prewhere_step)));
|
||||
}
|
||||
}
|
||||
|
||||
return prewhere_actions;
|
||||
}
|
||||
|
||||
|
||||
bool IMergeTreeSelectAlgorithm::getNewTask()
|
||||
{
|
||||
if (getNewTaskImpl())
|
||||
{
|
||||
finalizeNewTask();
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
ChunkAndProgress IMergeTreeSelectAlgorithm::read()
|
||||
{
|
||||
while (!is_cancelled)
|
||||
{
|
||||
try
|
||||
{
|
||||
if ((!task || task->isFinished()) && !getNewTask())
|
||||
break;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// See MergeTreeBaseSelectProcessor::getTaskFromBuffer()
|
||||
if (e.code() == ErrorCodes::QUERY_WAS_CANCELLED)
|
||||
break;
|
||||
throw;
|
||||
}
|
||||
|
||||
auto res = readFromPart();
|
||||
|
||||
if (res.row_count)
|
||||
{
|
||||
injectVirtualColumns(res.block, res.row_count, task.get(), partition_value_type, virt_column_names);
|
||||
|
||||
/// Reorder the columns according to result_header
|
||||
Columns ordered_columns;
|
||||
ordered_columns.reserve(result_header.columns());
|
||||
for (size_t i = 0; i < result_header.columns(); ++i)
|
||||
{
|
||||
auto name = result_header.getByPosition(i).name;
|
||||
ordered_columns.push_back(res.block.getByName(name).column);
|
||||
}
|
||||
|
||||
return ChunkAndProgress{
|
||||
.chunk = Chunk(ordered_columns, res.row_count),
|
||||
.num_read_rows = res.num_read_rows,
|
||||
.num_read_bytes = res.num_read_bytes,
|
||||
.is_finished = false};
|
||||
}
|
||||
else
|
||||
{
|
||||
return {Chunk(), res.num_read_rows, res.num_read_bytes, false};
|
||||
}
|
||||
}
|
||||
|
||||
return {Chunk(), 0, 0, true};
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask(
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
if (!task)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no task");
|
||||
|
||||
if (task->reader.valid())
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
reader = task->reader.get();
|
||||
}
|
||||
else
|
||||
{
|
||||
reader = task->data_part->getReader(
|
||||
task->task_columns.columns, storage_snapshot, task->mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
task->alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
if (!task->pre_reader_for_step.empty())
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
pre_reader_for_step.clear();
|
||||
for (auto & pre_reader : task->pre_reader_for_step)
|
||||
pre_reader_for_step.push_back(pre_reader.get());
|
||||
}
|
||||
else
|
||||
{
|
||||
initializeMergeTreePreReadersForPart(
|
||||
task->data_part, task->alter_conversions,
|
||||
task->task_columns, task->mark_ranges,
|
||||
value_size_map, profile_callback);
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
reader = data_part->getReader(
|
||||
task_columns.columns, storage_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback);
|
||||
|
||||
initializeMergeTreePreReadersForPart(
|
||||
data_part, alter_conversions, task_columns,
|
||||
mark_ranges, value_size_map, profile_callback);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback)
|
||||
{
|
||||
pre_reader_for_step.clear();
|
||||
|
||||
/// Add lightweight delete filtering step
|
||||
if (reader_settings.apply_deleted_mask && data_part->hasLightweightDelete())
|
||||
{
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
{LightweightDeleteDescription::FILTER_COLUMN}, storage_snapshot,
|
||||
mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
pre_reader_for_step.push_back(
|
||||
data_part->getReader(
|
||||
pre_columns_per_step, storage_snapshot, mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(),
|
||||
alter_conversions, reader_settings, value_size_map, profile_callback));
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReaders(MergeTreeReadTask & current_task)
|
||||
{
|
||||
return initializeRangeReadersImpl(
|
||||
current_task.range_reader, current_task.pre_range_readers, prewhere_actions,
|
||||
reader.get(), current_task.data_part->hasLightweightDelete(), reader_settings,
|
||||
pre_reader_for_step, lightweight_delete_filter_step, non_const_virtual_column_names);
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const std::vector<std::unique_ptr<IMergeTreeReader>> & pre_reader_for_step,
|
||||
const PrewhereExprStep & lightweight_delete_filter_step,
|
||||
const Names & non_const_virtual_column_names)
|
||||
{
|
||||
MergeTreeRangeReader * prev_reader = nullptr;
|
||||
bool last_reader = false;
|
||||
size_t pre_readers_shift = 0;
|
||||
|
||||
/// Add filtering step with lightweight delete mask
|
||||
if (reader_settings.apply_deleted_mask && has_lightweight_delete)
|
||||
{
|
||||
MergeTreeRangeReader pre_range_reader(pre_reader_for_step[0].get(), prev_reader, &lightweight_delete_filter_step, last_reader, non_const_virtual_column_names);
|
||||
pre_range_readers.push_back(std::move(pre_range_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
pre_readers_shift++;
|
||||
}
|
||||
|
||||
if (prewhere_actions.steps.size() + pre_readers_shift != pre_reader_for_step.size())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"PREWHERE steps count mismatch, actions: {}, readers: {}",
|
||||
prewhere_actions.steps.size(), pre_reader_for_step.size());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < prewhere_actions.steps.size(); ++i)
|
||||
{
|
||||
last_reader = reader->getColumns().empty() && (i + 1 == prewhere_actions.steps.size());
|
||||
|
||||
MergeTreeRangeReader current_reader(
|
||||
pre_reader_for_step[i + pre_readers_shift].get(),
|
||||
prev_reader, prewhere_actions.steps[i].get(),
|
||||
last_reader, non_const_virtual_column_names);
|
||||
|
||||
pre_range_readers.push_back(std::move(current_reader));
|
||||
prev_reader = &pre_range_readers.back();
|
||||
}
|
||||
|
||||
if (!last_reader)
|
||||
{
|
||||
range_reader = MergeTreeRangeReader(reader, prev_reader, nullptr, true, non_const_virtual_column_names);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If all columns are read by pre_range_readers than move last pre_range_reader into range_reader
|
||||
range_reader = std::move(pre_range_readers.back());
|
||||
pre_range_readers.pop_back();
|
||||
}
|
||||
}
|
||||
|
||||
static UInt64 estimateNumRows(const MergeTreeReadTask & current_task, UInt64 current_preferred_block_size_bytes,
|
||||
UInt64 current_max_block_size_rows, UInt64 current_preferred_max_column_in_block_size_bytes, double min_filtration_ratio, size_t min_marks_to_read)
|
||||
{
|
||||
const MergeTreeRangeReader & current_reader = current_task.range_reader;
|
||||
|
||||
if (!current_task.size_predictor)
|
||||
return static_cast<size_t>(current_max_block_size_rows);
|
||||
|
||||
/// Calculates number of rows will be read using preferred_block_size_bytes.
|
||||
/// Can't be less than avg_index_granularity.
|
||||
size_t rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes);
|
||||
if (!rows_to_read)
|
||||
return rows_to_read;
|
||||
auto total_row_in_current_granule = current_reader.numRowsInCurrentGranule();
|
||||
rows_to_read = std::max(total_row_in_current_granule, rows_to_read);
|
||||
|
||||
if (current_preferred_max_column_in_block_size_bytes)
|
||||
{
|
||||
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
|
||||
auto rows_to_read_for_max_size_column
|
||||
= current_task.size_predictor->estimateNumRowsForMaxSizeColumn(current_preferred_max_column_in_block_size_bytes);
|
||||
double filtration_ratio = std::max(min_filtration_ratio, 1.0 - current_task.size_predictor->filtered_rows_ratio);
|
||||
auto rows_to_read_for_max_size_column_with_filtration
|
||||
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
|
||||
|
||||
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than current_index_granularity.
|
||||
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
|
||||
}
|
||||
|
||||
auto unread_rows_in_current_granule = current_reader.numPendingRowsInCurrentGranule();
|
||||
if (unread_rows_in_current_granule >= rows_to_read)
|
||||
return rows_to_read;
|
||||
|
||||
const MergeTreeIndexGranularity & index_granularity = current_task.data_part->index_granularity;
|
||||
|
||||
return index_granularity.countMarksForRows(current_reader.currentMark(), rows_to_read, current_reader.numReadRowsInCurrentGranule(), min_marks_to_read);
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::BlockAndProgress IMergeTreeSelectAlgorithm::readFromPartImpl()
|
||||
{
|
||||
if (task->size_predictor)
|
||||
task->size_predictor->startBlock();
|
||||
|
||||
const UInt64 current_max_block_size_rows = max_block_size_rows;
|
||||
const UInt64 current_preferred_block_size_bytes = preferred_block_size_bytes;
|
||||
const UInt64 current_preferred_max_column_in_block_size_bytes = preferred_max_column_in_block_size_bytes;
|
||||
const double min_filtration_ratio = 0.00001;
|
||||
|
||||
UInt64 recommended_rows = estimateNumRows(*task, current_preferred_block_size_bytes,
|
||||
current_max_block_size_rows, current_preferred_max_column_in_block_size_bytes, min_filtration_ratio, min_marks_to_read);
|
||||
UInt64 rows_to_read = std::max(static_cast<UInt64>(1), std::min(current_max_block_size_rows, recommended_rows));
|
||||
|
||||
auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges);
|
||||
|
||||
/// All rows were filtered. Repeat.
|
||||
if (read_result.num_rows == 0)
|
||||
read_result.columns.clear();
|
||||
|
||||
const auto & sample_block = task->range_reader.getSampleBlock();
|
||||
if (read_result.num_rows != 0 && sample_block.columns() != read_result.columns.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent number of columns got from MergeTreeRangeReader. "
|
||||
"Have {} in sample block and {} columns in list",
|
||||
toString(sample_block.columns()), toString(read_result.columns.size()));
|
||||
|
||||
/// TODO: check columns have the same types as in header.
|
||||
|
||||
UInt64 num_filtered_rows = read_result.numReadRows() - read_result.num_rows;
|
||||
|
||||
size_t num_read_rows = read_result.numReadRows();
|
||||
size_t num_read_bytes = read_result.numBytesRead();
|
||||
|
||||
if (task->size_predictor)
|
||||
{
|
||||
task->size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows);
|
||||
|
||||
if (!read_result.columns.empty())
|
||||
task->size_predictor->update(sample_block, read_result.columns, read_result.num_rows);
|
||||
}
|
||||
|
||||
Block block;
|
||||
if (read_result.num_rows != 0)
|
||||
block = sample_block.cloneWithColumns(read_result.columns);
|
||||
|
||||
BlockAndProgress res = {
|
||||
.block = std::move(block),
|
||||
.row_count = read_result.num_rows,
|
||||
.num_read_rows = num_read_rows,
|
||||
.num_read_bytes = num_read_bytes };
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::BlockAndProgress IMergeTreeSelectAlgorithm::readFromPart()
|
||||
{
|
||||
if (!task->range_reader.isInitialized())
|
||||
initializeRangeReaders(*task);
|
||||
|
||||
return readFromPartImpl();
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
struct VirtualColumnsInserter
|
||||
{
|
||||
explicit VirtualColumnsInserter(Block & block_) : block(block_) {}
|
||||
|
||||
bool columnExists(const String & name) const { return block.has(name); }
|
||||
|
||||
void insertUInt8Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt8>(), name});
|
||||
}
|
||||
|
||||
void insertUInt64Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt64>(), name});
|
||||
}
|
||||
|
||||
void insertUUIDColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUUID>(), name});
|
||||
}
|
||||
|
||||
void insertLowCardinalityColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), name});
|
||||
}
|
||||
|
||||
void insertPartitionValueColumn(
|
||||
size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name)
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end()))
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = partition_value_type->createColumn();
|
||||
|
||||
block.insert({column, partition_value_type, name});
|
||||
}
|
||||
|
||||
Block & block;
|
||||
};
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are not const for all rows
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part_offset")
|
||||
{
|
||||
if (!rows)
|
||||
{
|
||||
inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!inserter.columnExists(virtual_column_name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column {} must have been filled part reader",
|
||||
virtual_column_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name)
|
||||
{
|
||||
/// If _row_exists column isn't present in the part then fill it here with 1s
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn();
|
||||
|
||||
inserter.insertUInt8Column(column, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are const for the whole part
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
/// add virtual columns
|
||||
/// Except _sample_factor, which is added from the outside.
|
||||
if (!virtual_columns.empty())
|
||||
{
|
||||
if (unlikely(rows && !task))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert virtual columns to non-empty chunk without specified task.");
|
||||
|
||||
const IMergeTreeDataPart * part = nullptr;
|
||||
if (rows)
|
||||
{
|
||||
part = task->data_part.get();
|
||||
if (part->isProjectionPart())
|
||||
part = part->getParentPart();
|
||||
}
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->name)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_index")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUInt64().createColumnConst(rows, task->part_index_in_query)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUInt64().createColumn();
|
||||
|
||||
inserter.insertUInt64Column(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_uuid")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUUID().createColumn();
|
||||
|
||||
inserter.insertUUIDColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_id")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->info.partition_id)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_value")
|
||||
{
|
||||
if (rows)
|
||||
inserter.insertPartitionValueColumn(rows, part->partition.value, partition_value_type, virtual_column_name);
|
||||
else
|
||||
inserter.insertPartitionValueColumn(rows, {}, partition_value_type, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeSelectAlgorithm::injectVirtualColumns(
|
||||
Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
/// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves.
|
||||
/// Note that the order is important: virtual columns filled by the range reader must go first
|
||||
injectNonConstVirtualColumns(row_count, block, virtual_columns);
|
||||
injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns);
|
||||
}
|
||||
|
||||
Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info)
|
||||
{
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
block = prewhere_info->row_level_filter->updateHeader(std::move(block));
|
||||
auto & row_level_column = block.getByName(prewhere_info->row_level_column_name);
|
||||
if (!row_level_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
row_level_column.type->getName());
|
||||
}
|
||||
|
||||
block.erase(prewhere_info->row_level_column_name);
|
||||
}
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
block = prewhere_info->prewhere_actions->updateHeader(std::move(block));
|
||||
|
||||
auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name);
|
||||
if (!prewhere_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
|
||||
if (prewhere_info->remove_prewhere_column)
|
||||
{
|
||||
block.erase(prewhere_info->prewhere_column_name);
|
||||
}
|
||||
else if (prewhere_info->need_filter)
|
||||
{
|
||||
WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type)));
|
||||
|
||||
if (which.isNativeInt() || which.isNativeUInt())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst();
|
||||
else if (which.isFloat())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f)->convertToFullColumnIfConst();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
|
||||
"Illegal type {} of column for filter",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
Block IMergeTreeSelectAlgorithm::transformHeader(
|
||||
Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
auto transformed = applyPrewhereActions(std::move(block), prewhere_info);
|
||||
injectVirtualColumns(transformed, 0, nullptr, partition_value_type, virtual_columns);
|
||||
return transformed;
|
||||
}
|
||||
|
||||
std::unique_ptr<MergeTreeBlockSizePredictor> IMergeTreeSelectAlgorithm::getSizePredictor(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const Block & sample_block)
|
||||
{
|
||||
const auto & required_column_names = task_columns.columns.getNames();
|
||||
NameSet complete_column_names(required_column_names.begin(), required_column_names.end());
|
||||
for (const auto & pre_columns_per_step : task_columns.pre_columns)
|
||||
{
|
||||
const auto & required_pre_column_names = pre_columns_per_step.getNames();
|
||||
complete_column_names.insert(required_pre_column_names.begin(), required_pre_column_names.end());
|
||||
}
|
||||
|
||||
return std::make_unique<MergeTreeBlockSizePredictor>(
|
||||
data_part, Names(complete_column_names.begin(), complete_column_names.end()), sample_block);
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeSelectAlgorithm::~IMergeTreeSelectAlgorithm() = default;
|
||||
|
||||
}
|
@ -1,217 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Processors/Chunk.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeReader;
|
||||
class UncompressedCache;
|
||||
class MarkCache;
|
||||
struct PrewhereExprInfo;
|
||||
|
||||
struct ChunkAndProgress
|
||||
{
|
||||
Chunk chunk;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
/// Explicitly indicate that we have read all data.
|
||||
/// This is needed to occasionally return empty chunk to indicate the progress while the rows are filtered out in PREWHERE.
|
||||
bool is_finished = false;
|
||||
};
|
||||
|
||||
struct ParallelReadingExtension
|
||||
{
|
||||
MergeTreeAllRangesCallback all_callback;
|
||||
MergeTreeReadTaskCallback callback;
|
||||
size_t count_participating_replicas{0};
|
||||
size_t number_of_current_replica{0};
|
||||
/// This is needed to estimate the number of bytes
|
||||
/// between a pair of marks to perform one request
|
||||
/// over the network for a 1Gb of data.
|
||||
Names columns_to_read;
|
||||
};
|
||||
|
||||
/// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm
|
||||
class IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
IMergeTreeSelectAlgorithm(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
UInt64 preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
bool use_uncompressed_cache_,
|
||||
const Names & virt_column_names_ = {});
|
||||
|
||||
virtual ~IMergeTreeSelectAlgorithm();
|
||||
|
||||
static Block transformHeader(
|
||||
Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns);
|
||||
|
||||
static std::unique_ptr<MergeTreeBlockSizePredictor> getSizePredictor(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const Block & sample_block);
|
||||
|
||||
Block getHeader() const { return result_header; }
|
||||
|
||||
ChunkAndProgress read();
|
||||
|
||||
void cancel() { is_cancelled = true; }
|
||||
|
||||
const MergeTreeReaderSettings & getSettings() const { return reader_settings; }
|
||||
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
static PrewhereExprInfo getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps);
|
||||
|
||||
protected:
|
||||
/// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk
|
||||
struct BlockAndProgress
|
||||
{
|
||||
Block block;
|
||||
size_t row_count = 0;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
};
|
||||
|
||||
/// Creates new this->task and return a flag whether it was successful or not
|
||||
virtual bool getNewTaskImpl() = 0;
|
||||
/// Creates new readers for a task it is needed. These methods are separate, because
|
||||
/// in case of parallel reading from replicas the whole task could be denied by a coodinator
|
||||
/// or it could modified somehow.
|
||||
virtual void finalizeNewTask() = 0;
|
||||
|
||||
size_t estimateMaxBatchSizeForHugeRanges();
|
||||
|
||||
/// Closes readers and unlock part locks
|
||||
virtual void finish() = 0;
|
||||
|
||||
virtual BlockAndProgress readFromPart();
|
||||
|
||||
BlockAndProgress readFromPartImpl();
|
||||
|
||||
/// Used for filling header with no rows as well as block with data
|
||||
static void
|
||||
injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns);
|
||||
|
||||
protected:
|
||||
static void initializeRangeReadersImpl(
|
||||
MergeTreeRangeReader & range_reader,
|
||||
std::deque<MergeTreeRangeReader> & pre_range_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
IMergeTreeReader * reader,
|
||||
bool has_lightweight_delete,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
const std::vector<std::unique_ptr<IMergeTreeReader>> & pre_reader_for_step,
|
||||
const PrewhereExprStep & lightweight_delete_filter_step,
|
||||
const Names & non_const_virtual_column_names);
|
||||
|
||||
/// Sets up data readers for each step of prewhere and where
|
||||
void initializeMergeTreeReadersForCurrentTask(
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
void initializeMergeTreeReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
/// Sets up range readers corresponding to data readers
|
||||
void initializeRangeReaders(MergeTreeReadTask & task);
|
||||
|
||||
const MergeTreeData & storage;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
|
||||
/// This step is added when the part has lightweight delete mask
|
||||
const PrewhereExprStep lightweight_delete_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = nullptr,
|
||||
.filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
ExpressionActionsSettings actions_settings;
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
|
||||
UInt64 max_block_size_rows;
|
||||
UInt64 preferred_block_size_bytes;
|
||||
UInt64 preferred_max_column_in_block_size_bytes;
|
||||
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
|
||||
bool use_uncompressed_cache;
|
||||
|
||||
Names virt_column_names;
|
||||
|
||||
/// These columns will be filled by the merge tree range reader
|
||||
Names non_const_virtual_column_names;
|
||||
|
||||
DataTypePtr partition_value_type;
|
||||
|
||||
/// This header is used for chunks from readFromPart().
|
||||
Block header_without_const_virtual_columns;
|
||||
/// A result of getHeader(). A chunk which this header is returned from read().
|
||||
Block result_header;
|
||||
|
||||
UncompressedCachePtr owned_uncompressed_cache;
|
||||
MarkCachePtr owned_mark_cache;
|
||||
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
MergeTreeReaderPtr reader;
|
||||
std::vector<MergeTreeReaderPtr> pre_reader_for_step;
|
||||
|
||||
MergeTreeReadTaskPtr task;
|
||||
|
||||
/// This setting is used in base algorithm only to additionally limit the number of granules to read.
|
||||
/// It is changed in ctor of MergeTreeThreadSelectAlgorithm.
|
||||
///
|
||||
/// The reason why we have it here is because MergeTreeReadPool takes the full task
|
||||
/// ignoring min_marks_to_read setting in case of remote disk (see MergeTreeReadPool::getTask).
|
||||
/// In this case, we won't limit the number of rows to read based on adaptive granularity settings.
|
||||
///
|
||||
/// Big reading tasks are better for remote disk and prefetches.
|
||||
/// So, for now it's easier to limit max_rows_to_read.
|
||||
/// Somebody need to refactor this later.
|
||||
size_t min_marks_to_read = 0;
|
||||
|
||||
private:
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeBaseSelectProcessor");
|
||||
|
||||
std::atomic<bool> is_cancelled{false};
|
||||
|
||||
bool getNewTask();
|
||||
|
||||
/// Initialize pre readers.
|
||||
void initializeMergeTreePreReadersForPart(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MergeTreeReadTaskColumns & task_columns,
|
||||
const MarkRanges & mark_ranges,
|
||||
const IMergeTreeReader::ValueSizeMap & value_size_map,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback);
|
||||
|
||||
static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info);
|
||||
};
|
||||
|
||||
using MergeTreeSelectAlgorithmPtr = std::unique_ptr<IMergeTreeSelectAlgorithm>;
|
||||
|
||||
}
|
@ -6,7 +6,7 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
@ -16,6 +16,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -136,43 +137,6 @@ NameSet injectRequiredColumns(
|
||||
return injected_columns;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadTask::MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
const MergeTreeReadTaskColumns & task_columns_,
|
||||
MergeTreeBlockSizePredictorPtr size_predictor_,
|
||||
Priority priority_,
|
||||
std::future<MergeTreeReaderPtr> reader_,
|
||||
std::vector<std::future<MergeTreeReaderPtr>> && pre_reader_for_step_)
|
||||
: data_part{data_part_}
|
||||
, alter_conversions{alter_conversions_}
|
||||
, mark_ranges{mark_ranges_}
|
||||
, part_index_in_query{part_index_in_query_}
|
||||
, column_name_set{column_name_set_}
|
||||
, task_columns{task_columns_}
|
||||
, size_predictor{size_predictor_}
|
||||
, reader(std::move(reader_))
|
||||
, pre_reader_for_step(std::move(pre_reader_for_step_))
|
||||
, priority(priority_)
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeReadTask::~MergeTreeReadTask()
|
||||
{
|
||||
if (reader.valid())
|
||||
reader.wait();
|
||||
|
||||
for (const auto & pre_reader : pre_reader_for_step)
|
||||
{
|
||||
if (pre_reader.valid())
|
||||
pre_reader.wait();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor(
|
||||
const DataPartPtr & data_part_, const Names & columns, const Block & sample_block)
|
||||
: data_part(data_part_)
|
||||
@ -195,9 +159,8 @@ void MergeTreeBlockSizePredictor::initialize(const Block & sample_block, const C
|
||||
for (size_t pos = 0; pos < num_columns; ++pos)
|
||||
{
|
||||
const auto & column_with_type_and_name = sample_block.getByPosition(pos);
|
||||
const String & column_name = column_with_type_and_name.name;
|
||||
const ColumnPtr & column_data = from_update ? columns[pos]
|
||||
: column_with_type_and_name.column;
|
||||
const auto & column_name = column_with_type_and_name.name;
|
||||
const auto & column_data = from_update ? columns[pos] : column_with_type_and_name.column;
|
||||
|
||||
if (!from_update && !names_set.contains(column_name))
|
||||
continue;
|
||||
@ -246,7 +209,6 @@ void MergeTreeBlockSizePredictor::startBlock()
|
||||
info.size_bytes = 0;
|
||||
}
|
||||
|
||||
|
||||
/// TODO: add last_read_row_in_part parameter to take into account gaps between adjacent ranges
|
||||
void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Columns & columns, size_t num_rows, double decay)
|
||||
{
|
||||
@ -296,7 +258,7 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
MergeTreeReadTask::Columns getReadTaskColumns(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const Names & required_columns,
|
||||
@ -317,7 +279,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
injectRequiredColumns(
|
||||
data_part_info_for_reader, storage_snapshot, with_subcolumns, column_to_read_after_prewhere);
|
||||
|
||||
MergeTreeReadTaskColumns result;
|
||||
MergeTreeReadTask::Columns result;
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::All)
|
||||
.withExtendedObjects()
|
||||
.withSystemColumns();
|
||||
@ -365,7 +327,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(
|
||||
auto prewhere_actions = MergeTreeSelectProcessor::getPrewhereActions(
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings.enable_multiple_prewhere_read_steps);
|
||||
@ -387,16 +349,4 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
std::string MergeTreeReadTaskColumns::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString s;
|
||||
for (size_t i = 0; i < pre_columns.size(); ++i)
|
||||
{
|
||||
s << "STEP " << i << ": " << pre_columns[i].toString() << "\n";
|
||||
}
|
||||
s << "COLUMNS: " << columns.toString() << "\n";
|
||||
return s.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,29 +2,15 @@
|
||||
|
||||
#include <optional>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeData;
|
||||
struct MergeTreeReadTask;
|
||||
struct MergeTreeReaderSettings;
|
||||
struct MergeTreeBlockSizePredictor;
|
||||
class IMergeTreeDataPartInfoForReader;
|
||||
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredictor>;
|
||||
|
||||
class IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
||||
|
||||
|
||||
/** If some of the requested columns are not in the part,
|
||||
* then find out which columns may need to be read further,
|
||||
* so that you can calculate the DEFAULT expression for these columns.
|
||||
@ -36,64 +22,7 @@ NameSet injectRequiredColumns(
|
||||
bool with_subcolumns,
|
||||
Names & columns);
|
||||
|
||||
struct MergeTreeReadTaskColumns
|
||||
{
|
||||
/// column names to read during WHERE
|
||||
NamesAndTypesList columns;
|
||||
/// column names to read during each PREWHERE step
|
||||
std::vector<NamesAndTypesList> pre_columns;
|
||||
|
||||
std::string dump() const;
|
||||
};
|
||||
|
||||
/// A batch of work for MergeTreeThreadSelectProcessor
|
||||
struct MergeTreeReadTask
|
||||
{
|
||||
/// Data part which should be read while performing this task
|
||||
DataPartPtr data_part;
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
/// Ranges to read from `data_part`.
|
||||
MarkRanges mark_ranges;
|
||||
/// for virtual `part_index` virtual column
|
||||
size_t part_index_in_query;
|
||||
/// used to determine whether column should be filtered during PREWHERE or WHERE
|
||||
const NameSet & column_name_set;
|
||||
/// column names to read during PREWHERE and WHERE
|
||||
const MergeTreeReadTaskColumns & task_columns;
|
||||
/// Used to satistfy preferred_block_size_bytes limitation
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
/// Used to save current range processing status
|
||||
MergeTreeRangeReader range_reader;
|
||||
/// Range readers for multiple filtering steps: row level security, PREWHERE etc.
|
||||
/// NOTE: we take references to elements and push_back new elements, that's why it is a deque but not a vector
|
||||
std::deque<MergeTreeRangeReader> pre_range_readers;
|
||||
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
std::future<MergeTreeReaderPtr> reader;
|
||||
std::vector<std::future<MergeTreeReaderPtr>> pre_reader_for_step;
|
||||
|
||||
Priority priority;
|
||||
|
||||
bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); }
|
||||
|
||||
MergeTreeReadTask(
|
||||
const DataPartPtr & data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
const MarkRanges & mark_ranges_,
|
||||
size_t part_index_in_query_,
|
||||
const NameSet & column_name_set_,
|
||||
const MergeTreeReadTaskColumns & task_columns_,
|
||||
MergeTreeBlockSizePredictorPtr size_predictor_,
|
||||
Priority priority_ = {},
|
||||
std::future<MergeTreeReaderPtr> reader_ = {},
|
||||
std::vector<std::future<MergeTreeReaderPtr>> && pre_reader_for_step_ = {});
|
||||
|
||||
~MergeTreeReadTask();
|
||||
};
|
||||
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
MergeTreeReadTask::Columns getReadTaskColumns(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
const Names & required_columns,
|
||||
@ -119,7 +48,6 @@ struct MergeTreeBlockSizePredictor
|
||||
return block_size_bytes;
|
||||
}
|
||||
|
||||
|
||||
/// Predicts what number of rows should be read to exhaust byte quota per column
|
||||
inline size_t estimateNumRowsForMaxSizeColumn(size_t bytes_quota) const
|
||||
{
|
||||
@ -153,7 +81,6 @@ struct MergeTreeBlockSizePredictor
|
||||
static double calculateDecay() { return 1. - std::pow(TARGET_WEIGHT, 1. / NUM_UPDATES_TO_TARGET_WEIGHT); }
|
||||
|
||||
protected:
|
||||
|
||||
DataPartPtr data_part;
|
||||
|
||||
struct ColumnInfo
|
||||
|
@ -68,7 +68,7 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/Freeze.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
|
@ -1,76 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
|
||||
#include "Storages/MergeTree/RangesInDataPart.h"
|
||||
#include <Storages/MergeTree/IntersectionsIndexes.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
bool MergeTreeInOrderSelectAlgorithm::getNewTaskImpl()
|
||||
try
|
||||
{
|
||||
if (all_mark_ranges.empty())
|
||||
return false;
|
||||
|
||||
if (!reader)
|
||||
initializeReaders();
|
||||
|
||||
MarkRanges mark_ranges_for_task;
|
||||
|
||||
if (!pool)
|
||||
{
|
||||
/// If we need to read few rows, set one range per task to reduce number of read data.
|
||||
if (has_limit_below_one_block)
|
||||
{
|
||||
mark_ranges_for_task = MarkRanges{};
|
||||
mark_ranges_for_task.emplace_front(std::move(all_mark_ranges.front()));
|
||||
all_mark_ranges.pop_front();
|
||||
}
|
||||
else
|
||||
{
|
||||
mark_ranges_for_task = std::move(all_mark_ranges);
|
||||
all_mark_ranges.clear();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto description = RangesInDataPartDescription{
|
||||
.info = data_part->info,
|
||||
/// We just ignore all the distribution done before
|
||||
/// Everything will be done on coordinator side
|
||||
.ranges = {},
|
||||
};
|
||||
|
||||
mark_ranges_for_task = pool->getNewTask(description);
|
||||
|
||||
if (mark_ranges_for_task.empty())
|
||||
return false;
|
||||
}
|
||||
|
||||
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Suspicion of the broken part. A part is added to the queue for verification.
|
||||
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
storage.reportBrokenPart(data_part);
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
@ -1,33 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Used to read data from single part with select query in order of primary key.
|
||||
/// Cares about PREWHERE, virtual columns, indexes etc.
|
||||
/// To read data from multiple parts, Storage (MergeTree) creates multiple such objects.
|
||||
class MergeTreeInOrderSelectAlgorithm final : public MergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeInOrderSelectAlgorithm(Args &&... args)
|
||||
: MergeTreeSelectAlgorithm{std::forward<Args>(args)...}
|
||||
{
|
||||
LOG_TRACE(log, "Reading {} ranges in order from part {}, approx. {} rows starting from {}",
|
||||
all_mark_ranges.size(), data_part->name, total_rows,
|
||||
data_part->index_granularity.getMarkStartingRow(all_mark_ranges.front().begin));
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeTreeInOrder"; }
|
||||
|
||||
private:
|
||||
bool getNewTaskImpl() override;
|
||||
void finalizeNewTask() override {}
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeInOrderSelectProcessor");
|
||||
};
|
||||
|
||||
}
|
@ -5,7 +5,6 @@
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
@ -14,10 +13,10 @@
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MergeTreePrefetchedReadPoolInit;
|
||||
extern const Event WaitPrefetchTaskMicroseconds;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -29,145 +28,124 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
bool MergeTreePrefetchedReadPool::TaskHolder::operator<(const TaskHolder & other) const
|
||||
{
|
||||
chassert(task->priority >= 0);
|
||||
chassert(other.task->priority >= 0);
|
||||
/// With default std::priority_queue, top() returns largest element.
|
||||
/// So closest to 0 will be on top with this comparator.
|
||||
return task->priority > other.task->priority; /// Less is better.
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::PrefetechedReaders::PrefetechedReaders(
|
||||
MergeTreeReadTask::Readers readers_,
|
||||
Priority priority_,
|
||||
MergeTreePrefetchedReadPool & pool_)
|
||||
: is_valid(true)
|
||||
, readers(std::move(readers_))
|
||||
{
|
||||
prefetch_futures.push_back(pool_.createPrefetchedFuture(readers.main.get(), priority_));
|
||||
|
||||
for (const auto & reader : readers.prewhere)
|
||||
prefetch_futures.push_back(pool_.createPrefetchedFuture(reader.get(), priority_));
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::PrefetechedReaders::wait()
|
||||
{
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
for (auto & prefetch_future : prefetch_futures)
|
||||
prefetch_future.wait();
|
||||
}
|
||||
|
||||
MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetechedReaders::get()
|
||||
{
|
||||
SCOPE_EXIT({ is_valid = false; });
|
||||
ProfileEventTimeIncrement<Microseconds> watch(ProfileEvents::WaitPrefetchTaskMicroseconds);
|
||||
for (auto & prefetch_future : prefetch_futures)
|
||||
prefetch_future.get();
|
||||
|
||||
return std::move(readers);
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
|
||||
size_t threads,
|
||||
size_t sum_marks_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
ContextPtr context_,
|
||||
bool use_uncompressed_cache_,
|
||||
bool is_remote_read_,
|
||||
const MergeTreeSettings & storage_settings_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_.empty() ? "" : parts_.front().data_part->storage.getStorageID().getNameForLogs()) + ")"))
|
||||
, header(storage_snapshot_->getSampleBlockForColumns(column_names_))
|
||||
, mark_cache(context_->getGlobalContext()->getMarkCache().get())
|
||||
, uncompressed_cache(use_uncompressed_cache_ ? context_->getGlobalContext()->getUncompressedCache().get() : nullptr)
|
||||
, profile_callback([this](ReadBufferFromFileBase::ProfileInfo info_) { profileFeedback(info_); })
|
||||
, index_granularity_bytes(storage_settings_.index_granularity_bytes)
|
||||
, fixed_index_granularity(storage_settings_.index_granularity)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, column_names(column_names_)
|
||||
, virtual_column_names(virtual_column_names_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, reader_settings(reader_settings_)
|
||||
, is_remote_read(is_remote_read_)
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, WithContext(context_)
|
||||
, prefetch_threadpool(getContext()->getPrefetchThreadpool())
|
||||
, log(&Poco::Logger::get("MergeTreePrefetchedReadPool(" + (parts_.empty() ? "" : parts_.front().data_part->storage.getStorageID().getNameForLogs()) + ")"))
|
||||
{
|
||||
/// Tasks creation might also create a lost of readers - check they do not
|
||||
/// do any time consuming operations in ctor.
|
||||
ProfileEventTimeIncrement<Milliseconds> watch(ProfileEvents::MergeTreePrefetchedReadPoolInit);
|
||||
|
||||
parts_infos = getPartsInfos(parts_, preferred_block_size_bytes_);
|
||||
threads_tasks = createThreadsTasks(threads, sum_marks_, min_marks_for_concurrent_read_);
|
||||
fillPerPartStatistics();
|
||||
fillPerThreadTasks(pool_settings.threads, pool_settings.sum_marks);
|
||||
}
|
||||
|
||||
struct MergeTreePrefetchedReadPool::PartInfo
|
||||
std::future<void> MergeTreePrefetchedReadPool::createPrefetchedFuture(IMergeTreeReader * reader, Priority priority)
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
AlterConversionsPtr alter_conversions;
|
||||
size_t part_index_in_query;
|
||||
size_t sum_marks = 0;
|
||||
MarkRanges ranges;
|
||||
|
||||
NameSet column_name_set;
|
||||
MergeTreeReadTaskColumns task_columns;
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
|
||||
size_t approx_size_of_mark = 0;
|
||||
size_t prefetch_step_marks = 0;
|
||||
|
||||
size_t estimated_memory_usage_for_single_prefetch = 0;
|
||||
size_t required_readers_num = 0;
|
||||
};
|
||||
|
||||
std::future<MergeTreeReaderPtr> MergeTreePrefetchedReadPool::createPrefetchedReader(
|
||||
const IMergeTreeDataPart & data_part,
|
||||
const NamesAndTypesList & columns,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MarkRanges & required_ranges,
|
||||
Priority priority) const
|
||||
{
|
||||
auto reader = data_part.getReader(
|
||||
columns, storage_snapshot, required_ranges,
|
||||
uncompressed_cache, mark_cache, alter_conversions, reader_settings,
|
||||
IMergeTreeReader::ValueSizeMap{}, profile_callback);
|
||||
|
||||
/// In order to make a prefetch we need to wait for marks to be loaded. But we just created
|
||||
/// a reader (which starts loading marks in its constructor), then if we do prefetch right
|
||||
/// after creating a reader, it will be very inefficient. We can do prefetch for all parts
|
||||
/// only inside this MergeTreePrefetchedReadPool, where read tasks are created and distributed,
|
||||
/// and we cannot block either, therefore make prefetch inside the pool and put the future
|
||||
/// into the read task (MergeTreeReadTask). When a thread calls getTask(), it will wait for
|
||||
/// it (if not yet ready) after getting the task.
|
||||
auto task = [=, my_reader = std::move(reader), context = getContext()]() mutable -> MergeTreeReaderPtr &&
|
||||
/// into the thread task. When a thread calls getTask(), it will wait for it is not ready yet.
|
||||
auto task = [=, context = getContext()]() mutable
|
||||
{
|
||||
/// For async read metrics in system.query_log.
|
||||
PrefetchIncrement watch(context->getAsyncReadCounters());
|
||||
|
||||
my_reader->prefetchBeginOfRange(priority);
|
||||
return std::move(my_reader);
|
||||
reader->prefetchBeginOfRange(priority);
|
||||
};
|
||||
return scheduleFromThreadPool<IMergeTreeDataPart::MergeTreeReaderPtr>(std::move(task), prefetch_threadpool, "ReadPrepare", priority);
|
||||
|
||||
return scheduleFromThreadPool<void>(std::move(task), prefetch_threadpool, "ReadPrepare", priority);
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::createPrefetchedReaderForTask(MergeTreeReadTask & task) const
|
||||
void MergeTreePrefetchedReadPool::createPrefetchedReadersForTask(ThreadTask & task)
|
||||
{
|
||||
if (task.reader.valid())
|
||||
if (task.readers_future.valid())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Task already has a reader");
|
||||
|
||||
task.reader = createPrefetchedReader(*task.data_part, task.task_columns.columns, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
|
||||
if (reader_settings.apply_deleted_mask && task.data_part->hasLightweightDelete())
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, {LightweightDeleteDescription::FILTER_COLUMN}, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
|
||||
for (const auto & pre_columns_per_step : task.task_columns.pre_columns)
|
||||
{
|
||||
auto pre_reader = createPrefetchedReader(*task.data_part, pre_columns_per_step, task.alter_conversions, task.mark_ranges, task.priority);
|
||||
task.pre_reader_for_step.push_back(std::move(pre_reader));
|
||||
}
|
||||
auto extras = getExtras();
|
||||
auto readers = MergeTreeReadTask::createReaders(task.read_info, extras, task.ranges);
|
||||
task.readers_future = PrefetechedReaders(std::move(readers), task.priority, *this);
|
||||
}
|
||||
|
||||
bool MergeTreePrefetchedReadPool::TaskHolder::operator <(const TaskHolder & other) const
|
||||
{
|
||||
chassert(task->priority >= 0);
|
||||
chassert(other.task->priority >= 0);
|
||||
return task->priority > other.task->priority; /// Less is better.
|
||||
/// With default std::priority_queue, top() returns largest element.
|
||||
/// So closest to 0 will be on top with this comparator.
|
||||
}
|
||||
|
||||
void MergeTreePrefetchedReadPool::startPrefetches() const
|
||||
void MergeTreePrefetchedReadPool::startPrefetches()
|
||||
{
|
||||
if (prefetch_queue.empty())
|
||||
return;
|
||||
|
||||
[[maybe_unused]] TaskHolder prev(nullptr, 0);
|
||||
[[maybe_unused]] TaskHolder prev;
|
||||
[[maybe_unused]] const Priority highest_priority{reader_settings.read_settings.priority.value + 1};
|
||||
assert(prefetch_queue.top().task->priority == highest_priority);
|
||||
|
||||
while (!prefetch_queue.empty())
|
||||
{
|
||||
const auto & top = prefetch_queue.top();
|
||||
createPrefetchedReaderForTask(*top.task);
|
||||
createPrefetchedReadersForTask(*top.task);
|
||||
#ifndef NDEBUG
|
||||
if (prev.task)
|
||||
{
|
||||
assert(top.task->priority >= highest_priority);
|
||||
if (prev.thread_id == top.thread_id)
|
||||
{
|
||||
assert(prev.task->priority < top.task->priority);
|
||||
}
|
||||
}
|
||||
prev = top;
|
||||
#endif
|
||||
@ -175,11 +153,11 @@ void MergeTreePrefetchedReadPool::startPrefetches() const
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t thread)
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t task_idx, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (threads_tasks.empty())
|
||||
if (per_thread_tasks.empty())
|
||||
return nullptr;
|
||||
|
||||
if (!started_prefetches)
|
||||
@ -188,112 +166,129 @@ MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::getTask(size_t thread)
|
||||
startPrefetches();
|
||||
}
|
||||
|
||||
auto it = threads_tasks.find(thread);
|
||||
if (it == threads_tasks.end())
|
||||
{
|
||||
ThreadsTasks::iterator non_prefetched_tasks_to_steal = threads_tasks.end();
|
||||
ThreadsTasks::iterator prefetched_tasks_to_steal = threads_tasks.end();
|
||||
int64_t best_prefetched_task_priority = -1;
|
||||
|
||||
/// There is no point stealing in order (like in MergeTreeReadPool, where tasks can be stolen
|
||||
/// only from the next thread). Even if we steal task from the next thread, which reads from
|
||||
/// the same part as we just read, it might seem that we can reuse our own reader, do some
|
||||
/// seek avoiding and it will have a good result as we avoided seek (new request). But it is
|
||||
/// not so, because this next task will most likely have its own reader a prefetch already on
|
||||
/// the fly. (Not to mention that in fact we cannot reuse our own reader if initially we did
|
||||
/// not accounted this range into range request to object storage).
|
||||
for (auto thread_tasks_it = threads_tasks.begin(); thread_tasks_it != threads_tasks.end(); ++thread_tasks_it)
|
||||
{
|
||||
/// Prefer to steal tasks which have an initialized reader (with prefetched data). Thus we avoid
|
||||
/// losing a prefetch by creating our own reader (or resusing our own reader if the part
|
||||
/// is the same as last read by this thread).
|
||||
auto & thread_tasks = thread_tasks_it->second;
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->reader.valid(); });
|
||||
|
||||
if (task_it == thread_tasks.end())
|
||||
{
|
||||
/// The follow back to non-prefetched task should lie on the thread which
|
||||
/// has more tasks than others.
|
||||
if (non_prefetched_tasks_to_steal == threads_tasks.end()
|
||||
|| non_prefetched_tasks_to_steal->second.size() < thread_tasks.size())
|
||||
non_prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
/// Try to steal task with the best (lowest) priority (because it will be executed faster).
|
||||
else if (prefetched_tasks_to_steal == threads_tasks.end()
|
||||
|| (*task_it)->priority < best_prefetched_task_priority)
|
||||
{
|
||||
best_prefetched_task_priority = (*task_it)->priority;
|
||||
chassert(best_prefetched_task_priority >= 0);
|
||||
prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
}
|
||||
|
||||
if (prefetched_tasks_to_steal != threads_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->reader.valid(); });
|
||||
assert(task_it != thread_tasks.end());
|
||||
|
||||
auto task = std::move(*task_it);
|
||||
thread_tasks.erase(task_it);
|
||||
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(prefetched_tasks_to_steal);
|
||||
|
||||
return task;
|
||||
}
|
||||
|
||||
/// TODO: it also makes sense to first try to steal from the next thread if it has ranges
|
||||
/// from the same part as current thread last read - to reuse the reader.
|
||||
|
||||
if (non_prefetched_tasks_to_steal != threads_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = non_prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
/// Get second half of the tasks.
|
||||
const size_t total_tasks = thread_tasks.size();
|
||||
const size_t half = total_tasks / 2;
|
||||
auto half_it = thread_tasks.begin() + half;
|
||||
assert(half_it != thread_tasks.end());
|
||||
|
||||
/// Give them to current thread, as current thread's tasks list is empty.
|
||||
auto & current_thread_tasks = threads_tasks[thread];
|
||||
current_thread_tasks.insert(
|
||||
current_thread_tasks.end(), make_move_iterator(half_it), make_move_iterator(thread_tasks.end()));
|
||||
|
||||
/// Erase them from the thread from which we steal.
|
||||
thread_tasks.resize(half);
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(non_prefetched_tasks_to_steal);
|
||||
|
||||
auto task = std::move(current_thread_tasks.front());
|
||||
current_thread_tasks.erase(current_thread_tasks.begin());
|
||||
if (current_thread_tasks.empty())
|
||||
threads_tasks.erase(thread);
|
||||
|
||||
return task;
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
auto it = per_thread_tasks.find(task_idx);
|
||||
if (it == per_thread_tasks.end())
|
||||
return stealTask(task_idx, previous_task);
|
||||
|
||||
auto & thread_tasks = it->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task = std::move(thread_tasks.front());
|
||||
auto thread_task = std::move(thread_tasks.front());
|
||||
thread_tasks.pop_front();
|
||||
|
||||
if (thread_tasks.empty())
|
||||
threads_tasks.erase(it);
|
||||
per_thread_tasks.erase(it);
|
||||
|
||||
return task;
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::stealTask(size_t thread, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
auto non_prefetched_tasks_to_steal = per_thread_tasks.end();
|
||||
auto prefetched_tasks_to_steal = per_thread_tasks.end();
|
||||
int64_t best_prefetched_task_priority = -1;
|
||||
|
||||
/// There is no point stealing in order (like in MergeTreeReadPool, where tasks can be stolen
|
||||
/// only from the next thread). Even if we steal task from the next thread, which reads from
|
||||
/// the same part as we just read, it might seem that we can reuse our own reader, do some
|
||||
/// seek avoiding and it will have a good result as we avoided seek (new request). But it is
|
||||
/// not so, because this next task will most likely have its own reader a prefetch already on
|
||||
/// the fly. (Not to mention that in fact we cannot reuse our own reader if initially we did
|
||||
/// not accounted this range into range request to object storage).
|
||||
for (auto thread_tasks_it = per_thread_tasks.begin(); thread_tasks_it != per_thread_tasks.end(); ++thread_tasks_it)
|
||||
{
|
||||
/// Prefer to steal tasks which have an initialized reader (with prefetched data). Thus we avoid
|
||||
/// losing a prefetch by creating our own reader (or resusing our own reader if the part
|
||||
/// is the same as last read by this thread).
|
||||
auto & thread_tasks = thread_tasks_it->second;
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->readers_future.valid(); });
|
||||
|
||||
if (task_it == thread_tasks.end())
|
||||
{
|
||||
/// The follow back to non-prefetched task should lie on the thread which
|
||||
/// has more tasks than others.
|
||||
if (non_prefetched_tasks_to_steal == per_thread_tasks.end()
|
||||
|| non_prefetched_tasks_to_steal->second.size() < thread_tasks.size())
|
||||
non_prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
/// Try to steal task with the best (lowest) priority (because it will be executed faster).
|
||||
else if (prefetched_tasks_to_steal == per_thread_tasks.end()
|
||||
|| (*task_it)->priority < best_prefetched_task_priority)
|
||||
{
|
||||
best_prefetched_task_priority = (*task_it)->priority;
|
||||
chassert(best_prefetched_task_priority >= 0);
|
||||
prefetched_tasks_to_steal = thread_tasks_it;
|
||||
}
|
||||
}
|
||||
|
||||
if (prefetched_tasks_to_steal != per_thread_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
auto task_it = std::find_if(
|
||||
thread_tasks.begin(), thread_tasks.end(),
|
||||
[](const auto & task) { return task->readers_future.valid(); });
|
||||
|
||||
assert(task_it != thread_tasks.end());
|
||||
auto thread_task = std::move(*task_it);
|
||||
thread_tasks.erase(task_it);
|
||||
|
||||
if (thread_tasks.empty())
|
||||
per_thread_tasks.erase(prefetched_tasks_to_steal);
|
||||
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
/// TODO: it also makes sense to first try to steal from the next thread if it has ranges
|
||||
/// from the same part as current thread last read - to reuse the reader.
|
||||
if (non_prefetched_tasks_to_steal != per_thread_tasks.end())
|
||||
{
|
||||
auto & thread_tasks = non_prefetched_tasks_to_steal->second;
|
||||
assert(!thread_tasks.empty());
|
||||
|
||||
/// Get second half of the tasks.
|
||||
const size_t total_tasks = thread_tasks.size();
|
||||
const size_t half = total_tasks / 2;
|
||||
auto half_it = thread_tasks.begin() + half;
|
||||
assert(half_it != thread_tasks.end());
|
||||
|
||||
/// Give them to current thread, as current thread's tasks list is empty.
|
||||
auto & current_thread_tasks = per_thread_tasks[thread];
|
||||
current_thread_tasks.insert(
|
||||
current_thread_tasks.end(), make_move_iterator(half_it), make_move_iterator(thread_tasks.end()));
|
||||
|
||||
/// Erase them from the thread from which we steal.
|
||||
thread_tasks.resize(half);
|
||||
if (thread_tasks.empty())
|
||||
per_thread_tasks.erase(non_prefetched_tasks_to_steal);
|
||||
|
||||
auto thread_task = std::move(current_thread_tasks.front());
|
||||
current_thread_tasks.erase(current_thread_tasks.begin());
|
||||
if (current_thread_tasks.empty())
|
||||
per_thread_tasks.erase(thread);
|
||||
|
||||
return createTask(*thread_task, previous_task);
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::createTask(ThreadTask & task, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
if (task.readers_future.valid())
|
||||
{
|
||||
auto size_predictor = task.read_info->shared_size_predictor
|
||||
? std::make_unique<MergeTreeBlockSizePredictor>(*task.read_info->shared_size_predictor)
|
||||
: nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(task.read_info, task.readers_future.get(), task.ranges, std::move(size_predictor));
|
||||
}
|
||||
|
||||
return MergeTreeReadPoolBase::createTask(task.read_info, task.ranges, previous_task);
|
||||
}
|
||||
|
||||
size_t getApproximateSizeOfGranule(const IMergeTreeDataPart & part, const Names & columns_to_read)
|
||||
@ -304,154 +299,111 @@ size_t getApproximateSizeOfGranule(const IMergeTreeDataPart & part, const Names
|
||||
return columns_size.data_compressed / part.getMarksCount();
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInfos(
|
||||
const RangesInDataParts & parts, size_t preferred_block_size_bytes) const
|
||||
void MergeTreePrefetchedReadPool::fillPerPartStatistics()
|
||||
{
|
||||
PartsInfos result;
|
||||
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||
per_part_statistics.clear();
|
||||
per_part_statistics.reserve(parts_ranges.size());
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
const bool predict_block_size_bytes = preferred_block_size_bytes > 0;
|
||||
|
||||
for (const auto & part : parts)
|
||||
for (size_t i = 0; i < parts_ranges.size(); ++i)
|
||||
{
|
||||
auto part_info = std::make_unique<PartInfo>();
|
||||
|
||||
part_info->data_part = part.data_part;
|
||||
part_info->alter_conversions = part.alter_conversions;
|
||||
part_info->part_index_in_query = part.part_index_in_query;
|
||||
part_info->ranges = part.ranges;
|
||||
std::sort(part_info->ranges.begin(), part_info->ranges.end());
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_reader_info(part.data_part, part_info->alter_conversions);
|
||||
auto & part_stat = per_part_statistics.emplace_back();
|
||||
const auto & read_info = *per_part_infos[i];
|
||||
|
||||
/// Sum up total size of all mark ranges in a data part.
|
||||
for (const auto & range : part.ranges)
|
||||
part_info->sum_marks += range.end - range.begin;
|
||||
for (const auto & range : parts_ranges[i].ranges)
|
||||
part_stat.sum_marks += range.end - range.begin;
|
||||
|
||||
const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info
|
||||
? prewhere_info->prewhere_actions->getRequiredColumnsNames()
|
||||
: column_names;
|
||||
part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, columns);
|
||||
|
||||
const auto task_columns = getReadTaskColumns(
|
||||
part_reader_info,
|
||||
storage_snapshot,
|
||||
column_names,
|
||||
virtual_column_names,
|
||||
prewhere_info,
|
||||
actions_settings,
|
||||
reader_settings,
|
||||
/* with_subcolumns */ true);
|
||||
part_stat.approx_size_of_mark = getApproximateSizeOfGranule(*read_info.data_part, columns);
|
||||
|
||||
part_info->size_predictor = !predict_block_size_bytes
|
||||
? nullptr
|
||||
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
|
||||
|
||||
/// Will be used to distinguish between PREWHERE and WHERE columns when applying filter.
|
||||
const auto & required_column_names = task_columns.columns.getNames();
|
||||
part_info->column_name_set = {required_column_names.begin(), required_column_names.end()};
|
||||
part_info->task_columns = task_columns;
|
||||
auto update_stat_for_column = [&](const auto & column_name)
|
||||
{
|
||||
size_t column_size = read_info.data_part->getColumnSize(column_name).data_compressed;
|
||||
part_stat.estimated_memory_usage_for_single_prefetch += std::min<size_t>(column_size, settings.prefetch_buffer_size);
|
||||
++part_stat.required_readers_num;
|
||||
};
|
||||
|
||||
/// adjustBufferSize(), which is done in MergeTreeReaderStream and MergeTreeReaderCompact,
|
||||
/// lowers buffer size if file size (or required read range) is less. So we know that the
|
||||
/// settings.prefetch_buffer_size will be lowered there, therefore we account it here as well.
|
||||
/// But here we make a more approximate lowering (because we do not have loaded marks yet),
|
||||
/// while in adjustBufferSize it will be presize.
|
||||
for (const auto & col : task_columns.columns)
|
||||
{
|
||||
const auto col_size = part.data_part->getColumnSize(col.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
if (reader_settings.apply_deleted_mask && part.data_part->hasLightweightDelete())
|
||||
{
|
||||
const auto col_size = part.data_part->getColumnSize(
|
||||
LightweightDeleteDescription::FILTER_COLUMN.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
if (prewhere_info)
|
||||
{
|
||||
for (const auto & cols : task_columns.pre_columns)
|
||||
{
|
||||
for (const auto & col : cols)
|
||||
{
|
||||
const size_t col_size = part.data_part->getColumnSize(col.name).data_compressed;
|
||||
part_info->estimated_memory_usage_for_single_prefetch += std::min<size_t>(col_size, settings.prefetch_buffer_size);
|
||||
++part_info->required_readers_num;
|
||||
}
|
||||
}
|
||||
}
|
||||
for (const auto & column : read_info.task_columns.columns)
|
||||
update_stat_for_column(column.name);
|
||||
|
||||
result.push_back(std::move(part_info));
|
||||
if (reader_settings.apply_deleted_mask && read_info.data_part->hasLightweightDelete())
|
||||
update_stat_for_column(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
|
||||
for (const auto & pre_columns : read_info.task_columns.pre_columns)
|
||||
for (const auto & column : pre_columns)
|
||||
update_stat_for_column(column.name);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThreadsTasks(
|
||||
size_t threads, size_t sum_marks, size_t /* min_marks_for_concurrent_read */) const
|
||||
void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_marks)
|
||||
{
|
||||
if (parts_infos.empty())
|
||||
return {};
|
||||
if (per_part_infos.empty())
|
||||
return;
|
||||
|
||||
const auto & context = getContext();
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
size_t total_size_approx = 0;
|
||||
for (const auto & part : parts_infos)
|
||||
{
|
||||
total_size_approx += part->sum_marks * part->approx_size_of_mark;
|
||||
}
|
||||
for (const auto & part : per_part_statistics)
|
||||
total_size_approx += part.sum_marks * part.approx_size_of_mark;
|
||||
|
||||
size_t min_prefetch_step_marks = 0;
|
||||
|
||||
for (const auto & part : parts_infos)
|
||||
for (size_t i = 0; i < per_part_infos.size(); ++i)
|
||||
{
|
||||
auto & part_stat = per_part_statistics[i];
|
||||
|
||||
if (settings.filesystem_prefetch_step_marks)
|
||||
{
|
||||
part->prefetch_step_marks = settings.filesystem_prefetch_step_marks;
|
||||
part_stat.prefetch_step_marks = settings.filesystem_prefetch_step_marks;
|
||||
}
|
||||
else if (settings.filesystem_prefetch_step_bytes && part->approx_size_of_mark)
|
||||
else if (settings.filesystem_prefetch_step_bytes && part_stat.approx_size_of_mark)
|
||||
{
|
||||
part->prefetch_step_marks = std::max<size_t>(
|
||||
1, static_cast<size_t>(std::round(static_cast<double>(settings.filesystem_prefetch_step_bytes) / part->approx_size_of_mark)));
|
||||
part_stat.prefetch_step_marks = std::max<size_t>(
|
||||
1, static_cast<size_t>(std::round(static_cast<double>(settings.filesystem_prefetch_step_bytes) / part_stat.approx_size_of_mark)));
|
||||
}
|
||||
|
||||
/// This limit is important to avoid spikes of slow aws getObject requests when parallelizing within one file.
|
||||
/// (The default is taken from here https://docs.aws.amazon.com/whitepapers/latest/s3-optimizing-performance-best-practices/use-byte-range-fetches.html).
|
||||
if (part->approx_size_of_mark
|
||||
if (part_stat.approx_size_of_mark
|
||||
&& settings.filesystem_prefetch_min_bytes_for_single_read_task
|
||||
&& part->approx_size_of_mark < settings.filesystem_prefetch_min_bytes_for_single_read_task)
|
||||
&& part_stat.approx_size_of_mark < settings.filesystem_prefetch_min_bytes_for_single_read_task)
|
||||
{
|
||||
const size_t min_prefetch_step_marks_by_total_cols = static_cast<size_t>(
|
||||
std::ceil(static_cast<double>(settings.filesystem_prefetch_min_bytes_for_single_read_task) / part->approx_size_of_mark));
|
||||
std::ceil(static_cast<double>(settings.filesystem_prefetch_min_bytes_for_single_read_task) / part_stat.approx_size_of_mark));
|
||||
|
||||
/// At least one task to start working on it right now and another one to prefetch in the meantime.
|
||||
const size_t new_min_prefetch_step_marks = std::min<size_t>(min_prefetch_step_marks_by_total_cols, sum_marks / threads / 2);
|
||||
if (min_prefetch_step_marks < new_min_prefetch_step_marks)
|
||||
{
|
||||
LOG_DEBUG(log, "Increasing min prefetch step from {} to {}", min_prefetch_step_marks, new_min_prefetch_step_marks);
|
||||
|
||||
min_prefetch_step_marks = new_min_prefetch_step_marks;
|
||||
}
|
||||
}
|
||||
|
||||
if (part->prefetch_step_marks < min_prefetch_step_marks)
|
||||
if (part_stat.prefetch_step_marks < min_prefetch_step_marks)
|
||||
{
|
||||
LOG_DEBUG(log, "Increasing prefetch step from {} to {}", part->prefetch_step_marks, min_prefetch_step_marks);
|
||||
|
||||
part->prefetch_step_marks = min_prefetch_step_marks;
|
||||
LOG_DEBUG(log, "Increasing prefetch step from {} to {}", part_stat.prefetch_step_marks, min_prefetch_step_marks);
|
||||
part_stat.prefetch_step_marks = min_prefetch_step_marks;
|
||||
}
|
||||
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})",
|
||||
part->data_part->name,
|
||||
part->sum_marks,
|
||||
part->approx_size_of_mark,
|
||||
parts_ranges[i].data_part->name,
|
||||
part_stat.sum_marks,
|
||||
part_stat.approx_size_of_mark,
|
||||
settings.filesystem_prefetch_step_bytes,
|
||||
part->prefetch_step_marks,
|
||||
toString(part->ranges));
|
||||
part_stat.prefetch_step_marks,
|
||||
toString(parts_ranges[i].ranges));
|
||||
}
|
||||
|
||||
const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1;
|
||||
@ -469,13 +421,24 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr
|
||||
size_t allowed_memory_usage = settings.filesystem_prefetch_max_memory_usage;
|
||||
if (!allowed_memory_usage)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `filesystem_prefetch_max_memory_usage` must be non-zero");
|
||||
|
||||
std::optional<size_t> allowed_prefetches_num = settings.filesystem_prefetches_limit
|
||||
? std::optional<size_t>(settings.filesystem_prefetches_limit)
|
||||
: std::nullopt;
|
||||
|
||||
ThreadsTasks result_threads_tasks;
|
||||
per_thread_tasks.clear();
|
||||
size_t total_tasks = 0;
|
||||
for (size_t i = 0, part_idx = 0; i < threads && part_idx < parts_infos.size(); ++i)
|
||||
|
||||
/// Make a copy to modify ranges.
|
||||
std::vector<MarkRanges> per_part_ranges;
|
||||
per_part_ranges.reserve(parts_ranges.size());
|
||||
for (const auto & part_with_ranges : parts_ranges)
|
||||
{
|
||||
auto & part_ranges = per_part_ranges.emplace_back(part_with_ranges.ranges);
|
||||
std::sort(part_ranges.begin(), part_ranges.end());
|
||||
}
|
||||
|
||||
for (size_t i = 0, part_idx = 0; i < threads && part_idx < per_part_infos.size(); ++i)
|
||||
{
|
||||
int64_t need_marks = min_marks_per_thread;
|
||||
|
||||
@ -486,119 +449,102 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr
|
||||
/// reads from pool which are from reader.
|
||||
Priority priority{reader_settings.read_settings.priority.value + 1};
|
||||
|
||||
while (need_marks > 0 && part_idx < parts_infos.size())
|
||||
while (need_marks > 0 && part_idx < per_part_infos.size())
|
||||
{
|
||||
auto & part = *parts_infos[part_idx];
|
||||
size_t & marks_in_part = part.sum_marks;
|
||||
auto & part_stat = per_part_statistics[part_idx];
|
||||
auto & part_ranges = per_part_ranges[part_idx];
|
||||
|
||||
if (marks_in_part == 0)
|
||||
if (part_stat.sum_marks == 0)
|
||||
{
|
||||
++part_idx;
|
||||
continue;
|
||||
}
|
||||
|
||||
MarkRanges ranges_to_get_from_part;
|
||||
size_t marks_to_get_from_part = std::min<size_t>(need_marks, marks_in_part);
|
||||
size_t marks_to_get_from_part = std::min<size_t>(need_marks, part_stat.sum_marks);
|
||||
|
||||
/// Split by prefetch step even if !allow_prefetch below. Because it will allow
|
||||
/// to make a better distribution of tasks which did not fill into memory limit
|
||||
/// or prefetches limit through tasks stealing.
|
||||
if (part.prefetch_step_marks)
|
||||
if (part_stat.prefetch_step_marks)
|
||||
{
|
||||
marks_to_get_from_part = std::min<size_t>(marks_to_get_from_part, part.prefetch_step_marks);
|
||||
marks_to_get_from_part = std::min<size_t>(marks_to_get_from_part, part_stat.prefetch_step_marks);
|
||||
}
|
||||
|
||||
if (marks_in_part == marks_to_get_from_part)
|
||||
if (part_stat.sum_marks == marks_to_get_from_part)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
ranges_to_get_from_part = part_ranges;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (part.sum_marks < marks_to_get_from_part)
|
||||
if (part_stat.sum_marks < marks_to_get_from_part)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Requested {} marks from part {}, but part has only {} marks",
|
||||
marks_to_get_from_part, part.data_part->name, part.sum_marks);
|
||||
marks_to_get_from_part, per_part_infos[part_idx]->data_part->name, part_stat.sum_marks);
|
||||
}
|
||||
|
||||
size_t get_marks_num = marks_to_get_from_part;
|
||||
while (get_marks_num > 0)
|
||||
size_t num_marks_to_get = marks_to_get_from_part;
|
||||
while (num_marks_to_get > 0)
|
||||
{
|
||||
MarkRange & range = part.ranges.front();
|
||||
MarkRange & range = part_ranges.front();
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, get_marks_num);
|
||||
get_marks_num -= marks_to_get_from_range;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, num_marks_to_get);
|
||||
num_marks_to_get -= marks_to_get_from_range;
|
||||
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range);
|
||||
range.begin += marks_to_get_from_range;
|
||||
|
||||
if (range.begin == range.end)
|
||||
{
|
||||
part.ranges.pop_front();
|
||||
part_ranges.pop_front();
|
||||
}
|
||||
else if (!get_marks_num && part.prefetch_step_marks && range.end - range.begin < part.prefetch_step_marks)
|
||||
else if (!num_marks_to_get && part_stat.prefetch_step_marks && range.end - range.begin < part_stat.prefetch_step_marks)
|
||||
{
|
||||
/// We already have `get_marks_num` marks, but current mark range has
|
||||
/// We already have `num_marks_to_get` marks, but current mark range has
|
||||
/// less than `prefetch_step_marks` marks, then add them too.
|
||||
ranges_to_get_from_part.emplace_back(range.begin, range.end);
|
||||
marks_to_get_from_part += range.end - range.begin;
|
||||
part.ranges.pop_front();
|
||||
part_ranges.pop_front();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
need_marks -= marks_to_get_from_part;
|
||||
sum_marks -= marks_to_get_from_part;
|
||||
marks_in_part -= marks_to_get_from_part;
|
||||
|
||||
auto curr_task_size_predictor = !part.size_predictor ? nullptr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(*part.size_predictor); /// make a copy
|
||||
|
||||
auto read_task = std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_get_from_part,
|
||||
part.part_index_in_query,
|
||||
part.column_name_set,
|
||||
part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
|
||||
read_task->priority = priority;
|
||||
part_stat.sum_marks -= marks_to_get_from_part;
|
||||
|
||||
bool allow_prefetch = false;
|
||||
if (allowed_memory_usage
|
||||
&& (allowed_prefetches_num.has_value() == false || allowed_prefetches_num.value() > 0))
|
||||
&& (!allowed_prefetches_num.has_value() || allowed_prefetches_num.value() > 0))
|
||||
{
|
||||
allow_prefetch = part.estimated_memory_usage_for_single_prefetch <= allowed_memory_usage
|
||||
&& (allowed_prefetches_num.has_value() == false
|
||||
|| part.required_readers_num <= allowed_prefetches_num.value());
|
||||
allow_prefetch = part_stat.estimated_memory_usage_for_single_prefetch <= allowed_memory_usage
|
||||
&& (!allowed_prefetches_num.has_value() || part_stat.required_readers_num <= allowed_prefetches_num.value());
|
||||
|
||||
if (allow_prefetch)
|
||||
{
|
||||
allowed_memory_usage -= part.estimated_memory_usage_for_single_prefetch;
|
||||
allowed_memory_usage -= part_stat.estimated_memory_usage_for_single_prefetch;
|
||||
if (allowed_prefetches_num.has_value())
|
||||
*allowed_prefetches_num -= part.required_readers_num;
|
||||
*allowed_prefetches_num -= part_stat.required_readers_num;
|
||||
}
|
||||
}
|
||||
|
||||
auto thread_task = std::make_unique<ThreadTask>(per_part_infos[part_idx], ranges_to_get_from_part, priority);
|
||||
if (allow_prefetch)
|
||||
{
|
||||
prefetch_queue.emplace(TaskHolder(read_task.get(), i));
|
||||
}
|
||||
++priority.value;
|
||||
prefetch_queue.emplace(TaskHolder{thread_task.get(), i});
|
||||
|
||||
result_threads_tasks[i].push_back(std::move(read_task));
|
||||
per_thread_tasks[i].push_back(std::move(thread_task));
|
||||
|
||||
++priority.value;
|
||||
++total_tasks;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_TEST(log, "Result tasks {} for {} threads: {}", total_tasks, threads, dumpTasks(result_threads_tasks));
|
||||
|
||||
return result_threads_tasks;
|
||||
LOG_TEST(log, "Result tasks {} for {} threads: {}", total_tasks, threads, dumpTasks(per_thread_tasks));
|
||||
}
|
||||
|
||||
std::string MergeTreePrefetchedReadPool::dumpTasks(const ThreadsTasks & tasks)
|
||||
std::string MergeTreePrefetchedReadPool::dumpTasks(const TasksPerThread & tasks)
|
||||
{
|
||||
WriteBufferFromOwnString result;
|
||||
for (const auto & [thread_id, thread_tasks] : tasks)
|
||||
@ -611,9 +557,9 @@ std::string MergeTreePrefetchedReadPool::dumpTasks(const ThreadsTasks & tasks)
|
||||
{
|
||||
result << '\t';
|
||||
result << ++no << ": ";
|
||||
result << "reader: " << task->reader.valid() << ", ";
|
||||
result << "part: " << task->data_part->name << ", ";
|
||||
result << "ranges: " << toString(task->mark_ranges);
|
||||
result << "reader future: " << task->readers_future.valid() << ", ";
|
||||
result << "part: " << task->read_info->data_part->name << ", ";
|
||||
result << "ranges: " << toString(task->ranges);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,9 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Interpreters/ExpressionActionsSettings.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeIOSettings.h>
|
||||
#include <IO/AsyncReadCounters.h>
|
||||
#include <boost/heap/priority_queue.hpp>
|
||||
#include <queue>
|
||||
@ -16,99 +13,110 @@ using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
/// A class which is responsible for creating read tasks
|
||||
/// which are later taken by readers via getTask method.
|
||||
/// Does prefetching for the read tasks it creates.
|
||||
class MergeTreePrefetchedReadPool : public IMergeTreeReadPool, private WithContext
|
||||
class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithContext
|
||||
{
|
||||
public:
|
||||
MergeTreePrefetchedReadPool(
|
||||
size_t threads,
|
||||
size_t sum_marks_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
ContextPtr context_,
|
||||
bool use_uncompressed_cache_,
|
||||
bool is_remote_read_,
|
||||
const MergeTreeSettings & storage_settings_);
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
MergeTreeReadTaskPtr getTask(size_t thread) override;
|
||||
String getName() const override { return "PrefetchedReadPool"; }
|
||||
bool preservesOrderOfRanges() const override { return false; }
|
||||
MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) override;
|
||||
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
|
||||
static bool checkReadMethodAllowed(LocalFSReadMethod method);
|
||||
static bool checkReadMethodAllowed(RemoteFSReadMethod method);
|
||||
|
||||
private:
|
||||
struct PartInfo;
|
||||
using PartInfoPtr = std::shared_ptr<PartInfo>;
|
||||
using PartsInfos = std::vector<PartInfoPtr>;
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
using ThreadTasks = std::deque<MergeTreeReadTaskPtr>;
|
||||
using ThreadsTasks = std::map<size_t, ThreadTasks>;
|
||||
struct PartStatistic
|
||||
{
|
||||
size_t sum_marks = 0;
|
||||
|
||||
std::future<MergeTreeReaderPtr> createPrefetchedReader(
|
||||
const IMergeTreeDataPart & data_part,
|
||||
const NamesAndTypesList & columns,
|
||||
const AlterConversionsPtr & alter_conversions,
|
||||
const MarkRanges & required_ranges,
|
||||
Priority priority) const;
|
||||
size_t approx_size_of_mark = 0;
|
||||
size_t prefetch_step_marks = 0;
|
||||
|
||||
void createPrefetchedReaderForTask(MergeTreeReadTask & task) const;
|
||||
size_t estimated_memory_usage_for_single_prefetch = 0;
|
||||
size_t required_readers_num = 0;
|
||||
};
|
||||
|
||||
size_t getApproxSizeOfGranule(const IMergeTreeDataPart & part) const;
|
||||
class PrefetechedReaders
|
||||
{
|
||||
public:
|
||||
PrefetechedReaders() = default;
|
||||
PrefetechedReaders(MergeTreeReadTask::Readers readers_, Priority priority_, MergeTreePrefetchedReadPool & pool_);
|
||||
|
||||
PartsInfos getPartsInfos(const RangesInDataParts & parts, size_t preferred_block_size_bytes) const;
|
||||
void wait();
|
||||
MergeTreeReadTask::Readers get();
|
||||
bool valid() const { return is_valid; }
|
||||
|
||||
ThreadsTasks createThreadsTasks(
|
||||
size_t threads,
|
||||
size_t sum_marks,
|
||||
size_t min_marks_for_concurrent_read) const;
|
||||
private:
|
||||
bool is_valid = false;
|
||||
MergeTreeReadTask::Readers readers;
|
||||
std::vector<std::future<void>> prefetch_futures;
|
||||
};
|
||||
|
||||
void startPrefetches() const;
|
||||
struct ThreadTask
|
||||
{
|
||||
using InfoPtr = MergeTreeReadTask::InfoPtr;
|
||||
|
||||
static std::string dumpTasks(const ThreadsTasks & tasks);
|
||||
ThreadTask(InfoPtr read_info_, MarkRanges ranges_, Priority priority_)
|
||||
: read_info(std::move(read_info_)), ranges(std::move(ranges_)), priority(priority_)
|
||||
{
|
||||
}
|
||||
|
||||
Poco::Logger * log;
|
||||
~ThreadTask()
|
||||
{
|
||||
if (readers_future.valid())
|
||||
readers_future.wait();
|
||||
}
|
||||
|
||||
Block header;
|
||||
MarkCache * mark_cache;
|
||||
UncompressedCache * uncompressed_cache;
|
||||
ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
size_t index_granularity_bytes;
|
||||
size_t fixed_index_granularity;
|
||||
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
const Names column_names;
|
||||
const Names virtual_column_names;
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
const ExpressionActionsSettings actions_settings;
|
||||
const MergeTreeReaderSettings reader_settings;
|
||||
RangesInDataParts parts_ranges;
|
||||
|
||||
[[ maybe_unused ]] const bool is_remote_read;
|
||||
ThreadPool & prefetch_threadpool;
|
||||
|
||||
PartsInfos parts_infos;
|
||||
|
||||
ThreadsTasks threads_tasks;
|
||||
std::mutex mutex;
|
||||
InfoPtr read_info;
|
||||
MarkRanges ranges;
|
||||
Priority priority;
|
||||
PrefetechedReaders readers_future;
|
||||
};
|
||||
|
||||
struct TaskHolder
|
||||
{
|
||||
explicit TaskHolder(MergeTreeReadTask * task_, size_t thread_id_) : task(task_), thread_id(thread_id_) {}
|
||||
MergeTreeReadTask * task;
|
||||
size_t thread_id;
|
||||
bool operator <(const TaskHolder & other) const;
|
||||
ThreadTask * task = nullptr;
|
||||
size_t thread_id = 0;
|
||||
bool operator<(const TaskHolder & other) const;
|
||||
};
|
||||
mutable std::priority_queue<TaskHolder> prefetch_queue; /// the smallest on top
|
||||
|
||||
using ThreadTaskPtr = std::unique_ptr<ThreadTask>;
|
||||
using ThreadTasks = std::deque<ThreadTaskPtr>;
|
||||
using TasksPerThread = std::map<size_t, ThreadTasks>;
|
||||
using PartStatistics = std::vector<PartStatistic>;
|
||||
|
||||
void fillPerPartStatistics();
|
||||
void fillPerThreadTasks(size_t threads, size_t sum_marks);
|
||||
|
||||
void startPrefetches();
|
||||
void createPrefetchedReadersForTask(ThreadTask & task);
|
||||
std::future<void> createPrefetchedFuture(IMergeTreeReader * reader, Priority priority);
|
||||
|
||||
MergeTreeReadTaskPtr stealTask(size_t thread, MergeTreeReadTask * previous_task);
|
||||
MergeTreeReadTaskPtr createTask(ThreadTask & thread_task, MergeTreeReadTask * previous_task);
|
||||
|
||||
static std::string dumpTasks(const TasksPerThread & tasks);
|
||||
|
||||
mutable std::mutex mutex;
|
||||
ThreadPool & prefetch_threadpool;
|
||||
|
||||
PartStatistics per_part_statistics;
|
||||
TasksPerThread per_thread_tasks;
|
||||
std::priority_queue<TaskHolder> prefetch_queue; /// the smallest on top
|
||||
bool started_prefetches = false;
|
||||
Poco::Logger * log;
|
||||
|
||||
/// A struct which allows to track max number of tasks which were in the
|
||||
/// threadpool simultaneously (similar to CurrentMetrics, but the result
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include "Storages/MergeTree/MergeTreeBlockReadUtils.h"
|
||||
#include "Storages/MergeTree/MergeTreeReadTask.h"
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <base/range.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
@ -32,9 +33,6 @@ size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & column
|
||||
}
|
||||
|
||||
MergeTreeReadPool::MergeTreeReadPool(
|
||||
size_t threads_,
|
||||
size_t sum_marks_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
@ -42,29 +40,22 @@ MergeTreeReadPool::MergeTreeReadPool(
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
ContextPtr context_,
|
||||
bool do_not_steal_tasks_)
|
||||
: storage_snapshot(storage_snapshot_)
|
||||
, column_names(column_names_)
|
||||
, virtual_column_names(virtual_column_names_)
|
||||
, min_marks_for_concurrent_read(min_marks_for_concurrent_read_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, reader_settings(reader_settings_)
|
||||
, parts_ranges(std::move(parts_))
|
||||
, predict_block_size_bytes(context_->getSettingsRef().preferred_block_size_bytes > 0)
|
||||
, do_not_steal_tasks(do_not_steal_tasks_)
|
||||
, merge_tree_use_const_size_tasks_for_remote_reading(context_->getSettingsRef().merge_tree_use_const_size_tasks_for_remote_reading)
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, min_marks_for_concurrent_read(pool_settings.min_marks_for_concurrent_read)
|
||||
, backoff_settings{context_->getSettingsRef()}
|
||||
, backoff_state{threads_}
|
||||
, backoff_state{pool_settings.threads}
|
||||
{
|
||||
/// parts don't contain duplicate MergeTreeDataPart's.
|
||||
const auto per_part_sum_marks = fillPerPartInfo(
|
||||
parts_ranges, storage_snapshot, is_part_on_remote_disk,
|
||||
predict_block_size_bytes,
|
||||
column_names, virtual_column_names, prewhere_info,
|
||||
actions_settings, reader_settings, per_part_params);
|
||||
|
||||
if (std::ranges::count(is_part_on_remote_disk, true))
|
||||
{
|
||||
const auto & settings = context_->getSettingsRef();
|
||||
@ -76,6 +67,7 @@ MergeTreeReadPool::MergeTreeReadPool(
|
||||
const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info
|
||||
? prewhere_info->prewhere_actions->getRequiredColumnsNames()
|
||||
: column_names_;
|
||||
|
||||
total_compressed_bytes += getApproxSizeOfPart(*part.data_part, columns);
|
||||
total_marks += part.getMarksCount();
|
||||
}
|
||||
@ -85,118 +77,60 @@ MergeTreeReadPool::MergeTreeReadPool(
|
||||
const auto min_bytes_per_task = settings.merge_tree_min_bytes_per_task_for_remote_reading;
|
||||
const auto avg_mark_bytes = std::max<size_t>(total_compressed_bytes / total_marks, 1);
|
||||
/// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible.
|
||||
const auto heuristic_min_marks = std::min<size_t>(total_marks / threads_, min_bytes_per_task / avg_mark_bytes);
|
||||
const auto heuristic_min_marks = std::min<size_t>(total_marks / pool_settings.threads, min_bytes_per_task / avg_mark_bytes);
|
||||
|
||||
if (heuristic_min_marks > min_marks_for_concurrent_read)
|
||||
{
|
||||
min_marks_for_concurrent_read = heuristic_min_marks;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges);
|
||||
fillPerThreadInfo(pool_settings.threads, pool_settings.sum_marks);
|
||||
}
|
||||
|
||||
std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(
|
||||
const RangesInDataParts & parts,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
std::vector<bool> & is_part_on_remote_disk,
|
||||
bool & predict_block_size_bytes,
|
||||
const Names & column_names,
|
||||
const Names & virtual_column_names,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const ExpressionActionsSettings & actions_settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
std::vector<MergeTreeReadPool::PerPartParams> & per_part_params)
|
||||
{
|
||||
std::vector<size_t> per_part_sum_marks;
|
||||
Block sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||
is_part_on_remote_disk.resize(parts.size());
|
||||
|
||||
for (const auto i : collections::range(0, parts.size()))
|
||||
{
|
||||
const auto & part = parts[i];
|
||||
#ifndef NDEBUG
|
||||
assertSortedAndNonIntersecting(part.ranges);
|
||||
#endif
|
||||
|
||||
bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk();
|
||||
is_part_on_remote_disk[i] = part_on_remote_disk;
|
||||
|
||||
/// Read marks for every data part.
|
||||
size_t sum_marks = 0;
|
||||
for (const auto & range : part.ranges)
|
||||
sum_marks += range.end - range.begin;
|
||||
|
||||
per_part_sum_marks.push_back(sum_marks);
|
||||
|
||||
auto & per_part = per_part_params.emplace_back();
|
||||
per_part.data_part = part;
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_info(part.data_part, part.alter_conversions);
|
||||
auto task_columns = getReadTaskColumns(
|
||||
part_info, storage_snapshot, column_names, virtual_column_names,
|
||||
prewhere_info, actions_settings,
|
||||
reader_settings, /*with_subcolumns=*/ true);
|
||||
|
||||
auto size_predictor = !predict_block_size_bytes ? nullptr
|
||||
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
|
||||
|
||||
per_part.size_predictor = std::move(size_predictor);
|
||||
|
||||
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const auto & required_column_names = task_columns.columns.getNames();
|
||||
per_part.column_name_set = {required_column_names.begin(), required_column_names.end()};
|
||||
per_part.task_columns = std::move(task_columns);
|
||||
}
|
||||
|
||||
return per_part_sum_marks;
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread)
|
||||
MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t task_idx, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
const std::lock_guard lock{mutex};
|
||||
|
||||
/// If number of threads was lowered due to backoff, then will assign work only for maximum 'backoff_state.current_threads' threads.
|
||||
if (thread >= backoff_state.current_threads)
|
||||
if (task_idx >= backoff_state.current_threads)
|
||||
return nullptr;
|
||||
|
||||
if (remaining_thread_tasks.empty())
|
||||
return nullptr;
|
||||
|
||||
const auto tasks_remaining_for_this_thread = !threads_tasks[thread].sum_marks_in_parts.empty();
|
||||
if (!tasks_remaining_for_this_thread && do_not_steal_tasks)
|
||||
const auto tasks_remaining_for_this_thread = !threads_tasks[task_idx].sum_marks_in_parts.empty();
|
||||
if (!tasks_remaining_for_this_thread && pool_settings.do_not_steal_tasks)
|
||||
return nullptr;
|
||||
|
||||
/// Steal task if nothing to do and it's not prohibited
|
||||
auto thread_idx = thread;
|
||||
auto thread_idx = task_idx;
|
||||
if (!tasks_remaining_for_this_thread)
|
||||
{
|
||||
auto it = remaining_thread_tasks.lower_bound(backoff_state.current_threads);
|
||||
// Grab the entire tasks of a thread which is killed by backoff
|
||||
if (it != remaining_thread_tasks.end())
|
||||
{
|
||||
threads_tasks[thread] = std::move(threads_tasks[*it]);
|
||||
threads_tasks[task_idx] = std::move(threads_tasks[*it]);
|
||||
remaining_thread_tasks.erase(it);
|
||||
remaining_thread_tasks.insert(thread);
|
||||
remaining_thread_tasks.insert(task_idx);
|
||||
}
|
||||
else // Try steal tasks from the next thread
|
||||
{
|
||||
it = remaining_thread_tasks.upper_bound(thread);
|
||||
it = remaining_thread_tasks.upper_bound(task_idx);
|
||||
if (it == remaining_thread_tasks.end())
|
||||
it = remaining_thread_tasks.begin();
|
||||
thread_idx = *it;
|
||||
}
|
||||
}
|
||||
|
||||
auto & thread_tasks = threads_tasks[thread_idx];
|
||||
|
||||
auto & thread_task = thread_tasks.parts_and_ranges.back();
|
||||
const auto part_idx = thread_task.part_idx;
|
||||
|
||||
auto & part = per_part_params[part_idx].data_part;
|
||||
const auto part_idx = thread_task.part_idx;
|
||||
auto & marks_in_part = thread_tasks.sum_marks_in_parts.back();
|
||||
|
||||
size_t need_marks;
|
||||
if (is_part_on_remote_disk[part_idx] && !merge_tree_use_const_size_tasks_for_remote_reading)
|
||||
if (is_part_on_remote_disk[part_idx] && !pool_settings.use_const_size_tasks_for_remote_reading)
|
||||
need_marks = marks_in_part;
|
||||
else /// Get whole part to read if it is small enough.
|
||||
need_marks = std::min(marks_in_part, min_marks_for_concurrent_read);
|
||||
@ -239,28 +173,12 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread)
|
||||
}
|
||||
}
|
||||
|
||||
const auto & per_part = per_part_params[part_idx];
|
||||
auto curr_task_size_predictor = !per_part.size_predictor ? nullptr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(*per_part.size_predictor); /// make a copy
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_get_from_part,
|
||||
part.part_index_in_query,
|
||||
per_part.column_name_set,
|
||||
per_part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
}
|
||||
|
||||
Block MergeTreeReadPool::getHeader() const
|
||||
{
|
||||
return storage_snapshot->getSampleBlockForColumns(column_names);
|
||||
return createTask(per_part_infos[part_idx], std::move(ranges_to_get_from_part), previous_task);
|
||||
}
|
||||
|
||||
void MergeTreeReadPool::profileFeedback(ReadBufferFromFileBase::ProfileInfo info)
|
||||
{
|
||||
if (backoff_settings.min_read_latency_ms == 0 || do_not_steal_tasks)
|
||||
if (backoff_settings.min_read_latency_ms == 0 || pool_settings.do_not_steal_tasks)
|
||||
return;
|
||||
|
||||
if (info.nanoseconds < backoff_settings.min_read_latency_ms * 1000000)
|
||||
@ -297,13 +215,10 @@ void MergeTreeReadPool::profileFeedback(ReadBufferFromFileBase::ProfileInfo info
|
||||
LOG_DEBUG(log, "Will lower number of threads to {}", backoff_state.current_threads);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeReadPool::fillPerThreadInfo(
|
||||
size_t threads, size_t sum_marks, std::vector<size_t> per_part_sum_marks,
|
||||
const RangesInDataParts & parts)
|
||||
void MergeTreeReadPool::fillPerThreadInfo(size_t threads, size_t sum_marks)
|
||||
{
|
||||
threads_tasks.resize(threads);
|
||||
if (parts.empty())
|
||||
if (parts_ranges.empty())
|
||||
return;
|
||||
|
||||
struct PartInfo
|
||||
@ -316,17 +231,19 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
using PartsInfo = std::vector<PartInfo>;
|
||||
std::queue<PartsInfo> parts_queue;
|
||||
|
||||
auto per_part_sum_marks = getPerPartSumMarks();
|
||||
|
||||
{
|
||||
/// Group parts by disk name.
|
||||
/// We try minimize the number of threads concurrently read from the same disk.
|
||||
/// It improves the performance for JBOD architecture.
|
||||
std::map<String, std::vector<PartInfo>> parts_per_disk;
|
||||
|
||||
for (size_t i = 0; i < parts.size(); ++i)
|
||||
for (size_t i = 0; i < parts_ranges.size(); ++i)
|
||||
{
|
||||
PartInfo part_info{parts[i], per_part_sum_marks[i], i};
|
||||
if (parts[i].data_part->isStoredOnDisk())
|
||||
parts_per_disk[parts[i].data_part->getDataPartStorage().getDiskName()].push_back(std::move(part_info));
|
||||
PartInfo part_info{parts_ranges[i], per_part_sum_marks[i], i};
|
||||
if (parts_ranges[i].data_part->isStoredOnDisk())
|
||||
parts_per_disk[parts_ranges[i].data_part->getDataPartStorage().getDiskName()].push_back(std::move(part_info));
|
||||
else
|
||||
parts_per_disk[""].push_back(std::move(part_info));
|
||||
}
|
||||
@ -346,7 +263,7 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
while (need_marks > 0 && !parts_queue.empty())
|
||||
{
|
||||
auto & current_parts = parts_queue.front();
|
||||
RangesInDataPart & part = current_parts.back().part;
|
||||
auto & part_with_ranges = current_parts.back().part;
|
||||
size_t & marks_in_part = current_parts.back().sum_marks;
|
||||
const auto part_idx = current_parts.back().part_idx;
|
||||
|
||||
@ -366,7 +283,7 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
/// Get whole part to read if it is small enough.
|
||||
if (marks_in_part <= need_marks)
|
||||
{
|
||||
ranges_to_get_from_part = part.ranges;
|
||||
ranges_to_get_from_part = part_with_ranges.ranges;
|
||||
marks_in_ranges = marks_in_part;
|
||||
|
||||
need_marks -= marks_in_part;
|
||||
@ -379,10 +296,10 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
/// Loop through part ranges.
|
||||
while (need_marks > 0)
|
||||
{
|
||||
if (part.ranges.empty())
|
||||
if (part_with_ranges.ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected end of ranges while spreading marks among threads");
|
||||
|
||||
MarkRange & range = part.ranges.front();
|
||||
MarkRange & range = part_with_ranges.ranges.front();
|
||||
|
||||
const size_t marks_in_range = range.end - range.begin;
|
||||
const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks);
|
||||
@ -392,11 +309,11 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
marks_in_part -= marks_to_get_from_range;
|
||||
need_marks -= marks_to_get_from_range;
|
||||
if (range.begin == range.end)
|
||||
part.ranges.pop_front();
|
||||
part_with_ranges.ranges.pop_front();
|
||||
}
|
||||
}
|
||||
|
||||
threads_tasks[i].parts_and_ranges.push_back({ part_idx, ranges_to_get_from_part });
|
||||
threads_tasks[i].parts_and_ranges.push_back({part_idx, ranges_to_get_from_part});
|
||||
threads_tasks[i].sum_marks_in_parts.push_back(marks_in_ranges);
|
||||
if (marks_in_ranges != 0)
|
||||
remaining_thread_tasks.insert(i);
|
||||
@ -415,152 +332,4 @@ void MergeTreeReadPool::fillPerThreadInfo(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadPoolParallelReplicas::~MergeTreeReadPoolParallelReplicas() = default;
|
||||
|
||||
|
||||
Block MergeTreeReadPoolParallelReplicas::getHeader() const
|
||||
{
|
||||
return storage_snapshot->getSampleBlockForColumns(extension.columns_to_read);
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread)
|
||||
{
|
||||
/// This parameter is needed only to satisfy the interface
|
||||
UNUSED(thread);
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (no_more_tasks_available)
|
||||
return nullptr;
|
||||
|
||||
if (buffered_ranges.empty())
|
||||
{
|
||||
auto result = extension.callback(ParallelReadRequest(
|
||||
CoordinationMode::Default,
|
||||
extension.number_of_current_replica,
|
||||
min_marks_for_concurrent_read * threads,
|
||||
/// For Default coordination mode we don't need to pass part names.
|
||||
RangesInDataPartsDescription{}));
|
||||
|
||||
if (!result || result->finish)
|
||||
{
|
||||
no_more_tasks_available = true;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
buffered_ranges = std::move(result->description);
|
||||
}
|
||||
|
||||
if (buffered_ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No tasks to read. This is a bug");
|
||||
|
||||
auto & current_task = buffered_ranges.front();
|
||||
|
||||
RangesInDataPart part;
|
||||
size_t part_idx = 0;
|
||||
for (size_t index = 0; index < per_part_params.size(); ++index)
|
||||
{
|
||||
auto & other_part = per_part_params[index];
|
||||
if (other_part.data_part.data_part->info == current_task.info)
|
||||
{
|
||||
part = other_part.data_part;
|
||||
part_idx = index;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
MarkRanges ranges_to_read;
|
||||
size_t current_sum_marks = 0;
|
||||
while (current_sum_marks < min_marks_for_concurrent_read && !current_task.ranges.empty())
|
||||
{
|
||||
auto diff = min_marks_for_concurrent_read - current_sum_marks;
|
||||
auto range = current_task.ranges.front();
|
||||
if (range.getNumberOfMarks() > diff)
|
||||
{
|
||||
auto new_range = range;
|
||||
new_range.end = range.begin + diff;
|
||||
range.begin += diff;
|
||||
|
||||
current_task.ranges.front() = range;
|
||||
ranges_to_read.push_back(new_range);
|
||||
current_sum_marks += new_range.getNumberOfMarks();
|
||||
continue;
|
||||
}
|
||||
|
||||
ranges_to_read.push_back(range);
|
||||
current_sum_marks += range.getNumberOfMarks();
|
||||
current_task.ranges.pop_front();
|
||||
}
|
||||
|
||||
if (current_task.ranges.empty())
|
||||
buffered_ranges.pop_front();
|
||||
|
||||
const auto & per_part = per_part_params[part_idx];
|
||||
|
||||
auto curr_task_size_predictor
|
||||
= !per_part.size_predictor ? nullptr : std::make_unique<MergeTreeBlockSizePredictor>(*per_part.size_predictor); /// make a copy
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
part.data_part,
|
||||
part.alter_conversions,
|
||||
ranges_to_read,
|
||||
part.part_index_in_query,
|
||||
per_part.column_name_set,
|
||||
per_part.task_columns,
|
||||
std::move(curr_task_size_predictor));
|
||||
}
|
||||
|
||||
|
||||
MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPartDescription description)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto get_from_buffer = [&]() -> std::optional<MarkRanges>
|
||||
{
|
||||
for (auto & desc : buffered_tasks)
|
||||
{
|
||||
if (desc.info == description.info && !desc.ranges.empty())
|
||||
{
|
||||
auto result = std::move(desc.ranges);
|
||||
desc.ranges = MarkRanges{};
|
||||
return result;
|
||||
}
|
||||
}
|
||||
return std::nullopt;
|
||||
};
|
||||
|
||||
if (auto result = get_from_buffer(); result)
|
||||
return result.value();
|
||||
|
||||
if (no_more_tasks)
|
||||
return {};
|
||||
|
||||
auto response = extension.callback(ParallelReadRequest(
|
||||
mode,
|
||||
extension.number_of_current_replica,
|
||||
min_marks_for_concurrent_read * request.size(),
|
||||
request
|
||||
));
|
||||
|
||||
if (!response || response->description.empty() || response->finish)
|
||||
{
|
||||
no_more_tasks = true;
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Fill the buffer
|
||||
for (size_t i = 0; i < request.size(); ++i)
|
||||
{
|
||||
auto & new_ranges = response->description[i].ranges;
|
||||
auto & old_ranges = buffered_tasks[i].ranges;
|
||||
std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges));
|
||||
}
|
||||
|
||||
if (auto result = get_from_buffer(); result)
|
||||
return result.value();
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,38 +1,30 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReadPool.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Provides read tasks for MergeTreeThreadSelectProcessor`s in fine-grained batches, allowing for more
|
||||
/** Provides read tasks for MergeTreeThreadSelectAlgorithm in fine-grained batches, allowing for more
|
||||
* uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads`
|
||||
* workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads
|
||||
* in "sequential" manner, requesting work in small batches. As soon as some thread has exhausted
|
||||
* it's workload, it either is signaled that no more work is available (`do_not_steal_tasks == false`) or
|
||||
* continues taking small batches from other threads' workloads (`do_not_steal_tasks == true`).
|
||||
*/
|
||||
class MergeTreeReadPool : public IMergeTreeReadPool
|
||||
class MergeTreeReadPool : public MergeTreeReadPoolBase
|
||||
{
|
||||
public:
|
||||
struct BackoffSettings;
|
||||
|
||||
MergeTreeReadPool(
|
||||
size_t threads_,
|
||||
size_t sum_marks_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
@ -40,12 +32,14 @@ public:
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
ContextPtr context_,
|
||||
bool do_not_steal_tasks_ = false);
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
~MergeTreeReadPool() override = default;
|
||||
|
||||
MergeTreeReadTaskPtr getTask(size_t thread) override;
|
||||
String getName() const override { return "ReadPool"; }
|
||||
bool preservesOrderOfRanges() const override { return false; }
|
||||
MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) override;
|
||||
|
||||
/** Each worker could call this method and pass information about read performance.
|
||||
* If read performance is too low, pool could decide to lower number of threads: do not assign more tasks to several threads.
|
||||
@ -53,8 +47,6 @@ public:
|
||||
*/
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) override;
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
/** Pull could dynamically lower (backoff) number of threads, if read operation are too slow.
|
||||
* Settings for that backoff.
|
||||
*/
|
||||
@ -82,50 +74,12 @@ public:
|
||||
BackoffSettings() : min_read_latency_ms(0) {}
|
||||
};
|
||||
|
||||
struct PerPartParams
|
||||
{
|
||||
MergeTreeReadTaskColumns task_columns;
|
||||
NameSet column_name_set;
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
RangesInDataPart data_part;
|
||||
};
|
||||
|
||||
static std::vector<size_t> fillPerPartInfo(
|
||||
const RangesInDataParts & parts,
|
||||
const StorageSnapshotPtr & storage_snapshot,
|
||||
std::vector<bool> & is_part_on_remote_disk,
|
||||
bool & predict_block_size_bytes,
|
||||
const Names & column_names,
|
||||
const Names & virtual_column_names,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
std::vector<MergeTreeReadPool::PerPartParams> & per_part_params);
|
||||
|
||||
private:
|
||||
void fillPerThreadInfo(
|
||||
size_t threads, size_t sum_marks, std::vector<size_t> per_part_sum_marks,
|
||||
const RangesInDataParts & parts);
|
||||
|
||||
/// Initialized in constructor
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
const Names column_names;
|
||||
const Names virtual_column_names;
|
||||
size_t min_marks_for_concurrent_read{0};
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
ExpressionActionsSettings actions_settings;
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
RangesInDataParts parts_ranges;
|
||||
bool predict_block_size_bytes;
|
||||
bool do_not_steal_tasks;
|
||||
bool merge_tree_use_const_size_tasks_for_remote_reading = false;
|
||||
|
||||
std::vector<PerPartParams> per_part_params;
|
||||
std::vector<bool> is_part_on_remote_disk;
|
||||
|
||||
BackoffSettings backoff_settings;
|
||||
void fillPerThreadInfo(size_t threads, size_t sum_marks);
|
||||
|
||||
mutable std::mutex mutex;
|
||||
size_t min_marks_for_concurrent_read = 0;
|
||||
|
||||
/// State to track numbers of slow reads.
|
||||
struct BackoffState
|
||||
{
|
||||
@ -135,16 +89,10 @@ private:
|
||||
|
||||
explicit BackoffState(size_t threads) : current_threads(threads) {}
|
||||
};
|
||||
|
||||
const BackoffSettings backoff_settings;
|
||||
BackoffState backoff_state;
|
||||
|
||||
struct Part
|
||||
{
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
size_t part_index_in_query;
|
||||
};
|
||||
|
||||
std::vector<Part> parts_with_idx;
|
||||
|
||||
struct ThreadTask
|
||||
{
|
||||
struct PartIndexAndRange
|
||||
@ -159,123 +107,8 @@ private:
|
||||
|
||||
std::vector<ThreadTask> threads_tasks;
|
||||
std::set<size_t> remaining_thread_tasks;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool");
|
||||
|
||||
};
|
||||
|
||||
class MergeTreeReadPoolParallelReplicas : public IMergeTreeReadPool
|
||||
{
|
||||
public:
|
||||
MergeTreeReadPoolParallelReplicas(
|
||||
StorageSnapshotPtr storage_snapshot_,
|
||||
size_t threads_,
|
||||
ParallelReadingExtension extension_,
|
||||
const RangesInDataParts & parts_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
size_t min_marks_for_concurrent_read_)
|
||||
: extension(extension_)
|
||||
, threads(threads_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, reader_settings(reader_settings_)
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, min_marks_for_concurrent_read(min_marks_for_concurrent_read_)
|
||||
, column_names(column_names_)
|
||||
, virtual_column_names(virtual_column_names_)
|
||||
, parts_ranges(std::move(parts_))
|
||||
{
|
||||
MergeTreeReadPool::fillPerPartInfo(
|
||||
parts_ranges, storage_snapshot, is_part_on_remote_disk,
|
||||
predict_block_size_bytes, column_names, virtual_column_names, prewhere_info,
|
||||
actions_settings, reader_settings, per_part_params);
|
||||
|
||||
extension.all_callback(InitialAllRangesAnnouncement(
|
||||
CoordinationMode::Default,
|
||||
parts_ranges.getDescriptions(),
|
||||
extension.number_of_current_replica
|
||||
));
|
||||
}
|
||||
|
||||
~MergeTreeReadPoolParallelReplicas() override;
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
MergeTreeReadTaskPtr getTask(size_t thread) override;
|
||||
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
|
||||
|
||||
private:
|
||||
ParallelReadingExtension extension;
|
||||
|
||||
RangesInDataPartsDescription buffered_ranges;
|
||||
size_t threads;
|
||||
bool no_more_tasks_available{false};
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas");
|
||||
|
||||
std::mutex mutex;
|
||||
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
ExpressionActionsSettings actions_settings;
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
size_t min_marks_for_concurrent_read;
|
||||
const Names column_names;
|
||||
const Names virtual_column_names;
|
||||
RangesInDataParts parts_ranges;
|
||||
|
||||
bool predict_block_size_bytes = false;
|
||||
std::vector<bool> is_part_on_remote_disk;
|
||||
std::vector<MergeTreeReadPool::PerPartParams> per_part_params;
|
||||
};
|
||||
|
||||
using MergeTreeReadPoolParallelReplicasPtr = std::shared_ptr<MergeTreeReadPoolParallelReplicas>;
|
||||
|
||||
|
||||
class MergeTreeInOrderReadPoolParallelReplicas : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
MergeTreeInOrderReadPoolParallelReplicas(
|
||||
RangesInDataParts parts_,
|
||||
ParallelReadingExtension extension_,
|
||||
CoordinationMode mode_,
|
||||
size_t min_marks_for_concurrent_read_)
|
||||
: parts_ranges(parts_)
|
||||
, extension(extension_)
|
||||
, mode(mode_)
|
||||
, min_marks_for_concurrent_read(min_marks_for_concurrent_read_)
|
||||
{
|
||||
for (const auto & part : parts_ranges)
|
||||
request.push_back({part.data_part->info, MarkRanges{}});
|
||||
|
||||
for (const auto & part : parts_ranges)
|
||||
buffered_tasks.push_back({part.data_part->info, MarkRanges{}});
|
||||
|
||||
extension.all_callback(InitialAllRangesAnnouncement(
|
||||
mode,
|
||||
parts_ranges.getDescriptions(),
|
||||
extension.number_of_current_replica
|
||||
));
|
||||
}
|
||||
|
||||
MarkRanges getNewTask(RangesInDataPartDescription description);
|
||||
|
||||
|
||||
RangesInDataParts parts_ranges;
|
||||
ParallelReadingExtension extension;
|
||||
CoordinationMode mode;
|
||||
size_t min_marks_for_concurrent_read{0};
|
||||
|
||||
bool no_more_tasks{false};
|
||||
RangesInDataPartsDescription request;
|
||||
RangesInDataPartsDescription buffered_tasks;
|
||||
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
using MergeTreeInOrderReadPoolParallelReplicasPtr = std::shared_ptr<MergeTreeInOrderReadPoolParallelReplicas>;
|
||||
|
||||
}
|
||||
|
149
src/Storages/MergeTree/MergeTreeReadPoolBase.cpp
Normal file
149
src/Storages/MergeTree/MergeTreeReadPoolBase.cpp
Normal file
@ -0,0 +1,149 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MergeTreeReadPoolBase::MergeTreeReadPoolBase(
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & pool_settings_,
|
||||
const ContextPtr & context_)
|
||||
: parts_ranges(std::move(parts_))
|
||||
, storage_snapshot(storage_snapshot_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, reader_settings(reader_settings_)
|
||||
, column_names(column_names_)
|
||||
, virtual_column_names(virtual_column_names_)
|
||||
, pool_settings(pool_settings_)
|
||||
, owned_mark_cache(context_->getGlobalContext()->getMarkCache())
|
||||
, owned_uncompressed_cache(pool_settings_.use_uncompressed_cache ? context_->getGlobalContext()->getUncompressedCache() : nullptr)
|
||||
, header(storage_snapshot->getSampleBlockForColumns(column_names))
|
||||
, profile_callback([this](ReadBufferFromFileBase::ProfileInfo info_) { profileFeedback(info_); })
|
||||
{
|
||||
fillPerPartInfos();
|
||||
}
|
||||
|
||||
void MergeTreeReadPoolBase::fillPerPartInfos()
|
||||
{
|
||||
per_part_infos.reserve(parts_ranges.size());
|
||||
is_part_on_remote_disk.reserve(parts_ranges.size());
|
||||
|
||||
auto sample_block = storage_snapshot->metadata->getSampleBlock();
|
||||
|
||||
for (const auto & part_with_ranges : parts_ranges)
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
assertSortedAndNonIntersecting(part_with_ranges.ranges);
|
||||
#endif
|
||||
|
||||
MergeTreeReadTask::Info read_task_info;
|
||||
|
||||
read_task_info.data_part = part_with_ranges.data_part;
|
||||
read_task_info.part_index_in_query = part_with_ranges.part_index_in_query;
|
||||
read_task_info.alter_conversions = part_with_ranges.alter_conversions;
|
||||
|
||||
LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions);
|
||||
|
||||
read_task_info.task_columns = getReadTaskColumns(
|
||||
part_info, storage_snapshot, column_names, virtual_column_names,
|
||||
prewhere_info, actions_settings,
|
||||
reader_settings, /*with_subcolumns=*/ true);
|
||||
|
||||
if (pool_settings.preferred_block_size_bytes > 0)
|
||||
{
|
||||
const auto & result_column_names = read_task_info.task_columns.columns.getNames();
|
||||
NameSet all_column_names(result_column_names.begin(), result_column_names.end());
|
||||
|
||||
for (const auto & pre_columns_per_step : read_task_info.task_columns.pre_columns)
|
||||
{
|
||||
const auto & pre_column_names = pre_columns_per_step.getNames();
|
||||
all_column_names.insert(pre_column_names.begin(), pre_column_names.end());
|
||||
}
|
||||
|
||||
read_task_info.shared_size_predictor = std::make_unique<MergeTreeBlockSizePredictor>(
|
||||
read_task_info.data_part,
|
||||
Names(all_column_names.begin(), all_column_names.end()),
|
||||
sample_block);
|
||||
}
|
||||
|
||||
is_part_on_remote_disk.push_back(part_with_ranges.data_part->isStoredOnRemoteDisk());
|
||||
per_part_infos.push_back(std::make_shared<MergeTreeReadTask::Info>(std::move(read_task_info)));
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<size_t> MergeTreeReadPoolBase::getPerPartSumMarks() const
|
||||
{
|
||||
std::vector<size_t> per_part_sum_marks;
|
||||
per_part_sum_marks.reserve(parts_ranges.size());
|
||||
|
||||
for (const auto & part_with_ranges : parts_ranges)
|
||||
{
|
||||
size_t sum_marks = 0;
|
||||
for (const auto & range : part_with_ranges.ranges)
|
||||
sum_marks += range.end - range.begin;
|
||||
|
||||
per_part_sum_marks.push_back(sum_marks);
|
||||
}
|
||||
|
||||
return per_part_sum_marks;
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask(
|
||||
MergeTreeReadTask::InfoPtr read_info,
|
||||
MarkRanges ranges,
|
||||
MergeTreeReadTask * previous_task) const
|
||||
{
|
||||
auto task_size_predictor = read_info->shared_size_predictor
|
||||
? std::make_unique<MergeTreeBlockSizePredictor>(*read_info->shared_size_predictor)
|
||||
: nullptr; /// make a copy
|
||||
|
||||
auto get_part_name = [](const auto & task_info) -> const String &
|
||||
{
|
||||
return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPart()->name : task_info.data_part->name;
|
||||
};
|
||||
|
||||
auto extras = getExtras();
|
||||
MergeTreeReadTask::Readers task_readers;
|
||||
|
||||
if (!previous_task)
|
||||
{
|
||||
task_readers = MergeTreeReadTask::createReaders(read_info, extras, ranges);
|
||||
}
|
||||
else if (get_part_name(previous_task->getInfo()) != get_part_name(*read_info))
|
||||
{
|
||||
extras.value_size_map = previous_task->getMainReader().getAvgValueSizeHints();
|
||||
task_readers = MergeTreeReadTask::createReaders(read_info, extras, ranges);
|
||||
}
|
||||
else
|
||||
{
|
||||
task_readers = previous_task->releaseReaders();
|
||||
}
|
||||
|
||||
return std::make_unique<MergeTreeReadTask>(
|
||||
read_info,
|
||||
std::move(task_readers),
|
||||
std::move(ranges),
|
||||
std::move(task_size_predictor));
|
||||
}
|
||||
|
||||
MergeTreeReadTask::Extras MergeTreeReadPoolBase::getExtras() const
|
||||
{
|
||||
return
|
||||
{
|
||||
.uncompressed_cache = owned_uncompressed_cache.get(),
|
||||
.mark_cache = owned_mark_cache.get(),
|
||||
.reader_settings = reader_settings,
|
||||
.storage_snapshot = storage_snapshot,
|
||||
.profile_callback = profile_callback,
|
||||
};
|
||||
}
|
||||
|
||||
}
|
67
src/Storages/MergeTree/MergeTreeReadPoolBase.h
Normal file
67
src/Storages/MergeTree/MergeTreeReadPoolBase.h
Normal file
@ -0,0 +1,67 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReadPool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeReadPoolBase : public IMergeTreeReadPool
|
||||
{
|
||||
public:
|
||||
struct PoolSettings
|
||||
{
|
||||
size_t threads = 0;
|
||||
size_t sum_marks = 0;
|
||||
size_t min_marks_for_concurrent_read = 0;
|
||||
size_t preferred_block_size_bytes = 0;
|
||||
|
||||
bool use_uncompressed_cache = false;
|
||||
bool do_not_steal_tasks = false;
|
||||
bool use_const_size_tasks_for_remote_reading = false;
|
||||
};
|
||||
|
||||
MergeTreeReadPoolBase(
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
Block getHeader() const override { return header; }
|
||||
|
||||
protected:
|
||||
/// Initialized in constructor
|
||||
const RangesInDataParts parts_ranges;
|
||||
const StorageSnapshotPtr storage_snapshot;
|
||||
const PrewhereInfoPtr prewhere_info;
|
||||
const ExpressionActionsSettings actions_settings;
|
||||
const MergeTreeReaderSettings reader_settings;
|
||||
const Names column_names;
|
||||
const Names virtual_column_names;
|
||||
const PoolSettings pool_settings;
|
||||
const MarkCachePtr owned_mark_cache;
|
||||
const UncompressedCachePtr owned_uncompressed_cache;
|
||||
const Block header;
|
||||
|
||||
void fillPerPartInfos();
|
||||
std::vector<size_t> getPerPartSumMarks() const;
|
||||
|
||||
MergeTreeReadTaskPtr createTask(
|
||||
MergeTreeReadTask::InfoPtr read_info,
|
||||
MarkRanges ranges,
|
||||
MergeTreeReadTask * previous_task) const;
|
||||
|
||||
MergeTreeReadTask::Extras getExtras() const;
|
||||
|
||||
std::vector<MergeTreeReadTask::InfoPtr> per_part_infos;
|
||||
std::vector<bool> is_part_on_remote_disk;
|
||||
|
||||
ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
};
|
||||
|
||||
}
|
73
src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp
Normal file
73
src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp
Normal file
@ -0,0 +1,73 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolInOrder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder(
|
||||
bool has_limit_below_one_block_,
|
||||
MergeTreeReadType read_type_,
|
||||
RangesInDataParts parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, has_limit_below_one_block(has_limit_below_one_block_)
|
||||
, read_type(read_type_)
|
||||
{
|
||||
per_part_mark_ranges.reserve(parts_ranges.size());
|
||||
for (const auto & part_with_ranges : parts_ranges)
|
||||
per_part_mark_ranges.push_back(part_with_ranges.ranges);
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolInOrder::getTask(size_t task_idx, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
if (task_idx >= per_part_infos.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Requested task with idx {}, but there are only {} parts",
|
||||
task_idx, per_part_infos.size());
|
||||
|
||||
auto & all_mark_ranges = per_part_mark_ranges[task_idx];
|
||||
if (all_mark_ranges.empty())
|
||||
return nullptr;
|
||||
|
||||
MarkRanges mark_ranges_for_task;
|
||||
if (read_type == MergeTreeReadType::InReverseOrder)
|
||||
{
|
||||
/// Read ranges from right to left.
|
||||
mark_ranges_for_task.emplace_back(std::move(all_mark_ranges.back()));
|
||||
all_mark_ranges.pop_back();
|
||||
}
|
||||
else if (has_limit_below_one_block)
|
||||
{
|
||||
/// If we need to read few rows, set one range per task to reduce number of read data.
|
||||
mark_ranges_for_task.emplace_back(std::move(all_mark_ranges.front()));
|
||||
all_mark_ranges.pop_front();
|
||||
}
|
||||
else
|
||||
{
|
||||
mark_ranges_for_task = std::move(all_mark_ranges);
|
||||
}
|
||||
|
||||
return createTask(per_part_infos[task_idx], std::move(mark_ranges_for_task), previous_task);
|
||||
}
|
||||
|
||||
}
|
35
src/Storages/MergeTree/MergeTreeReadPoolInOrder.h
Normal file
35
src/Storages/MergeTree/MergeTreeReadPoolInOrder.h
Normal file
@ -0,0 +1,35 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeReadPoolInOrder : public MergeTreeReadPoolBase
|
||||
{
|
||||
public:
|
||||
MergeTreeReadPoolInOrder(
|
||||
bool has_limit_below_one_block_,
|
||||
MergeTreeReadType read_type_,
|
||||
RangesInDataParts parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
String getName() const override { return "ReadPoolInOrder"; }
|
||||
bool preservesOrderOfRanges() const override { return true; }
|
||||
MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) override;
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
|
||||
|
||||
private:
|
||||
const bool has_limit_below_one_block;
|
||||
const MergeTreeReadType read_type;
|
||||
|
||||
std::vector<MarkRanges> per_part_mark_ranges;
|
||||
};
|
||||
|
||||
}
|
110
src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp
Normal file
110
src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp
Normal file
@ -0,0 +1,110 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
|
||||
ParallelReadingExtension extension_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, extension(std::move(extension_))
|
||||
{
|
||||
extension.all_callback(InitialAllRangesAnnouncement(
|
||||
CoordinationMode::Default,
|
||||
parts_ranges.getDescriptions(),
|
||||
extension.number_of_current_replica
|
||||
));
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_idx*/, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (no_more_tasks_available)
|
||||
return nullptr;
|
||||
|
||||
if (buffered_ranges.empty())
|
||||
{
|
||||
auto result = extension.callback(ParallelReadRequest(
|
||||
CoordinationMode::Default,
|
||||
extension.number_of_current_replica,
|
||||
pool_settings.min_marks_for_concurrent_read * pool_settings.threads,
|
||||
/// For Default coordination mode we don't need to pass part names.
|
||||
RangesInDataPartsDescription{}));
|
||||
|
||||
if (!result || result->finish)
|
||||
{
|
||||
no_more_tasks_available = true;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
buffered_ranges = std::move(result->description);
|
||||
}
|
||||
|
||||
if (buffered_ranges.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No tasks to read. This is a bug");
|
||||
|
||||
auto & current_task = buffered_ranges.front();
|
||||
|
||||
size_t part_idx = 0;
|
||||
for (size_t index = 0; index < per_part_infos.size(); ++index)
|
||||
{
|
||||
if (per_part_infos[index]->data_part->info == current_task.info)
|
||||
{
|
||||
part_idx = index;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
MarkRanges ranges_to_read;
|
||||
size_t current_sum_marks = 0;
|
||||
while (current_sum_marks < pool_settings.min_marks_for_concurrent_read && !current_task.ranges.empty())
|
||||
{
|
||||
auto diff = pool_settings.min_marks_for_concurrent_read - current_sum_marks;
|
||||
auto range = current_task.ranges.front();
|
||||
if (range.getNumberOfMarks() > diff)
|
||||
{
|
||||
auto new_range = range;
|
||||
new_range.end = range.begin + diff;
|
||||
range.begin += diff;
|
||||
|
||||
current_task.ranges.front() = range;
|
||||
ranges_to_read.push_back(new_range);
|
||||
current_sum_marks += new_range.getNumberOfMarks();
|
||||
continue;
|
||||
}
|
||||
|
||||
ranges_to_read.push_back(range);
|
||||
current_sum_marks += range.getNumberOfMarks();
|
||||
current_task.ranges.pop_front();
|
||||
}
|
||||
|
||||
if (current_task.ranges.empty())
|
||||
buffered_ranges.pop_front();
|
||||
|
||||
return createTask(per_part_infos[part_idx], std::move(ranges_to_read), previous_task);
|
||||
}
|
||||
|
||||
}
|
39
src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h
Normal file
39
src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeReadPoolParallelReplicas : public MergeTreeReadPoolBase
|
||||
{
|
||||
public:
|
||||
MergeTreeReadPoolParallelReplicas(
|
||||
ParallelReadingExtension extension_,
|
||||
RangesInDataParts && parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
~MergeTreeReadPoolParallelReplicas() override = default;
|
||||
|
||||
String getName() const override { return "ReadPoolParallelReplicas"; }
|
||||
bool preservesOrderOfRanges() const override { return false; }
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
|
||||
MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) override;
|
||||
|
||||
private:
|
||||
mutable std::mutex mutex;
|
||||
|
||||
const ParallelReadingExtension extension;
|
||||
RangesInDataPartsDescription buffered_ranges;
|
||||
bool no_more_tasks_available{false};
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas");
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,106 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrder(
|
||||
ParallelReadingExtension extension_,
|
||||
CoordinationMode mode_,
|
||||
RangesInDataParts parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_)
|
||||
: MergeTreeReadPoolBase(
|
||||
std::move(parts_),
|
||||
storage_snapshot_,
|
||||
prewhere_info_,
|
||||
actions_settings_,
|
||||
reader_settings_,
|
||||
column_names_,
|
||||
virtual_column_names_,
|
||||
settings_,
|
||||
context_)
|
||||
, extension(std::move(extension_))
|
||||
, mode(mode_)
|
||||
{
|
||||
for (const auto & part : parts_ranges)
|
||||
request.push_back({part.data_part->info, MarkRanges{}});
|
||||
|
||||
for (const auto & part : parts_ranges)
|
||||
buffered_tasks.push_back({part.data_part->info, MarkRanges{}});
|
||||
|
||||
extension.all_callback(InitialAllRangesAnnouncement(
|
||||
mode,
|
||||
parts_ranges.getDescriptions(),
|
||||
extension.number_of_current_replica
|
||||
));
|
||||
}
|
||||
|
||||
MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t task_idx, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (task_idx >= per_part_infos.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Requested task with idx {}, but there are only {} parts",
|
||||
task_idx, per_part_infos.size());
|
||||
|
||||
const auto & part_info = per_part_infos[task_idx]->data_part->info;
|
||||
auto get_from_buffer = [&]() -> std::optional<MarkRanges>
|
||||
{
|
||||
for (auto & desc : buffered_tasks)
|
||||
{
|
||||
if (desc.info == part_info && !desc.ranges.empty())
|
||||
{
|
||||
auto result = std::move(desc.ranges);
|
||||
desc.ranges = MarkRanges{};
|
||||
return result;
|
||||
}
|
||||
}
|
||||
return std::nullopt;
|
||||
};
|
||||
|
||||
if (auto result = get_from_buffer(); result)
|
||||
return createTask(per_part_infos[task_idx], std::move(*result), previous_task);
|
||||
|
||||
if (no_more_tasks)
|
||||
return nullptr;
|
||||
|
||||
auto response = extension.callback(ParallelReadRequest(
|
||||
mode,
|
||||
extension.number_of_current_replica,
|
||||
pool_settings.min_marks_for_concurrent_read * request.size(),
|
||||
request
|
||||
));
|
||||
|
||||
if (!response || response->description.empty() || response->finish)
|
||||
{
|
||||
no_more_tasks = true;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/// Fill the buffer
|
||||
for (size_t i = 0; i < request.size(); ++i)
|
||||
{
|
||||
auto & new_ranges = response->description[i].ranges;
|
||||
auto & old_ranges = buffered_tasks[i].ranges;
|
||||
std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges));
|
||||
}
|
||||
|
||||
if (auto result = get_from_buffer(); result)
|
||||
return createTask(per_part_infos[task_idx], std::move(*result), previous_task);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeReadPoolBase.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeReadPoolParallelReplicasInOrder : public MergeTreeReadPoolBase
|
||||
{
|
||||
public:
|
||||
MergeTreeReadPoolParallelReplicasInOrder(
|
||||
ParallelReadingExtension extension_,
|
||||
CoordinationMode mode_,
|
||||
RangesInDataParts parts_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & column_names_,
|
||||
const Names & virtual_column_names_,
|
||||
const PoolSettings & settings_,
|
||||
const ContextPtr & context_);
|
||||
|
||||
String getName() const override { return "ReadPoolParallelReplicasInOrder"; }
|
||||
bool preservesOrderOfRanges() const override { return true; }
|
||||
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
|
||||
MergeTreeReadTaskPtr getTask(size_t task_idx, MergeTreeReadTask * previous_task) override;
|
||||
|
||||
private:
|
||||
const ParallelReadingExtension extension;
|
||||
const CoordinationMode mode;
|
||||
|
||||
bool no_more_tasks{false};
|
||||
RangesInDataPartsDescription request;
|
||||
RangesInDataPartsDescription buffered_tasks;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
||||
};
|
193
src/Storages/MergeTree/MergeTreeReadTask.cpp
Normal file
193
src/Storages/MergeTree/MergeTreeReadTask.cpp
Normal file
@ -0,0 +1,193 @@
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
String MergeTreeReadTask::Columns::dump() const
|
||||
{
|
||||
WriteBufferFromOwnString s;
|
||||
for (size_t i = 0; i < pre_columns.size(); ++i)
|
||||
{
|
||||
s << "STEP " << i << ": " << pre_columns[i].toString() << "\n";
|
||||
}
|
||||
s << "COLUMNS: " << columns.toString() << "\n";
|
||||
return s.str();
|
||||
}
|
||||
|
||||
MergeTreeReadTask::MergeTreeReadTask(
|
||||
InfoPtr info_,
|
||||
Readers readers_,
|
||||
MarkRanges mark_ranges_,
|
||||
MergeTreeBlockSizePredictorPtr size_predictor_)
|
||||
: info(std::move(info_))
|
||||
, readers(std::move(readers_))
|
||||
, mark_ranges(std::move(mark_ranges_))
|
||||
, size_predictor(std::move(size_predictor_))
|
||||
{
|
||||
}
|
||||
|
||||
MergeTreeReadTask::Readers MergeTreeReadTask::createReaders(
|
||||
const InfoPtr & read_info, const Extras & extras, const MarkRanges & ranges)
|
||||
{
|
||||
Readers new_readers;
|
||||
|
||||
auto create_reader = [&](const NamesAndTypesList & columns_to_read)
|
||||
{
|
||||
return read_info->data_part->getReader(
|
||||
columns_to_read, extras.storage_snapshot, ranges,
|
||||
extras.uncompressed_cache, extras.mark_cache,
|
||||
read_info->alter_conversions, extras.reader_settings, extras.value_size_map, extras.profile_callback);
|
||||
};
|
||||
|
||||
new_readers.main = create_reader(read_info->task_columns.columns);
|
||||
|
||||
/// Add lightweight delete filtering step
|
||||
if (extras.reader_settings.apply_deleted_mask && read_info->data_part->hasLightweightDelete())
|
||||
new_readers.prewhere.push_back(create_reader({LightweightDeleteDescription::FILTER_COLUMN}));
|
||||
|
||||
for (const auto & pre_columns_per_step : read_info->task_columns.pre_columns)
|
||||
new_readers.prewhere.push_back(create_reader(pre_columns_per_step));
|
||||
|
||||
return new_readers;
|
||||
}
|
||||
|
||||
MergeTreeReadTask::RangeReaders MergeTreeReadTask::createRangeReaders(
|
||||
const Readers & task_readers,
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
const Names & non_const_virtual_column_names)
|
||||
{
|
||||
MergeTreeReadTask::RangeReaders new_range_readers;
|
||||
if (prewhere_actions.steps.size() != task_readers.prewhere.size())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"PREWHERE steps count mismatch, actions: {}, readers: {}",
|
||||
prewhere_actions.steps.size(), task_readers.prewhere.size());
|
||||
|
||||
MergeTreeRangeReader * prev_reader = nullptr;
|
||||
bool last_reader = false;
|
||||
|
||||
for (size_t i = 0; i < prewhere_actions.steps.size(); ++i)
|
||||
{
|
||||
last_reader = task_readers.main->getColumns().empty() && (i + 1 == prewhere_actions.steps.size());
|
||||
|
||||
MergeTreeRangeReader current_reader(
|
||||
task_readers.prewhere[i].get(),
|
||||
prev_reader, prewhere_actions.steps[i].get(),
|
||||
last_reader, non_const_virtual_column_names);
|
||||
|
||||
new_range_readers.prewhere.push_back(std::move(current_reader));
|
||||
prev_reader = &new_range_readers.prewhere.back();
|
||||
}
|
||||
|
||||
if (!last_reader)
|
||||
{
|
||||
new_range_readers.main = MergeTreeRangeReader(task_readers.main.get(), prev_reader, nullptr, true, non_const_virtual_column_names);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If all columns are read by prewhere range readers than move last prewhere range reader to main.
|
||||
new_range_readers.main = std::move(new_range_readers.prewhere.back());
|
||||
new_range_readers.prewhere.pop_back();
|
||||
}
|
||||
|
||||
return new_range_readers;
|
||||
}
|
||||
|
||||
void MergeTreeReadTask::initializeRangeReaders(
|
||||
const PrewhereExprInfo & prewhere_actions,
|
||||
const Names & non_const_virtual_column_names)
|
||||
{
|
||||
if (range_readers.main.isInitialized())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Range reader is already initialized");
|
||||
|
||||
range_readers = createRangeReaders(readers, prewhere_actions, non_const_virtual_column_names);
|
||||
}
|
||||
|
||||
UInt64 MergeTreeReadTask::estimateNumRows(const BlockSizeParams & params) const
|
||||
{
|
||||
if (!size_predictor)
|
||||
return static_cast<size_t>(params.max_block_size_rows);
|
||||
|
||||
/// Calculates number of rows will be read using preferred_block_size_bytes.
|
||||
/// Can't be less than avg_index_granularity.
|
||||
size_t rows_to_read = size_predictor->estimateNumRows(params.preferred_block_size_bytes);
|
||||
if (!rows_to_read)
|
||||
return rows_to_read;
|
||||
|
||||
auto total_row_in_current_granule = range_readers.main.numRowsInCurrentGranule();
|
||||
rows_to_read = std::max(total_row_in_current_granule, rows_to_read);
|
||||
|
||||
if (params.preferred_max_column_in_block_size_bytes)
|
||||
{
|
||||
/// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes.
|
||||
auto rows_to_read_for_max_size_column = size_predictor->estimateNumRowsForMaxSizeColumn(params.preferred_max_column_in_block_size_bytes);
|
||||
|
||||
double filtration_ratio = std::max(params.min_filtration_ratio, 1.0 - size_predictor->filtered_rows_ratio);
|
||||
auto rows_to_read_for_max_size_column_with_filtration
|
||||
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
|
||||
|
||||
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than current_index_granularity.
|
||||
rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration);
|
||||
}
|
||||
|
||||
auto unread_rows_in_current_granule = range_readers.main.numPendingRowsInCurrentGranule();
|
||||
if (unread_rows_in_current_granule >= rows_to_read)
|
||||
return rows_to_read;
|
||||
|
||||
const auto & index_granularity = info->data_part->index_granularity;
|
||||
return index_granularity.countMarksForRows(range_readers.main.currentMark(), rows_to_read, range_readers.main.numReadRowsInCurrentGranule(), params.min_marks_to_read);
|
||||
}
|
||||
|
||||
MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParams & params)
|
||||
{
|
||||
if (size_predictor)
|
||||
size_predictor->startBlock();
|
||||
|
||||
UInt64 recommended_rows = estimateNumRows(params);
|
||||
UInt64 rows_to_read = std::max(static_cast<UInt64>(1), std::min(params.max_block_size_rows, recommended_rows));
|
||||
|
||||
auto read_result = range_readers.main.read(rows_to_read, mark_ranges);
|
||||
|
||||
/// All rows were filtered. Repeat.
|
||||
if (read_result.num_rows == 0)
|
||||
read_result.columns.clear();
|
||||
|
||||
const auto & sample_block = range_readers.main.getSampleBlock();
|
||||
if (read_result.num_rows != 0 && sample_block.columns() != read_result.columns.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent number of columns got from MergeTreeRangeReader. "
|
||||
"Have {} in sample block and {} columns in list",
|
||||
toString(sample_block.columns()), toString(read_result.columns.size()));
|
||||
|
||||
/// TODO: check columns have the same types as in header.
|
||||
UInt64 num_filtered_rows = read_result.numReadRows() - read_result.num_rows;
|
||||
|
||||
size_t num_read_rows = read_result.numReadRows();
|
||||
size_t num_read_bytes = read_result.numBytesRead();
|
||||
|
||||
if (size_predictor)
|
||||
{
|
||||
size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows);
|
||||
if (!read_result.columns.empty())
|
||||
size_predictor->update(sample_block, read_result.columns, read_result.num_rows);
|
||||
}
|
||||
|
||||
Block block;
|
||||
if (read_result.num_rows != 0)
|
||||
block = sample_block.cloneWithColumns(read_result.columns);
|
||||
|
||||
BlockAndProgress res = {
|
||||
.block = std::move(block),
|
||||
.row_count = read_result.num_rows,
|
||||
.num_read_rows = num_read_rows,
|
||||
.num_read_bytes = num_read_bytes };
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
156
src/Storages/MergeTree/MergeTreeReadTask.h
Normal file
156
src/Storages/MergeTree/MergeTreeReadTask.h
Normal file
@ -0,0 +1,156 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/core/noncopyable.hpp>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/AlterConversions.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class UncompressedCache;
|
||||
class MarkCache;
|
||||
|
||||
struct MergeTreeBlockSizePredictor;
|
||||
using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredictor>;
|
||||
|
||||
class IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
||||
using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
|
||||
|
||||
enum class MergeTreeReadType
|
||||
{
|
||||
/// By default, read will use MergeTreeReadPool and return pipe with num_streams outputs.
|
||||
/// If num_streams == 1, will read without pool, in order specified in parts.
|
||||
Default,
|
||||
/// Read in sorting key order.
|
||||
/// Returned pipe will have the number of ports equals to parts.size().
|
||||
/// Parameter num_streams_ is ignored in this case.
|
||||
/// User should add MergingSorted itself if needed.
|
||||
InOrder,
|
||||
/// The same as InOrder, but in reverse order.
|
||||
/// For every part, read ranges and granules from end to begin. Also add ReverseTransform.
|
||||
InReverseOrder,
|
||||
/// A special type of reading where every replica
|
||||
/// talks to a remote coordinator (which is located on the initiator node)
|
||||
/// and who spreads marks and parts across them.
|
||||
ParallelReplicas,
|
||||
};
|
||||
|
||||
/// A batch of work for MergeTreeSelectProcessor
|
||||
struct MergeTreeReadTask : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
struct Columns
|
||||
{
|
||||
/// Column names to read during WHERE
|
||||
NamesAndTypesList columns;
|
||||
/// Column names to read during each PREWHERE step
|
||||
std::vector<NamesAndTypesList> pre_columns;
|
||||
|
||||
String dump() const;
|
||||
};
|
||||
|
||||
struct Info
|
||||
{
|
||||
/// Data part which should be read while performing this task
|
||||
DataPartPtr data_part;
|
||||
/// For virtual `part_index` virtual column
|
||||
size_t part_index_in_query;
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
/// Column names to read during PREWHERE and WHERE
|
||||
Columns task_columns;
|
||||
/// Shared initialized size predictor. It is copied for each new task.
|
||||
MergeTreeBlockSizePredictorPtr shared_size_predictor;
|
||||
};
|
||||
|
||||
using InfoPtr = std::shared_ptr<const Info>;
|
||||
|
||||
/// Extra params that required for creation of reader.
|
||||
struct Extras
|
||||
{
|
||||
UncompressedCache * uncompressed_cache = nullptr;
|
||||
MarkCache * mark_cache = nullptr;
|
||||
MergeTreeReaderSettings reader_settings;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
IMergeTreeReader::ValueSizeMap value_size_map;
|
||||
ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
};
|
||||
|
||||
struct Readers
|
||||
{
|
||||
MergeTreeReaderPtr main;
|
||||
std::vector<MergeTreeReaderPtr> prewhere;
|
||||
};
|
||||
|
||||
struct RangeReaders
|
||||
{
|
||||
/// Used to save current range processing status
|
||||
MergeTreeRangeReader main;
|
||||
|
||||
/// Range readers for multiple filtering steps: row level security, PREWHERE etc.
|
||||
/// NOTE: we take references to elements and push_back new elements, that's why it is a deque but not a vector
|
||||
std::deque<MergeTreeRangeReader> prewhere;
|
||||
};
|
||||
|
||||
struct BlockSizeParams
|
||||
{
|
||||
UInt64 max_block_size_rows = DEFAULT_BLOCK_SIZE;
|
||||
UInt64 preferred_block_size_bytes = 1000000;
|
||||
UInt64 preferred_max_column_in_block_size_bytes = 0;
|
||||
UInt64 min_marks_to_read = 0;
|
||||
double min_filtration_ratio = 0.00001;
|
||||
};
|
||||
|
||||
/// The result of reading from task.
|
||||
struct BlockAndProgress
|
||||
{
|
||||
Block block;
|
||||
size_t row_count = 0;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
};
|
||||
|
||||
MergeTreeReadTask(InfoPtr info_, Readers readers_, MarkRanges mark_ranges_, MergeTreeBlockSizePredictorPtr size_predictor_);
|
||||
|
||||
void initializeRangeReaders(const PrewhereExprInfo & prewhere_actions, const Names & non_const_virtual_column_names);
|
||||
|
||||
BlockAndProgress read(const BlockSizeParams & params);
|
||||
bool isFinished() const { return mark_ranges.empty() && range_readers.main.isCurrentRangeFinished(); }
|
||||
|
||||
const Info & getInfo() const { return *info; }
|
||||
const MergeTreeRangeReader & getMainRangeReader() const { return range_readers.main; }
|
||||
const IMergeTreeReader & getMainReader() const { return *readers.main; }
|
||||
|
||||
Readers releaseReaders() { return std::move(readers); }
|
||||
|
||||
static Readers createReaders(const InfoPtr & read_info, const Extras & extras, const MarkRanges & ranges);
|
||||
static RangeReaders createRangeReaders(const Readers & readers, const PrewhereExprInfo & prewhere_actions, const Names & non_const_virtual_column_names);
|
||||
|
||||
private:
|
||||
UInt64 estimateNumRows(const BlockSizeParams & params) const;
|
||||
|
||||
/// Shared information required for reading.
|
||||
InfoPtr info;
|
||||
|
||||
/// Readers for data_part of this task.
|
||||
/// May be reused and released to the next task.
|
||||
Readers readers;
|
||||
|
||||
/// Range readers to read mark_ranges from data_part
|
||||
RangeReaders range_readers;
|
||||
|
||||
/// Ranges to read from data_part
|
||||
MarkRanges mark_ranges;
|
||||
|
||||
/// Used to satistfy preferred_block_size_bytes limitation
|
||||
MergeTreeBlockSizePredictorPtr size_predictor;
|
||||
};
|
||||
|
||||
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
||||
|
||||
}
|
@ -1,134 +0,0 @@
|
||||
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/IntersectionsIndexes.h>
|
||||
#include "Storages/MergeTree/MergeTreeBaseSelectProcessor.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
bool MergeTreeReverseSelectAlgorithm::getNewTaskImpl()
|
||||
try
|
||||
{
|
||||
if (pool)
|
||||
return getNewTaskParallelReplicas();
|
||||
else
|
||||
return getNewTaskOrdinaryReading();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Suspicion of the broken part. A part is added to the queue for verification.
|
||||
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
storage.reportBrokenPart(data_part);
|
||||
throw;
|
||||
}
|
||||
|
||||
|
||||
bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading()
|
||||
{
|
||||
if (chunks.empty() && all_mark_ranges.empty())
|
||||
return false;
|
||||
|
||||
/// We have some blocks to return in buffer.
|
||||
/// Return true to continue reading, but actually don't create a task.
|
||||
if (all_mark_ranges.empty())
|
||||
return true;
|
||||
|
||||
if (!reader)
|
||||
initializeReaders();
|
||||
|
||||
/// Read ranges from right to left.
|
||||
MarkRanges mark_ranges_for_task{std::move(all_mark_ranges.back())};
|
||||
all_mark_ranges.pop_back();
|
||||
|
||||
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
|
||||
}
|
||||
|
||||
bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas()
|
||||
{
|
||||
if (chunks.empty() && no_more_tasks)
|
||||
return false;
|
||||
|
||||
/// We have some blocks to return in buffer.
|
||||
/// Return true to continue reading, but actually don't create a task.
|
||||
if (no_more_tasks)
|
||||
return true;
|
||||
|
||||
if (!reader)
|
||||
initializeReaders();
|
||||
|
||||
auto description = RangesInDataPartDescription{
|
||||
.info = data_part->info,
|
||||
/// We just ignore all the distribution done before
|
||||
/// Everything will be done on coordinator side
|
||||
.ranges = {},
|
||||
};
|
||||
|
||||
auto mark_ranges_for_task = pool->getNewTask(description);
|
||||
if (mark_ranges_for_task.empty())
|
||||
{
|
||||
/// If we have chunks in buffer - return true to continue reading from them
|
||||
return !chunks.empty();
|
||||
}
|
||||
|
||||
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
|
||||
: getSizePredictor(data_part, task_columns, sample_block);
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part,
|
||||
alter_conversions,
|
||||
mark_ranges_for_task,
|
||||
part_index_in_query,
|
||||
column_name_set,
|
||||
task_columns,
|
||||
std::move(size_predictor));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
MergeTreeReverseSelectAlgorithm::BlockAndProgress MergeTreeReverseSelectAlgorithm::readFromPart()
|
||||
{
|
||||
BlockAndProgress res;
|
||||
|
||||
if (!chunks.empty())
|
||||
{
|
||||
res = std::move(chunks.back());
|
||||
chunks.pop_back();
|
||||
return res;
|
||||
}
|
||||
|
||||
if (!task->range_reader.isInitialized())
|
||||
initializeRangeReaders(*task);
|
||||
|
||||
while (!task->isFinished())
|
||||
{
|
||||
auto chunk = readFromPartImpl();
|
||||
chunks.push_back(std::move(chunk));
|
||||
}
|
||||
|
||||
if (chunks.empty())
|
||||
return {};
|
||||
|
||||
res = std::move(chunks.back());
|
||||
chunks.pop_back();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
@ -1,44 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Used to read data from single part with select query
|
||||
/// in reverse order of primary key.
|
||||
/// Cares about PREWHERE, virtual columns, indexes etc.
|
||||
/// To read data from multiple parts, Storage (MergeTree) creates multiple such objects.
|
||||
class MergeTreeReverseSelectAlgorithm final : public MergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReverseSelectAlgorithm(Args &&... args)
|
||||
: MergeTreeSelectAlgorithm{std::forward<Args>(args)...}
|
||||
{
|
||||
LOG_TRACE(log, "Reading {} ranges in reverse order from part {}, approx. {} rows starting from {}",
|
||||
all_mark_ranges.size(), data_part->name, total_rows,
|
||||
data_part->index_granularity.getMarkStartingRow(all_mark_ranges.front().begin));
|
||||
}
|
||||
|
||||
String getName() const override { return "MergeTreeReverse"; }
|
||||
|
||||
private:
|
||||
bool getNewTaskImpl() override;
|
||||
void finalizeNewTask() override {}
|
||||
|
||||
bool getNewTaskParallelReplicas();
|
||||
bool getNewTaskOrdinaryReading();
|
||||
|
||||
BlockAndProgress readFromPart() override;
|
||||
|
||||
std::vector<BlockAndProgress> chunks;
|
||||
|
||||
/// Used for parallel replicas
|
||||
bool no_more_tasks{false};
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor");
|
||||
};
|
||||
|
||||
}
|
87
src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp
Normal file
87
src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp
Normal file
@ -0,0 +1,87 @@
|
||||
#include <Storages/MergeTree/MergeTreeSelectAlgorithms.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReadPool.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeThreadSelectAlgorithm::TaskResult MergeTreeThreadSelectAlgorithm::getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
TaskResult res;
|
||||
res.first = pool.getTask(thread_idx, previous_task);
|
||||
res.second = !!res.first;
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeReadTask::BlockAndProgress MergeTreeThreadSelectAlgorithm::readFromTask(MergeTreeReadTask * task, const MergeTreeReadTask::BlockSizeParams & params)
|
||||
{
|
||||
if (!task)
|
||||
return {};
|
||||
|
||||
return task->read(params);
|
||||
}
|
||||
|
||||
IMergeTreeSelectAlgorithm::TaskResult MergeTreeInOrderSelectAlgorithm::getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
if (!pool.preservesOrderOfRanges())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"MergeTreeInOrderSelectAlgorithm requires read pool that preserves order of ranges, got: {}", pool.getName());
|
||||
|
||||
TaskResult res;
|
||||
res.first = pool.getTask(part_idx, previous_task);
|
||||
res.second = !!res.first;
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeReadTask::BlockAndProgress MergeTreeInOrderSelectAlgorithm::readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params)
|
||||
{
|
||||
if (!task)
|
||||
return {};
|
||||
|
||||
return task->read(params);
|
||||
}
|
||||
|
||||
IMergeTreeSelectAlgorithm::TaskResult MergeTreeInReverseOrderSelectAlgorithm::getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task)
|
||||
{
|
||||
if (!pool.preservesOrderOfRanges())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"MergeTreeInReverseOrderSelectAlgorithm requires read pool that preserves order of ranges, got: {}", pool.getName());
|
||||
|
||||
TaskResult res;
|
||||
res.first = pool.getTask(part_idx, previous_task);
|
||||
/// We may have some chunks to return in buffer.
|
||||
/// Set continue_reading to true but actually don't create a new task.
|
||||
res.second = !!res.first || !chunks.empty();
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeReadTask::BlockAndProgress MergeTreeInReverseOrderSelectAlgorithm::readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params)
|
||||
{
|
||||
MergeTreeReadTask::BlockAndProgress res;
|
||||
|
||||
if (!chunks.empty())
|
||||
{
|
||||
res = std::move(chunks.back());
|
||||
chunks.pop_back();
|
||||
return res;
|
||||
}
|
||||
|
||||
if (!task)
|
||||
return {};
|
||||
|
||||
while (!task->isFinished())
|
||||
chunks.push_back(task->read(params));
|
||||
|
||||
if (chunks.empty())
|
||||
return {};
|
||||
|
||||
res = std::move(chunks.back());
|
||||
chunks.pop_back();
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
64
src/Storages/MergeTree/MergeTreeSelectAlgorithms.h
Normal file
64
src/Storages/MergeTree/MergeTreeSelectAlgorithms.h
Normal file
@ -0,0 +1,64 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
#include <boost/core/noncopyable.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeReadPool;
|
||||
|
||||
class IMergeTreeSelectAlgorithm : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/// The pair of {task, continue_reading}.
|
||||
using TaskResult = std::pair<MergeTreeReadTaskPtr, bool>;
|
||||
using BlockSizeParams = MergeTreeReadTask::BlockSizeParams;
|
||||
|
||||
virtual ~IMergeTreeSelectAlgorithm() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
virtual TaskResult getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) = 0;
|
||||
virtual MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params) = 0;
|
||||
};
|
||||
|
||||
using MergeTreeSelectAlgorithmPtr = std::unique_ptr<IMergeTreeSelectAlgorithm>;
|
||||
|
||||
class MergeTreeThreadSelectAlgorithm : public IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeThreadSelectAlgorithm(size_t thread_idx_) : thread_idx(thread_idx_) {}
|
||||
String getName() const override { return "Thread"; }
|
||||
TaskResult getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override;
|
||||
MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params) override;
|
||||
|
||||
private:
|
||||
const size_t thread_idx;
|
||||
};
|
||||
|
||||
class MergeTreeInOrderSelectAlgorithm : public IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeInOrderSelectAlgorithm(size_t part_idx_) : part_idx(part_idx_) {}
|
||||
String getName() const override { return "InOrder"; }
|
||||
TaskResult getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override;
|
||||
MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params) override;
|
||||
|
||||
private:
|
||||
const size_t part_idx;
|
||||
};
|
||||
|
||||
class MergeTreeInReverseOrderSelectAlgorithm : public IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeInReverseOrderSelectAlgorithm(size_t part_idx_) : part_idx(part_idx_) {}
|
||||
String getName() const override { return "InReverseOrder"; }
|
||||
TaskResult getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override;
|
||||
MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask * task, const BlockSizeParams & params) override;
|
||||
|
||||
private:
|
||||
const size_t part_idx;
|
||||
std::vector<MergeTreeReadTask::BlockAndProgress> chunks;
|
||||
};
|
||||
|
||||
}
|
@ -1,85 +1,435 @@
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
#include <Common/ElapsedTimeProfileEventIncrement.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
#include <city.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm(
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int QUERY_WAS_CANCELLED;
|
||||
}
|
||||
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns);
|
||||
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns);
|
||||
|
||||
MergeTreeSelectProcessor::MergeTreeSelectProcessor(
|
||||
MergeTreeReadPoolPtr pool_,
|
||||
MergeTreeSelectAlgorithmPtr algorithm_,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
UInt64 max_block_size_rows_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
size_t preferred_max_column_in_block_size_bytes_,
|
||||
Names required_columns_,
|
||||
MarkRanges mark_ranges_,
|
||||
bool use_uncompressed_cache_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReadTask::BlockSizeParams & block_size_params_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool_,
|
||||
const Names & virt_column_names_,
|
||||
size_t part_index_in_query_,
|
||||
bool has_limit_below_one_block_)
|
||||
: IMergeTreeSelectAlgorithm{
|
||||
storage_snapshot_->getSampleBlockForColumns(required_columns_),
|
||||
storage_, storage_snapshot_, prewhere_info_, actions_settings_, max_block_size_rows_,
|
||||
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
|
||||
reader_settings_, use_uncompressed_cache_, virt_column_names_},
|
||||
required_columns{std::move(required_columns_)},
|
||||
data_part{owned_data_part_},
|
||||
alter_conversions(alter_conversions_),
|
||||
sample_block(storage_snapshot_->metadata->getSampleBlock()),
|
||||
all_mark_ranges(std::move(mark_ranges_)),
|
||||
part_index_in_query(part_index_in_query_),
|
||||
has_limit_below_one_block(has_limit_below_one_block_),
|
||||
pool(pool_),
|
||||
total_rows(data_part->index_granularity.getRowsCountInRanges(all_mark_ranges))
|
||||
const Names & virt_column_names_)
|
||||
: pool(std::move(pool_))
|
||||
, algorithm(std::move(algorithm_))
|
||||
, prewhere_info(prewhere_info_)
|
||||
, actions_settings(actions_settings_)
|
||||
, prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps))
|
||||
, reader_settings(reader_settings_)
|
||||
, block_size_params(block_size_params_)
|
||||
, virt_column_names(virt_column_names_)
|
||||
, partition_value_type(storage_.getPartitionValueType())
|
||||
{
|
||||
ordered_names = header_without_const_virtual_columns.getNames();
|
||||
if (reader_settings.apply_deleted_mask)
|
||||
{
|
||||
PrewhereExprStep step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = nullptr,
|
||||
.filter_column_name = LightweightDeleteDescription::FILTER_COLUMN.name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
lightweight_delete_filter_step = std::make_shared<PrewhereExprStep>(std::move(step));
|
||||
}
|
||||
|
||||
header_without_const_virtual_columns = applyPrewhereActions(pool->getHeader(), prewhere_info);
|
||||
size_t non_const_columns_offset = header_without_const_virtual_columns.columns();
|
||||
injectNonConstVirtualColumns(0, header_without_const_virtual_columns, virt_column_names);
|
||||
|
||||
for (size_t col_num = non_const_columns_offset; col_num < header_without_const_virtual_columns.columns(); ++col_num)
|
||||
non_const_virtual_column_names.emplace_back(header_without_const_virtual_columns.getByPosition(col_num).name);
|
||||
|
||||
result_header = header_without_const_virtual_columns;
|
||||
injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names);
|
||||
|
||||
if (!prewhere_actions.steps.empty())
|
||||
LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions.steps.size(), prewhere_actions.dumpConditions());
|
||||
|
||||
if (prewhere_info)
|
||||
LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}",
|
||||
(prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("<nullptr>")),
|
||||
(!prewhere_actions.steps.empty() ? prewhere_actions.dump() : std::string("<nullptr>")));
|
||||
}
|
||||
|
||||
void MergeTreeSelectAlgorithm::initializeReaders()
|
||||
String MergeTreeSelectProcessor::getName() const
|
||||
{
|
||||
LoadedMergeTreeDataPartInfoForReader part_info(data_part, alter_conversions);
|
||||
|
||||
task_columns = getReadTaskColumns(
|
||||
part_info, storage_snapshot,
|
||||
required_columns, virt_column_names,
|
||||
prewhere_info,
|
||||
actions_settings, reader_settings, /*with_subcolumns=*/ true);
|
||||
|
||||
/// Will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const auto & column_names = task_columns.columns.getNames();
|
||||
column_name_set = NameSet{column_names.begin(), column_names.end()};
|
||||
|
||||
if (use_uncompressed_cache)
|
||||
owned_uncompressed_cache = storage.getContext()->getUncompressedCache();
|
||||
|
||||
owned_mark_cache = storage.getContext()->getMarkCache();
|
||||
|
||||
initializeMergeTreeReadersForPart(
|
||||
data_part, alter_conversions, task_columns,
|
||||
all_mark_ranges, /*value_size_map=*/ {}, /*profile_callback=*/ {});
|
||||
return fmt::format("MergeTreeSelect(pool: {}, algorithm: {})", pool->getName(), algorithm->getName());
|
||||
}
|
||||
|
||||
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere);
|
||||
|
||||
void MergeTreeSelectAlgorithm::finish()
|
||||
PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
|
||||
{
|
||||
/** 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_for_step.clear();
|
||||
data_part.reset();
|
||||
PrewhereExprInfo prewhere_actions;
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
PrewhereExprStep row_level_filter_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter, actions_settings),
|
||||
.filter_column_name = prewhere_info->row_level_column_name,
|
||||
.remove_filter_column = true,
|
||||
.need_filter = true,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(row_level_filter_step)));
|
||||
}
|
||||
|
||||
if (!enable_multiple_prewhere_read_steps ||
|
||||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions))
|
||||
{
|
||||
PrewhereExprStep prewhere_step
|
||||
{
|
||||
.type = PrewhereExprStep::Filter,
|
||||
.actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings),
|
||||
.filter_column_name = prewhere_info->prewhere_column_name,
|
||||
.remove_filter_column = prewhere_info->remove_prewhere_column,
|
||||
.need_filter = prewhere_info->need_filter,
|
||||
.perform_alter_conversions = true,
|
||||
};
|
||||
|
||||
prewhere_actions.steps.emplace_back(std::make_shared<PrewhereExprStep>(std::move(prewhere_step)));
|
||||
}
|
||||
}
|
||||
|
||||
return prewhere_actions;
|
||||
}
|
||||
|
||||
MergeTreeSelectAlgorithm::~MergeTreeSelectAlgorithm() = default;
|
||||
ChunkAndProgress MergeTreeSelectProcessor::read()
|
||||
{
|
||||
while (!is_cancelled)
|
||||
{
|
||||
try
|
||||
{
|
||||
bool continue_reading = true;
|
||||
if (!task || task->isFinished())
|
||||
std::tie(task, continue_reading) = algorithm->getNewTask(*pool, task.get());
|
||||
|
||||
if (!continue_reading)
|
||||
break;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::QUERY_WAS_CANCELLED)
|
||||
break;
|
||||
throw;
|
||||
}
|
||||
|
||||
if (task && !task->getMainRangeReader().isInitialized())
|
||||
initializeRangeReaders();
|
||||
|
||||
auto res = algorithm->readFromTask(task.get(), block_size_params);
|
||||
|
||||
if (res.row_count)
|
||||
{
|
||||
injectVirtualColumns(res.block, res.row_count, task.get(), partition_value_type, virt_column_names);
|
||||
|
||||
/// Reorder the columns according to result_header
|
||||
Columns ordered_columns;
|
||||
ordered_columns.reserve(result_header.columns());
|
||||
for (size_t i = 0; i < result_header.columns(); ++i)
|
||||
{
|
||||
auto name = result_header.getByPosition(i).name;
|
||||
ordered_columns.push_back(res.block.getByName(name).column);
|
||||
}
|
||||
|
||||
return ChunkAndProgress{
|
||||
.chunk = Chunk(ordered_columns, res.row_count),
|
||||
.num_read_rows = res.num_read_rows,
|
||||
.num_read_bytes = res.num_read_bytes,
|
||||
.is_finished = false};
|
||||
}
|
||||
else
|
||||
{
|
||||
return {Chunk(), res.num_read_rows, res.num_read_bytes, false};
|
||||
}
|
||||
}
|
||||
|
||||
return {Chunk(), 0, 0, true};
|
||||
}
|
||||
|
||||
void MergeTreeSelectProcessor::initializeRangeReaders()
|
||||
{
|
||||
PrewhereExprInfo all_prewhere_actions;
|
||||
if (lightweight_delete_filter_step && task->getInfo().data_part->hasLightweightDelete())
|
||||
all_prewhere_actions.steps.push_back(lightweight_delete_filter_step);
|
||||
|
||||
for (const auto & step : prewhere_actions.steps)
|
||||
all_prewhere_actions.steps.push_back(step);
|
||||
|
||||
task->initializeRangeReaders(all_prewhere_actions, non_const_virtual_column_names);
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
struct VirtualColumnsInserter
|
||||
{
|
||||
explicit VirtualColumnsInserter(Block & block_) : block(block_) {}
|
||||
|
||||
bool columnExists(const String & name) const { return block.has(name); }
|
||||
|
||||
void insertUInt8Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt8>(), name});
|
||||
}
|
||||
|
||||
void insertUInt64Column(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUInt64>(), name});
|
||||
}
|
||||
|
||||
void insertUUIDColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeUUID>(), name});
|
||||
}
|
||||
|
||||
void insertLowCardinalityColumn(const ColumnPtr & column, const String & name)
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), name});
|
||||
}
|
||||
|
||||
void insertPartitionValueColumn(
|
||||
size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name)
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = partition_value_type->createColumnConst(rows, Tuple(partition_value.begin(), partition_value.end()))
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = partition_value_type->createColumn();
|
||||
|
||||
block.insert({column, partition_value_type, name});
|
||||
}
|
||||
|
||||
Block & block;
|
||||
};
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are not const for all rows
|
||||
static void injectNonConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part_offset")
|
||||
{
|
||||
if (!rows)
|
||||
{
|
||||
inserter.insertUInt64Column(DataTypeUInt64().createColumn(), virtual_column_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!inserter.columnExists(virtual_column_name))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column {} must have been filled part reader",
|
||||
virtual_column_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (virtual_column_name == LightweightDeleteDescription::FILTER_COLUMN.name)
|
||||
{
|
||||
/// If _row_exists column isn't present in the part then fill it here with 1s
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumnConst(rows, 1)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = LightweightDeleteDescription::FILTER_COLUMN.type->createColumn();
|
||||
|
||||
inserter.insertUInt8Column(column, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Adds virtual columns that are const for the whole part
|
||||
static void injectPartConstVirtualColumns(
|
||||
size_t rows,
|
||||
Block & block,
|
||||
MergeTreeReadTask * task,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns)
|
||||
{
|
||||
VirtualColumnsInserter inserter(block);
|
||||
/// add virtual columns
|
||||
/// Except _sample_factor, which is added from the outside.
|
||||
if (!virtual_columns.empty())
|
||||
{
|
||||
if (unlikely(rows && !task))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insert virtual columns to non-empty chunk without specified task.");
|
||||
|
||||
const IMergeTreeDataPart * part = nullptr;
|
||||
|
||||
if (rows)
|
||||
{
|
||||
part = task->getInfo().data_part.get();
|
||||
if (part->isProjectionPart())
|
||||
part = part->getParentPart();
|
||||
}
|
||||
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->name)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_index")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUInt64().createColumnConst(rows, task->getInfo().part_index_in_query)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUInt64().createColumn();
|
||||
|
||||
inserter.insertUInt64Column(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_part_uuid")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeUUID().createColumnConst(rows, part->uuid)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeUUID().createColumn();
|
||||
|
||||
inserter.insertUUIDColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_id")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}
|
||||
.createColumnConst(rows, part->info.partition_id)
|
||||
->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn();
|
||||
|
||||
inserter.insertLowCardinalityColumn(column, virtual_column_name);
|
||||
}
|
||||
else if (virtual_column_name == "_partition_value")
|
||||
{
|
||||
if (rows)
|
||||
inserter.insertPartitionValueColumn(rows, part->partition.value, partition_value_type, virtual_column_name);
|
||||
else
|
||||
inserter.insertPartitionValueColumn(rows, {}, partition_value_type, virtual_column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeSelectProcessor::injectVirtualColumns(
|
||||
Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
/// First add non-const columns that are filled by the range reader and then const columns that we will fill ourselves.
|
||||
/// Note that the order is important: virtual columns filled by the range reader must go first
|
||||
injectNonConstVirtualColumns(row_count, block, virtual_columns);
|
||||
injectPartConstVirtualColumns(row_count, block, task, partition_value_type, virtual_columns);
|
||||
}
|
||||
|
||||
Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info)
|
||||
{
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
block = prewhere_info->row_level_filter->updateHeader(std::move(block));
|
||||
auto & row_level_column = block.getByName(prewhere_info->row_level_column_name);
|
||||
if (!row_level_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
row_level_column.type->getName());
|
||||
}
|
||||
|
||||
block.erase(prewhere_info->row_level_column_name);
|
||||
}
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
block = prewhere_info->prewhere_actions->updateHeader(std::move(block));
|
||||
|
||||
auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name);
|
||||
if (!prewhere_column.type->canBeUsedInBooleanContext())
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
|
||||
if (prewhere_info->remove_prewhere_column)
|
||||
{
|
||||
block.erase(prewhere_info->prewhere_column_name);
|
||||
}
|
||||
else if (prewhere_info->need_filter)
|
||||
{
|
||||
WhichDataType which(removeNullable(recursiveRemoveLowCardinality(prewhere_column.type)));
|
||||
|
||||
if (which.isNativeInt() || which.isNativeUInt())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst();
|
||||
else if (which.isFloat())
|
||||
prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), 1.0f)->convertToFullColumnIfConst();
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
|
||||
"Illegal type {} of column for filter",
|
||||
prewhere_column.type->getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
Block MergeTreeSelectProcessor::transformHeader(
|
||||
Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns)
|
||||
{
|
||||
auto transformed = applyPrewhereActions(std::move(block), prewhere_info);
|
||||
injectVirtualColumns(transformed, 0, nullptr, partition_value_type, virtual_columns);
|
||||
return transformed;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,78 +1,118 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadTask.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectAlgorithms.h>
|
||||
#include <boost/core/noncopyable.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct PrewhereExprInfo;
|
||||
|
||||
/// Used to read data from single part with select query
|
||||
/// Cares about PREWHERE, virtual columns, indexes etc.
|
||||
/// To read data from multiple parts, Storage (MergeTree) creates multiple such objects.
|
||||
class MergeTreeSelectAlgorithm : public IMergeTreeSelectAlgorithm
|
||||
struct ChunkAndProgress
|
||||
{
|
||||
public:
|
||||
MergeTreeSelectAlgorithm(
|
||||
const MergeTreeData & storage,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part_,
|
||||
const AlterConversionsPtr & alter_conversions_,
|
||||
UInt64 max_block_size_rows,
|
||||
size_t preferred_block_size_bytes,
|
||||
size_t preferred_max_column_in_block_size_bytes,
|
||||
Names required_columns_,
|
||||
MarkRanges mark_ranges,
|
||||
bool use_uncompressed_cache,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool_,
|
||||
const Names & virt_column_names = {},
|
||||
size_t part_index_in_query_ = 0,
|
||||
bool has_limit_below_one_block_ = false);
|
||||
|
||||
~MergeTreeSelectAlgorithm() override;
|
||||
|
||||
protected:
|
||||
/// Defer initialization from constructor, because it may be heavy
|
||||
/// and it's better to do it lazily in `getNewTaskImpl`, which is executing in parallel.
|
||||
void initializeReaders();
|
||||
void finish() final;
|
||||
|
||||
/// Used by Task
|
||||
Names required_columns;
|
||||
/// Names from header. Used in order to order columns in read blocks.
|
||||
Names ordered_names;
|
||||
NameSet column_name_set;
|
||||
|
||||
MergeTreeReadTaskColumns task_columns;
|
||||
|
||||
/// Data part will not be removed if the pointer owns it
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
|
||||
/// Alter converversionss that should be applied on-fly for part.
|
||||
AlterConversionsPtr alter_conversions;
|
||||
|
||||
/// Cache getSampleBlock call, which might be heavy.
|
||||
Block sample_block;
|
||||
|
||||
/// Mark ranges we should read (in ascending order)
|
||||
MarkRanges all_mark_ranges;
|
||||
/// Value of _part_index virtual column (used only in SelectExecutor)
|
||||
size_t part_index_in_query = 0;
|
||||
/// If true, every task will be created only with one range.
|
||||
/// It reduces amount of read data for queries with small LIMIT.
|
||||
bool has_limit_below_one_block = false;
|
||||
|
||||
/// Pool for reading in order
|
||||
MergeTreeInOrderReadPoolParallelReplicasPtr pool;
|
||||
|
||||
size_t total_rows = 0;
|
||||
Chunk chunk;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
/// Explicitly indicate that we have read all data.
|
||||
/// This is needed to occasionally return empty chunk to indicate the progress while the rows are filtered out in PREWHERE.
|
||||
bool is_finished = false;
|
||||
};
|
||||
|
||||
struct ParallelReadingExtension
|
||||
{
|
||||
MergeTreeAllRangesCallback all_callback;
|
||||
MergeTreeReadTaskCallback callback;
|
||||
size_t count_participating_replicas{0};
|
||||
size_t number_of_current_replica{0};
|
||||
/// This is needed to estimate the number of bytes
|
||||
/// between a pair of marks to perform one request
|
||||
/// over the network for a 1Gb of data.
|
||||
Names columns_to_read;
|
||||
};
|
||||
|
||||
/// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm
|
||||
class MergeTreeSelectProcessor : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
MergeTreeSelectProcessor(
|
||||
MergeTreeReadPoolPtr pool_,
|
||||
MergeTreeSelectAlgorithmPtr algorithm_,
|
||||
const MergeTreeData & storage_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReadTask::BlockSizeParams & block_size_params_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & virt_column_names_);
|
||||
|
||||
String getName() const;
|
||||
|
||||
static Block transformHeader(
|
||||
Block block,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
const DataTypePtr & partition_value_type,
|
||||
const Names & virtual_columns);
|
||||
|
||||
Block getHeader() const { return result_header; }
|
||||
|
||||
ChunkAndProgress read();
|
||||
|
||||
void cancel() { is_cancelled = true; }
|
||||
|
||||
const MergeTreeReaderSettings & getSettings() const { return reader_settings; }
|
||||
|
||||
static PrewhereExprInfo getPrewhereActions(
|
||||
PrewhereInfoPtr prewhere_info,
|
||||
const ExpressionActionsSettings & actions_settings,
|
||||
bool enable_multiple_prewhere_read_steps);
|
||||
|
||||
private:
|
||||
/// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk
|
||||
struct BlockAndProgress
|
||||
{
|
||||
Block block;
|
||||
size_t row_count = 0;
|
||||
size_t num_read_rows = 0;
|
||||
size_t num_read_bytes = 0;
|
||||
};
|
||||
|
||||
/// Used for filling header with no rows as well as block with data
|
||||
static void injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns);
|
||||
static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info);
|
||||
|
||||
/// Sets up range readers corresponding to data readers
|
||||
void initializeRangeReaders();
|
||||
|
||||
const MergeTreeReadPoolPtr pool;
|
||||
const MergeTreeSelectAlgorithmPtr algorithm;
|
||||
|
||||
const PrewhereInfoPtr prewhere_info;
|
||||
const ExpressionActionsSettings actions_settings;
|
||||
const PrewhereExprInfo prewhere_actions;
|
||||
|
||||
const MergeTreeReaderSettings reader_settings;
|
||||
const MergeTreeReadTask::BlockSizeParams block_size_params;
|
||||
const Names virt_column_names;
|
||||
const DataTypePtr partition_value_type;
|
||||
|
||||
/// Current task to read from.
|
||||
MergeTreeReadTaskPtr task;
|
||||
/// This step is added when the part has lightweight delete mask
|
||||
PrewhereExprStepPtr lightweight_delete_filter_step;
|
||||
/// These columns will be filled by the merge tree range reader
|
||||
Names non_const_virtual_column_names;
|
||||
/// This header is used for chunks from readFromPart().
|
||||
Block header_without_const_virtual_columns;
|
||||
/// A result of getHeader(). A chunk which this header is returned from read().
|
||||
Block result_header;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeTreeSelectProcessor");
|
||||
std::atomic<bool> is_cancelled{false};
|
||||
};
|
||||
|
||||
using MergeTreeSelectProcessorPtr = std::unique_ptr<MergeTreeSelectProcessor>;
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeSource.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
#include <Common/EventFD.h>
|
||||
@ -24,7 +24,7 @@ struct MergeTreeSource::AsyncReadingState
|
||||
/// which can be called from background thread.
|
||||
/// Invariant:
|
||||
/// * background thread changes status InProgress -> IsFinished
|
||||
/// * (status == InProgress) => (MergeTreeBaseSelectProcessor is alive)
|
||||
/// * (status == InProgress) => (MergeTreeSelectProcessor is alive)
|
||||
|
||||
void setResult(ChunkAndProgress chunk_)
|
||||
{
|
||||
@ -118,7 +118,7 @@ struct MergeTreeSource::AsyncReadingState
|
||||
/// (executing thread) (bg pool thread)
|
||||
/// Control::finish()
|
||||
/// stage = Stage::IsFinished;
|
||||
/// ~MergeTreeBaseSelectProcessor()
|
||||
/// ~MergeTreeSelectProcessor()
|
||||
/// ~AsyncReadingState()
|
||||
/// control->stage != Stage::InProgress
|
||||
/// ~EventFD()
|
||||
@ -133,12 +133,12 @@ private:
|
||||
};
|
||||
#endif
|
||||
|
||||
MergeTreeSource::MergeTreeSource(MergeTreeSelectAlgorithmPtr algorithm_)
|
||||
: ISource(algorithm_->getHeader())
|
||||
, algorithm(std::move(algorithm_))
|
||||
MergeTreeSource::MergeTreeSource(MergeTreeSelectProcessorPtr processor_)
|
||||
: ISource(processor_->getHeader())
|
||||
, processor(std::move(processor_))
|
||||
{
|
||||
#if defined(OS_LINUX)
|
||||
if (algorithm->getSettings().use_asynchronous_read_from_pool)
|
||||
if (processor->getSettings().use_asynchronous_read_from_pool)
|
||||
async_reading_state = std::make_unique<AsyncReadingState>();
|
||||
#endif
|
||||
}
|
||||
@ -147,12 +147,12 @@ MergeTreeSource::~MergeTreeSource() = default;
|
||||
|
||||
std::string MergeTreeSource::getName() const
|
||||
{
|
||||
return algorithm->getName();
|
||||
return processor->getName();
|
||||
}
|
||||
|
||||
void MergeTreeSource::onCancel()
|
||||
{
|
||||
algorithm->cancel();
|
||||
processor->cancel();
|
||||
}
|
||||
|
||||
ISource::Status MergeTreeSource::prepare()
|
||||
@ -184,7 +184,7 @@ Chunk MergeTreeSource::processReadResult(ChunkAndProgress chunk)
|
||||
finished = chunk.is_finished;
|
||||
|
||||
/// We can return a chunk with no rows even if are not finished.
|
||||
/// This allows to report progress when all the rows are filtered out inside MergeTreeBaseSelectProcessor by PREWHERE logic.
|
||||
/// This allows to report progress when all the rows are filtered out inside MergeTreeSelectProcessor by PREWHERE logic.
|
||||
return std::move(chunk.chunk);
|
||||
}
|
||||
|
||||
@ -200,7 +200,7 @@ std::optional<Chunk> MergeTreeSource::tryGenerate()
|
||||
chassert(async_reading_state->getStage() == AsyncReadingState::Stage::NotStarted);
|
||||
|
||||
/// It is important to store control into job.
|
||||
/// Otherwise, race between job and ~MergeTreeBaseSelectProcessor is possible.
|
||||
/// Otherwise, race between job and ~MergeTreeSelectProcessor is possible.
|
||||
auto job = [this, control = async_reading_state->start()]() mutable
|
||||
{
|
||||
auto holder = std::move(control);
|
||||
@ -208,7 +208,7 @@ std::optional<Chunk> MergeTreeSource::tryGenerate()
|
||||
try
|
||||
{
|
||||
OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"};
|
||||
holder->setResult(algorithm->read());
|
||||
holder->setResult(processor->read());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -223,7 +223,7 @@ std::optional<Chunk> MergeTreeSource::tryGenerate()
|
||||
#endif
|
||||
|
||||
OpenTelemetry::SpanHolder span{"MergeTreeSource::tryGenerate()"};
|
||||
return processReadResult(algorithm->read());
|
||||
return processReadResult(processor->read());
|
||||
}
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
@ -4,15 +4,15 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeSelectAlgorithm;
|
||||
using MergeTreeSelectAlgorithmPtr = std::unique_ptr<IMergeTreeSelectAlgorithm>;
|
||||
class MergeTreeSelectProcessor;
|
||||
using MergeTreeSelectProcessorPtr = std::unique_ptr<MergeTreeSelectProcessor>;
|
||||
|
||||
struct ChunkAndProgress;
|
||||
|
||||
class MergeTreeSource final : public ISource
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeSource(MergeTreeSelectAlgorithmPtr algorithm_);
|
||||
explicit MergeTreeSource(MergeTreeSelectProcessorPtr processor_);
|
||||
~MergeTreeSource() override;
|
||||
|
||||
std::string getName() const override;
|
||||
@ -29,7 +29,7 @@ protected:
|
||||
void onCancel() override;
|
||||
|
||||
private:
|
||||
MergeTreeSelectAlgorithmPtr algorithm;
|
||||
MergeTreeSelectProcessorPtr processor;
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
struct AsyncReadingState;
|
||||
|
@ -1,73 +0,0 @@
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm(
|
||||
size_t thread_,
|
||||
IMergeTreeReadPoolPtr pool_,
|
||||
size_t min_marks_for_concurrent_read_,
|
||||
size_t max_block_size_rows_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
size_t preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
bool use_uncompressed_cache_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & virt_column_names_)
|
||||
: IMergeTreeSelectAlgorithm{
|
||||
pool_->getHeader(), storage_, storage_snapshot_, prewhere_info_, actions_settings_, max_block_size_rows_,
|
||||
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
|
||||
reader_settings_, use_uncompressed_cache_, virt_column_names_},
|
||||
thread{thread_},
|
||||
pool{std::move(pool_)}
|
||||
{
|
||||
min_marks_to_read = min_marks_for_concurrent_read_;
|
||||
}
|
||||
|
||||
/// Requests read task from MergeTreeReadPool and signals whether it got one
|
||||
bool MergeTreeThreadSelectAlgorithm::getNewTaskImpl()
|
||||
{
|
||||
task = pool->getTask(thread);
|
||||
return static_cast<bool>(task);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeThreadSelectAlgorithm::finalizeNewTask()
|
||||
{
|
||||
const std::string part_name = task->data_part->isProjectionPart() ? task->data_part->getParentPart()->name : task->data_part->name;
|
||||
|
||||
/// Allows pool to reduce number of threads in case of too slow reads.
|
||||
auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); };
|
||||
IMergeTreeReader::ValueSizeMap value_size_map;
|
||||
|
||||
if (reader && part_name != last_read_part_name)
|
||||
{
|
||||
value_size_map = reader->getAvgValueSizeHints();
|
||||
}
|
||||
|
||||
/// task->reader.valid() means there is a prefetched reader in this test, use it.
|
||||
const bool init_new_readers = !reader || task->reader.valid() || part_name != last_read_part_name;
|
||||
if (init_new_readers)
|
||||
initializeMergeTreeReadersForCurrentTask(value_size_map, profile_callback);
|
||||
|
||||
last_read_part_name = part_name;
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeThreadSelectAlgorithm::finish()
|
||||
{
|
||||
reader.reset();
|
||||
pre_reader_for_step.clear();
|
||||
}
|
||||
|
||||
|
||||
MergeTreeThreadSelectAlgorithm::~MergeTreeThreadSelectAlgorithm() = default;
|
||||
|
||||
}
|
@ -1,54 +0,0 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IMergeTreeReadPool;
|
||||
using IMergeTreeReadPoolPtr = std::shared_ptr<IMergeTreeReadPool>;
|
||||
|
||||
/** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked
|
||||
* to perform.
|
||||
*/
|
||||
class MergeTreeThreadSelectAlgorithm final : public IMergeTreeSelectAlgorithm
|
||||
{
|
||||
public:
|
||||
MergeTreeThreadSelectAlgorithm(
|
||||
size_t thread_,
|
||||
IMergeTreeReadPoolPtr pool_,
|
||||
size_t min_marks_for_concurrent_read,
|
||||
size_t max_block_size_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
size_t preferred_max_column_in_block_size_bytes_,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageSnapshotPtr & storage_snapshot_,
|
||||
bool use_uncompressed_cache_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const ExpressionActionsSettings & actions_settings_,
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
const Names & virt_column_names_);
|
||||
|
||||
String getName() const override { return "MergeTreeThread"; }
|
||||
|
||||
~MergeTreeThreadSelectAlgorithm() override;
|
||||
|
||||
protected:
|
||||
/// Requests read task from MergeTreeReadPool and signals whether it got one
|
||||
bool getNewTaskImpl() override;
|
||||
|
||||
void finalizeNewTask() override;
|
||||
|
||||
void finish() override;
|
||||
|
||||
private:
|
||||
/// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1])
|
||||
size_t thread;
|
||||
|
||||
IMergeTreeReadPoolPtr pool;
|
||||
|
||||
/// Last part read in this thread
|
||||
std::string last_read_part_name;
|
||||
};
|
||||
|
||||
}
|
@ -9,9 +9,9 @@ ExpressionTransform × 3
|
||||
ExpressionTransform × 3
|
||||
(ReadFromMergeTree)
|
||||
ExpressionTransform × 4
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
@ -4,7 +4,7 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
ReplacingSorted 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
0 0
|
||||
1 1
|
||||
2 2
|
||||
@ -20,9 +20,9 @@ ExpressionTransform × 2
|
||||
FilterSortedStreamByRange
|
||||
Description: filter values in [(5), +inf)
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
ReplacingSorted 2 → 1
|
||||
FilterSortedStreamByRange × 2
|
||||
Description: filter values in [-inf, (5))
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
|
@ -14,7 +14,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
@ -32,9 +32,9 @@ ExpressionTransform
|
||||
ExpressionTransform × 2
|
||||
(ReadFromMergeTree)
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
2020-10-01 9
|
||||
@ -51,7 +51,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
2020-10-11 0
|
||||
2020-10-11 0
|
||||
2020-10-11 0
|
||||
@ -65,7 +65,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
@ -77,7 +77,7 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
2020-10-11 0 0
|
||||
2020-10-11 0 10
|
||||
2020-10-11 0 20
|
||||
@ -93,7 +93,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
@ -107,7 +107,7 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
2020-10-12 0
|
||||
2020-10-12 1
|
||||
2020-10-12 2
|
||||
@ -129,7 +129,7 @@ ExpressionTransform
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
@ -142,7 +142,7 @@ ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
ExpressionTransform
|
||||
ReverseTransform
|
||||
MergeTreeReverse 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
|
||||
2020-10-12 99999
|
||||
2020-10-12 99998
|
||||
2020-10-12 99997
|
||||
@ -169,7 +169,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
2020-10-10 00:00:00 0.01
|
||||
2020-10-10 00:00:00 0.01
|
||||
2020-10-10 00:00:00 0.01
|
||||
@ -183,7 +183,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
2020-10-10 00:00:00 0.01
|
||||
2020-10-10 00:00:00 0.01
|
||||
2020-10-10 00:00:00 0.01
|
||||
|
@ -19,7 +19,7 @@ ExpressionTransform × 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
explain pipeline select parent_key, child_key, count() from data_02233 group by parent_key, child_key with totals order by parent_key, child_key settings max_threads=1, optimize_aggregation_in_order=0, read_in_order_two_level_merge_threshold=1;
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
@ -36,7 +36,7 @@ ExpressionTransform × 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
select parent_key, child_key, count() from data_02233 group by parent_key, child_key with totals order by parent_key, child_key settings max_threads=1, optimize_aggregation_in_order=1;
|
||||
0 0 4
|
||||
0 1 3
|
||||
|
@ -38,19 +38,19 @@ DistinctTransform
|
||||
DistinctTransform
|
||||
-- Check reading in order for distinct
|
||||
-- disabled, distinct columns match sorting key
|
||||
MergeTreeThread
|
||||
algorithm: Thread
|
||||
-- enabled, distinct columns match sorting key
|
||||
MergeTreeInOrder
|
||||
algorithm: InOrder
|
||||
-- enabled, distinct columns form prefix of sorting key
|
||||
MergeTreeInOrder
|
||||
algorithm: InOrder
|
||||
-- enabled, distinct columns DON't form prefix of sorting key
|
||||
MergeTreeThread
|
||||
algorithm: Thread
|
||||
-- enabled, distinct columns contains constant columns, non-const columns form prefix of sorting key
|
||||
MergeTreeInOrder
|
||||
algorithm: InOrder
|
||||
-- enabled, distinct columns contains constant columns, non-const columns match prefix of sorting key
|
||||
MergeTreeInOrder
|
||||
algorithm: InOrder
|
||||
-- enabled, only part of distinct columns form prefix of sorting key
|
||||
MergeTreeThread
|
||||
algorithm: Thread
|
||||
=== disable new analyzer ===
|
||||
-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct
|
||||
Sorting (Stream): a ASC, b ASC
|
||||
|
@ -12,8 +12,8 @@ GREP_DISTINCT="grep 'DistinctSortedChunkTransform\|DistinctSortedStreamTransform
|
||||
TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'"
|
||||
REMOVE_NON_LETTERS="sed 's/[^a-zA-Z]//g'"
|
||||
FIND_DISTINCT="$GREP_DISTINCT | $TRIM_LEADING_SPACES | $REMOVE_NON_LETTERS"
|
||||
FIND_READING_IN_ORDER="grep 'MergeTreeInOrder' | $TRIM_LEADING_SPACES | $REMOVE_NON_LETTERS"
|
||||
FIND_READING_DEFAULT="grep 'MergeTreeThread' | $TRIM_LEADING_SPACES | $REMOVE_NON_LETTERS"
|
||||
FIND_READING_IN_ORDER="grep -o 'algorithm: InOrder' | $TRIM_LEADING_SPACES"
|
||||
FIND_READING_DEFAULT="grep -o 'algorithm: Thread' | $TRIM_LEADING_SPACES"
|
||||
FIND_SORTING_PROPERTIES="grep 'Sorting (Stream)' | $TRIM_LEADING_SPACES"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync"
|
||||
|
@ -1,2 +1,2 @@
|
||||
data_02340 1_2_2_0 5
|
||||
data_02340_rep 1_0_0_0 5
|
||||
data_02340 1_2_2_0 6
|
||||
data_02340_rep 1_0_0_0 6
|
||||
|
@ -100,7 +100,7 @@ ExpressionTransform × 16
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(ReadFromRemote)
|
||||
explain pipeline SELECT k1, k3, sum(value) v FROM remote('127.0.0.{1,2}', currentDatabase(), proj_agg_02343) GROUP BY k1, k3 SETTINGS distributed_aggregation_memory_efficient = 1;
|
||||
(Expression)
|
||||
@ -117,7 +117,7 @@ ExpressionTransform × 16
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(ReadFromRemote)
|
||||
-- { echoOn }
|
||||
|
||||
@ -144,6 +144,6 @@ ExpressionTransform × 2
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
(ReadFromRemote)
|
||||
(ReadFromRemote)
|
||||
|
@ -5,6 +5,8 @@ set max_threads = 16;
|
||||
set prefer_localhost_replica = 1;
|
||||
set optimize_aggregation_in_order = 0;
|
||||
set max_block_size = 65505;
|
||||
set allow_prefetched_read_pool_for_remote_filesystem = 0;
|
||||
set allow_prefetched_read_pool_for_local_filesystem = 0;
|
||||
|
||||
-- { echoOn }
|
||||
|
||||
|
@ -16,7 +16,7 @@ ExpressionTransform × 4
|
||||
(Expression)
|
||||
ExpressionTransform × 4
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 4 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 4 0 → 1
|
||||
(ReadFromRemote)
|
||||
select a from remote(test_cluster_two_shards, currentDatabase(), t) group by a order by a limit 5 offset 100500;
|
||||
100500
|
||||
@ -47,7 +47,7 @@ ExpressionTransform × 4
|
||||
(Expression)
|
||||
ExpressionTransform × 4
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 4 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 4 0 → 1
|
||||
(ReadFromRemote)
|
||||
(ReadFromRemote)
|
||||
select a from remote(test_cluster_two_shards, currentDatabase(), dist_t) group by a order by a limit 5 offset 100500;
|
||||
@ -84,7 +84,7 @@ ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform × 4
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder × 4 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 4 0 → 1
|
||||
(ReadFromRemote)
|
||||
select a, count() from dist_t_different_dbs group by a order by a limit 5 offset 500;
|
||||
500 2000
|
||||
|
@ -57,6 +57,8 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off
|
||||
|
||||
-- { echoOff } --
|
||||
|
||||
drop table if exists pr_t;
|
||||
|
||||
create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a;
|
||||
insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
|
||||
|
||||
|
@ -3,33 +3,33 @@
|
||||
-- The number of output streams is limited by max_streams_for_merge_tree_reading
|
||||
select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0;
|
||||
49999995000000
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
Resize 16 → 32
|
||||
StrictResize 16 → 16
|
||||
MergeTreeThread × 16 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 16 0 → 1
|
||||
-- Without asynchronous_read, max_streams_for_merge_tree_reading limits max_streams * max_streams_to_max_threads_ratio
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8;
|
||||
49999995000000
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
Resize 16 → 4
|
||||
StrictResize 16 → 16
|
||||
MergeTreeThread × 16 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 16 0 → 1
|
||||
-- With asynchronous_read, read in max_streams_for_merge_tree_reading async streams and resize to max_threads
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1;
|
||||
49999995000000
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
Resize 4 → 4
|
||||
StrictResize 4 → 4
|
||||
Resize 16 → 4
|
||||
MergeTreeThread × 16 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 16 0 → 1
|
||||
-- With asynchronous_read, read using max_streams * max_streams_to_max_threads_ratio async streams, resize to max_streams_for_merge_tree_reading outp[ut streams, resize to max_threads after aggregation
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8;
|
||||
49999995000000
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
Resize 16 → 4
|
||||
StrictResize 16 → 16
|
||||
Resize 32 → 16
|
||||
MergeTreeThread × 32 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 32 0 → 1
|
||||
-- For read-in-order, disable everything
|
||||
set query_plan_remove_redundant_sorting=0; -- to keep reading in order
|
||||
select sum(x) from (select x from t order by x) settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, optimize_read_in_order=1, query_plan_read_in_order=1;
|
||||
|
@ -4,23 +4,26 @@ drop table if exists t;
|
||||
create table t (x UInt64) engine = MergeTree order by x;
|
||||
insert into t select number from numbers_mt(10000000) settings max_insert_threads=8;
|
||||
|
||||
set allow_prefetched_read_pool_for_remote_filesystem = 0;
|
||||
set allow_prefetched_read_pool_for_local_filesystem = 0;
|
||||
|
||||
-- { echo }
|
||||
|
||||
-- The number of output streams is limited by max_streams_for_merge_tree_reading
|
||||
select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0;
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=32, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
|
||||
-- Without asynchronous_read, max_streams_for_merge_tree_reading limits max_streams * max_streams_to_max_threads_ratio
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8;
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=0, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
|
||||
-- With asynchronous_read, read in max_streams_for_merge_tree_reading async streams and resize to max_threads
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1;
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
|
||||
-- With asynchronous_read, read using max_streams * max_streams_to_max_threads_ratio async streams, resize to max_streams_for_merge_tree_reading outp[ut streams, resize to max_threads after aggregation
|
||||
select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8;
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeThread%';
|
||||
select * from (explain pipeline select sum(x) from t settings max_threads=4, max_streams_for_merge_tree_reading=16, allow_asynchronous_read_from_io_pool_for_merge_tree=1, max_streams_to_max_threads_ratio=8) where explain like '%Resize%' or explain like '%MergeTreeSelect%';
|
||||
|
||||
-- For read-in-order, disable everything
|
||||
set query_plan_remove_redundant_sorting=0; -- to keep reading in order
|
||||
|
@ -7,13 +7,13 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 4
|
||||
(ReadFromMergeTree)
|
||||
Resize 3 → 1
|
||||
MergeTreeThread × 3 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 3 0 → 1
|
||||
Resize 3 → 1
|
||||
MergeTreeThread × 3 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 3 0 → 1
|
||||
Resize 3 → 1
|
||||
MergeTreeThread × 3 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 3 0 → 1
|
||||
Resize 3 → 1
|
||||
MergeTreeThread × 3 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 3 0 → 1
|
||||
1000000
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
@ -24,21 +24,21 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 8
|
||||
(ReadFromMergeTree)
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
Resize 2 → 1
|
||||
MergeTreeThread × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1
|
||||
1000000
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
@ -49,37 +49,37 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 16
|
||||
(ReadFromMergeTree)
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
Concat 2 → 1
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
1000000
|
||||
1000000
|
||||
(Expression)
|
||||
@ -94,16 +94,16 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 4
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
1000000
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
@ -117,28 +117,28 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 8
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
1000000
|
||||
(Expression)
|
||||
ExpressionTransform × 16
|
||||
@ -151,52 +151,52 @@ ExpressionTransform × 16
|
||||
ExpressionTransform × 16
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
MergingSortedTransform 2 → 1
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
1000000
|
||||
Skip merging: 1
|
||||
Skip merging: 1
|
||||
|
@ -5,6 +5,9 @@ set allow_aggregate_partitions_independently = 1;
|
||||
set force_aggregate_partitions_independently = 1;
|
||||
set optimize_use_projections = 0;
|
||||
|
||||
set allow_prefetched_read_pool_for_remote_filesystem = 0;
|
||||
set allow_prefetched_read_pool_for_local_filesystem = 0;
|
||||
|
||||
create table t1(a UInt32) engine=MergeTree order by tuple() partition by a % 4 settings index_granularity = 8192, index_granularity_bytes = 10485760;
|
||||
|
||||
system stop merges t1;
|
||||
|
@ -40,7 +40,7 @@ ExpressionTransform × 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
||||
---Result---
|
||||
2023-01-05 all
|
||||
@ -59,4 +59,4 @@ ExpressionTransform × 2
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
@ -4,4 +4,4 @@ ExpressionTransform
|
||||
Limit
|
||||
(ReadFromMergeTree)
|
||||
Concat 3 → 1
|
||||
MergeTreeInOrder × 3 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 3 0 → 1
|
||||
|
@ -1,3 +1,5 @@
|
||||
drop table if exists t;
|
||||
|
||||
create table t(a UInt64) engine=MergeTree order by tuple();
|
||||
|
||||
system stop merges t;
|
||||
|
@ -1 +1 @@
|
||||
MergeTreeInOrder
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
@ -5,4 +5,4 @@ set allow_experimental_analyzer=1;
|
||||
create table t (a UInt64, b UInt64) engine=MergeTree() order by (a);
|
||||
insert into t select number % 2, number from numbers(10);
|
||||
|
||||
select splitByChar(' ', trimBoth(explain))[1] from (explain pipeline select distinct a from t) where explain like '%MergeTreeInOrder%';
|
||||
select trimBoth(explain) from (explain pipeline select distinct a from t) where explain like '%InOrder%';
|
||||
|
@ -12,12 +12,12 @@ ExpressionTransform × 2
|
||||
FilterSortedStreamByRange × 2
|
||||
Description: filter values in [(999424), +inf)
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
AggregatingSortedTransform
|
||||
FilterSortedStreamByRange
|
||||
Description: filter values in [-inf, (999424))
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY
|
||||
SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=0
|
||||
FORMAT LineAsString;
|
||||
@ -31,9 +31,9 @@ ExpressionTransform × 2
|
||||
FilterSortedStreamByRange × 2
|
||||
Description: filter values in [(999424), +inf)
|
||||
ExpressionTransform × 2
|
||||
MergeTreeInOrder × 2 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1
|
||||
AggregatingSortedTransform
|
||||
FilterSortedStreamByRange
|
||||
Description: filter values in [-inf, (999424))
|
||||
ExpressionTransform
|
||||
MergeTreeInOrder 0 → 1
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
@ -1,11 +1,11 @@
|
||||
[1mexplain[0m
|
||||
[1mexplain[0m
|
||||
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Aggregating)
|
||||
FinalizeAggregatedTransform
|
||||
AggregatingInOrderTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Aggregating)
|
||||
FinalizeAggregatedTransform
|
||||
AggregatingInOrderTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
|
@ -8,9 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
|
||||
check_replicas_read_in_order() {
|
||||
# to check this we actually look for at least one log message from MergeTreeInOrderSelectProcessor.
|
||||
# hopefully logger's names are a bit more stable than log messages itself
|
||||
#
|
||||
# NOTE: lack of "current_database = '$CLICKHOUSE_DATABASE'" filter is made on purpose
|
||||
$CLICKHOUSE_CLIENT -nq "
|
||||
SYSTEM FLUSH LOGS;
|
||||
@ -18,7 +15,7 @@ check_replicas_read_in_order() {
|
||||
SELECT COUNT() > 0
|
||||
FROM system.text_log
|
||||
WHERE query_id IN (SELECT query_id FROM system.query_log WHERE query_id != '$1' AND initial_query_id = '$1' AND event_date >= yesterday())
|
||||
AND event_date >= yesterday() AND logger_name = 'MergeTreeInOrderSelectProcessor'"
|
||||
AND event_date >= yesterday() AND message ILIKE '%Reading%ranges in order%'"
|
||||
}
|
||||
|
||||
# replicas should use reading in order following initiator's decision to execute aggregation in order.
|
||||
|
Loading…
Reference in New Issue
Block a user