Set total rows for in order queries with limit

+ renaming misleading query_info.limit -> trivial_limit
This commit is contained in:
Igor Nikonov 2024-06-27 18:54:32 +00:00
parent 77e51dc693
commit 7618ce12bb
9 changed files with 29 additions and 29 deletions

View File

@ -910,7 +910,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
UInt64 max_rows = maxBlockSizeByLimit();
if (settings.max_rows_to_read)
max_rows = max_rows ? std::min(max_rows, settings.max_rows_to_read.value) : settings.max_rows_to_read;
query_info_copy.limit = max_rows;
query_info_copy.trivial_limit = max_rows;
/// Apply filters to prewhere and add them to the query_info so we can filter out parts efficiently during row estimation
applyFiltersToPrewhereInAnalysis(analysis_copy);
@ -2445,13 +2445,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
if (local_limits.local_limits.size_limits.max_rows != 0)
{
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
query_info.limit = max_block_limited;
query_info.trivial_limit = max_block_limited;
else if (local_limits.local_limits.size_limits.max_rows < std::numeric_limits<UInt64>::max()) /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered).
query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows;
query_info.trivial_limit = 1 + local_limits.local_limits.size_limits.max_rows;
}
else
{
query_info.limit = max_block_limited;
query_info.trivial_limit = max_block_limited;
}
}

View File

@ -693,14 +693,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows != 0)
{
if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows)
table_expression_query_info.limit = max_block_size_limited;
table_expression_query_info.trivial_limit = max_block_size_limited;
/// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered).
else if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows < std::numeric_limits<UInt64>::max())
table_expression_query_info.limit = 1 + select_query_info.local_storage_limits.local_limits.size_limits.max_rows;
table_expression_query_info.trivial_limit = 1 + select_query_info.local_storage_limits.local_limits.size_limits.max_rows;
}
else
{
table_expression_query_info.limit = max_block_size_limited;
table_expression_query_info.trivial_limit = max_block_size_limited;
}
}
@ -913,8 +913,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto result_ptr = reading->selectRangesToRead();
UInt64 rows_to_read = result_ptr->selected_rows;
if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read)
rows_to_read = table_expression_query_info.limit;
if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read)
rows_to_read = table_expression_query_info.trivial_limit;
if (max_block_size_limited && (max_block_size_limited < rows_to_read))
rows_to_read = max_block_size_limited;

View File

@ -250,9 +250,9 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c
{
/// Fail fast if estimated number of rows to read exceeds the limit
size_t total_rows_estimate = selected_rows;
if (query_info_.limit > 0 && total_rows_estimate > query_info_.limit)
if (query_info_.trivial_limit > 0 && total_rows_estimate > query_info_.trivial_limit)
{
total_rows_estimate = query_info_.limit;
total_rows_estimate = query_info_.trivial_limit;
}
limits.check(total_rows_estimate, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS);
leaf_limits.check(
@ -398,8 +398,8 @@ Pipe ReadFromMergeTree::readFromPool(
{
size_t total_rows = parts_with_range.getRowsCountAllParts();
if (query_info.limit > 0 && query_info.limit < total_rows)
total_rows = query_info.limit;
if (query_info.trivial_limit > 0 && query_info.trivial_limit < total_rows)
total_rows = query_info.trivial_limit;
const auto & settings = context->getSettingsRef();
@ -436,7 +436,7 @@ Pipe ReadFromMergeTree::readFromPool(
* Because time spend during filling per thread tasks can be greater than whole query
* execution for big tables with small limit.
*/
bool use_prefetched_read_pool = query_info.limit == 0 && (allow_prefetched_remote || allow_prefetched_local);
bool use_prefetched_read_pool = query_info.trivial_limit == 0 && (allow_prefetched_remote || allow_prefetched_local);
if (use_prefetched_read_pool)
{
@ -563,9 +563,8 @@ Pipe ReadFromMergeTree::readInOrder(
/// Actually it means that parallel reading from replicas enabled
/// and we have to collaborate with initiator.
/// In this case we won't set approximate rows, because it will be accounted multiple times.
/// Also do not count amount of read rows if we read in order of sorting key,
/// because we don't know actual amount of read rows in case when limit is set.
const bool set_total_rows_approx = !is_parallel_reading_from_replicas && !query_info.limit;
const auto in_order_limit = query_info.input_order_info->limit;
const bool set_total_rows_approx = !is_parallel_reading_from_replicas;
Pipes pipes;
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
@ -573,8 +572,10 @@ Pipe ReadFromMergeTree::readInOrder(
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;
if (query_info.trivial_limit > 0 && query_info.trivial_limit < total_rows)
total_rows = query_info.trivial_limit;
else if (in_order_limit > 0 && in_order_limit < total_rows)
total_rows = in_order_limit;
LOG_TRACE(log, "Reading {} ranges in{}order from part {}, approx. {} rows starting from {}",
part_with_ranges.ranges.size(),

View File

@ -393,7 +393,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep(
, num_streams{num_streams_}
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery &>(), context))
, should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first))
, query_info_limit(query_info.limit)
, query_info_limit(query_info.trivial_limit)
, storage_limits(query_info.storage_limits)
{
storage_snapshot->check(column_names);

View File

@ -2,7 +2,6 @@
#include <Common/formatReadable.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <string>
namespace ProfileEvents

View File

@ -7111,8 +7111,8 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead(
query_context->getSettingsRef().max_threads);
UInt64 total_rows = result_ptr->selected_rows;
if (query_info.limit > 0 && query_info.limit < total_rows)
total_rows = query_info.limit;
if (query_info.trivial_limit > 0 && query_info.trivial_limit < total_rows)
total_rows = query_info.trivial_limit;
return total_rows;
}

View File

@ -229,8 +229,8 @@ struct SelectQueryInfo
bool is_parameterized_view = false;
bool optimize_trivial_count = false;
// If limit is not 0, that means it's a trivial limit query.
UInt64 limit = 0;
// If not 0, that means it's a trivial limit query.
UInt64 trivial_limit = 0;
/// For IStorageSystemOneBlock
std::vector<UInt8> columns_mask;

View File

@ -705,7 +705,7 @@ Pipe StorageGenerateRandom::read(
}
}
UInt64 query_limit = query_info.limit;
UInt64 query_limit = query_info.trivial_limit;
if (query_limit && num_streams * max_block_size > query_limit)
{
/// We want to avoid spawning more streams than necessary
@ -717,7 +717,7 @@ Pipe StorageGenerateRandom::read(
/// Will create more seed values for each source from initial seed.
pcg64 generate(random_seed);
auto shared_state = std::make_shared<GenerateRandomState>(query_info.limit);
auto shared_state = std::make_shared<GenerateRandomState>(query_info.trivial_limit);
for (UInt64 i = 0; i < num_streams; ++i)
{

View File

@ -109,8 +109,8 @@ Pipe StorageSystemZeros::read(
storage_snapshot->check(column_names);
UInt64 query_limit = limit ? *limit : 0;
if (query_info.limit)
query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit;
if (query_info.trivial_limit)
query_limit = query_limit ? std::min(query_limit, query_info.trivial_limit) : query_info.trivial_limit;
if (query_limit && query_limit < max_block_size)
max_block_size = query_limit;