Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom

This commit is contained in:
Alexey Milovidov 2024-03-24 00:42:59 +01:00
parent 5651c187cc
commit ea2fd57cca
10 changed files with 86 additions and 16 deletions

View File

@ -2501,10 +2501,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
max_block_size = std::max<UInt64>(1, max_block_limited);
max_threads_execute_query = max_streams = 1;
}
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;
else /// 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;
}
else
{

View File

@ -12,8 +12,8 @@
#include <fmt/format.h>
#include <Common/iota.h>
#include <Common/typeid_cast.h>
#include "Core/Types.h"
#include "base/types.h"
#include <Core/Types.h>
namespace DB
{
@ -443,7 +443,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
/// Build rpn of query filters
KeyCondition condition(filter_actions_dag, context, column_names, key_expression);
if (condition.extractPlainRanges(ranges))
{
/// Intersect ranges with table range
@ -505,7 +504,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
}
}
/// ranges is blank, return a source who has no data
if (intersected_ranges.empty())
{

View File

@ -8,6 +8,7 @@
#include <Storages/SelectQueryInfo.h>
#include <Storages/StorageSnapshot.h>
namespace DB
{
@ -43,4 +44,5 @@ private:
UInt64 limit;
std::shared_ptr<const StorageLimitsList> storage_limits;
};
}

View File

@ -3,6 +3,7 @@
#include <Storages/StorageGenerateRandom.h>
#include <Storages/StorageFactory.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Storages/SelectQueryInfo.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <QueryPipeline/Pipe.h>
#include <Parsers/ASTLiteral.h>
@ -30,12 +31,9 @@
#include <Common/SipHash.h>
#include <Common/randomSeed.h>
#include <Interpreters/Context.h>
#include <base/unaligned.h>
#include <Functions/FunctionFactory.h>
#include <pcg_random.hpp>
namespace DB
{
@ -639,7 +637,7 @@ void registerStorageGenerateRandom(StorageFactory & factory)
Pipe StorageGenerateRandom::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & /*query_info*/,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
@ -682,7 +680,14 @@ Pipe StorageGenerateRandom::read(
pcg64 generate(random_seed);
for (UInt64 i = 0; i < num_streams; ++i)
pipes.emplace_back(std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context));
{
auto source = std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context);
if (i == 0 && query_info.limit)
source->addTotalRowsApprox(query_info.limit);
pipes.emplace_back(std::move(source));
}
return Pipe::unitePipes(std::move(pipes));
}

View File

@ -1,17 +1,14 @@
#include <Storages/System/StorageSystemNumbers.h>
#include <mutex>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Processors/ISource.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
#include <Processors/Sources/NullSource.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/SelectQueryInfo.h>
namespace DB
{

View File

@ -1,4 +1,5 @@
#include <Storages/System/StorageSystemZeros.h>
#include <Storages/SelectQueryInfo.h>
#include <Processors/ISource.h>
#include <QueryPipeline/Pipe.h>
@ -6,6 +7,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
namespace DB
{
@ -93,7 +95,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith
Pipe StorageSystemZeros::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo &,
SelectQueryInfo & query_info,
ContextPtr /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
@ -123,8 +125,13 @@ Pipe StorageSystemZeros::read(
{
auto source = std::make_shared<ZerosSource>(max_block_size, limit ? *limit : 0, state);
if (limit && i == 0)
if (i == 0)
{
if (limit)
source->addTotalRowsApprox(*limit);
else if (query_info.limit)
source->addTotalRowsApprox(query_info.limit);
}
res.addSource(std::move(source));
}

View File

@ -0,0 +1,49 @@
#!/usr/bin/expect -f
set basedir [file dirname $argv0]
set basename [file tail $argv0]
if {[info exists env(CLICKHOUSE_TMP)]} {
set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP)
} else {
set CLICKHOUSE_TMP "."
}
exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0
log_user 0
set timeout 60
match_max 100000
set stty_init "rows 25 cols 120"
expect_after {
-i $any_spawn_id eof { exp_continue }
-i $any_spawn_id timeout { exit 1 }
}
spawn clickhouse-local
expect ":) "
# Trivial SELECT with LIMIT from system.zeros shows progress bar.
send "SELECT * FROM system.zeros LIMIT 1e11 FORMAT Null\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "SELECT * FROM system.zeros_mt LIMIT 1e11 FORMAT Null\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
# As well as from generateRandom
send "SELECT * FROM generateRandom() LIMIT 1e9 FORMAT Null\r"
expect "Progress: "
expect "█"
send "\3"
expect "Query was cancelled."
expect ":) "
send "exit\r"
expect eof

View File

@ -0,0 +1,9 @@
SET max_rows_to_read = 1e11;
SELECT * FROM system.numbers LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.numbers_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.zeros LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM system.zeros_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }
SELECT * FROM generateRandom() LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS }