mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Set total rows for in order queries with limit
+ renaming misleading query_info.limit -> trivial_limit
This commit is contained in:
parent
77e51dc693
commit
7618ce12bb
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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(),
|
||||
|
@ -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);
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user