mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 20:42:04 +00:00
Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom
This commit is contained in:
parent
5651c187cc
commit
ea2fd57cca
@ -2501,10 +2501,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
|||||||
max_block_size = std::max<UInt64>(1, max_block_limited);
|
max_block_size = std::max<UInt64>(1, max_block_limited);
|
||||||
max_threads_execute_query = max_streams = 1;
|
max_threads_execute_query = max_streams = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (local_limits.local_limits.size_limits.max_rows != 0)
|
if (local_limits.local_limits.size_limits.max_rows != 0)
|
||||||
{
|
{
|
||||||
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
|
if (max_block_limited < local_limits.local_limits.size_limits.max_rows)
|
||||||
query_info.limit = max_block_limited;
|
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
|
else
|
||||||
{
|
{
|
||||||
|
@ -12,8 +12,8 @@
|
|||||||
#include <fmt/format.h>
|
#include <fmt/format.h>
|
||||||
#include <Common/iota.h>
|
#include <Common/iota.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include "Core/Types.h"
|
#include <Core/Types.h>
|
||||||
#include "base/types.h"
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -443,7 +443,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
|
|||||||
/// Build rpn of query filters
|
/// Build rpn of query filters
|
||||||
KeyCondition condition(filter_actions_dag, context, column_names, key_expression);
|
KeyCondition condition(filter_actions_dag, context, column_names, key_expression);
|
||||||
|
|
||||||
|
|
||||||
if (condition.extractPlainRanges(ranges))
|
if (condition.extractPlainRanges(ranges))
|
||||||
{
|
{
|
||||||
/// Intersect ranges with table range
|
/// Intersect ranges with table range
|
||||||
@ -505,7 +504,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/// ranges is blank, return a source who has no data
|
/// ranges is blank, return a source who has no data
|
||||||
if (intersected_ranges.empty())
|
if (intersected_ranges.empty())
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
#include <Storages/SelectQueryInfo.h>
|
#include <Storages/SelectQueryInfo.h>
|
||||||
#include <Storages/StorageSnapshot.h>
|
#include <Storages/StorageSnapshot.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -43,4 +44,5 @@ private:
|
|||||||
UInt64 limit;
|
UInt64 limit;
|
||||||
std::shared_ptr<const StorageLimitsList> storage_limits;
|
std::shared_ptr<const StorageLimitsList> storage_limits;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <Storages/StorageGenerateRandom.h>
|
#include <Storages/StorageGenerateRandom.h>
|
||||||
#include <Storages/StorageFactory.h>
|
#include <Storages/StorageFactory.h>
|
||||||
#include <Storages/checkAndGetLiteralArgument.h>
|
#include <Storages/checkAndGetLiteralArgument.h>
|
||||||
|
#include <Storages/SelectQueryInfo.h>
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
@ -30,12 +31,9 @@
|
|||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
#include <Common/randomSeed.h>
|
#include <Common/randomSeed.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <base/unaligned.h>
|
|
||||||
|
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
|
||||||
#include <pcg_random.hpp>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -639,7 +637,7 @@ void registerStorageGenerateRandom(StorageFactory & factory)
|
|||||||
Pipe StorageGenerateRandom::read(
|
Pipe StorageGenerateRandom::read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
const StorageSnapshotPtr & storage_snapshot,
|
const StorageSnapshotPtr & storage_snapshot,
|
||||||
SelectQueryInfo & /*query_info*/,
|
SelectQueryInfo & query_info,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
QueryProcessingStage::Enum /*processed_stage*/,
|
QueryProcessingStage::Enum /*processed_stage*/,
|
||||||
size_t max_block_size,
|
size_t max_block_size,
|
||||||
@ -682,7 +680,14 @@ Pipe StorageGenerateRandom::read(
|
|||||||
pcg64 generate(random_seed);
|
pcg64 generate(random_seed);
|
||||||
|
|
||||||
for (UInt64 i = 0; i < num_streams; ++i)
|
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));
|
return Pipe::unitePipes(std::move(pipes));
|
||||||
}
|
}
|
||||||
|
@ -1,17 +1,14 @@
|
|||||||
#include <Storages/System/StorageSystemNumbers.h>
|
#include <Storages/System/StorageSystemNumbers.h>
|
||||||
|
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <Columns/ColumnsNumber.h>
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
#include <Processors/ISource.h>
|
|
||||||
#include <Processors/LimitTransform.h>
|
#include <Processors/LimitTransform.h>
|
||||||
#include <Processors/QueryPlan/QueryPlan.h>
|
#include <Processors/QueryPlan/QueryPlan.h>
|
||||||
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
|
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
|
||||||
#include <Processors/Sources/NullSource.h>
|
|
||||||
#include <QueryPipeline/Pipe.h>
|
|
||||||
#include <Storages/SelectQueryInfo.h>
|
#include <Storages/SelectQueryInfo.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Storages/System/StorageSystemZeros.h>
|
#include <Storages/System/StorageSystemZeros.h>
|
||||||
|
#include <Storages/SelectQueryInfo.h>
|
||||||
|
|
||||||
#include <Processors/ISource.h>
|
#include <Processors/ISource.h>
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
@ -6,6 +7,7 @@
|
|||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <Columns/ColumnsNumber.h>
|
#include <Columns/ColumnsNumber.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -93,7 +95,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith
|
|||||||
Pipe StorageSystemZeros::read(
|
Pipe StorageSystemZeros::read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
const StorageSnapshotPtr & storage_snapshot,
|
const StorageSnapshotPtr & storage_snapshot,
|
||||||
SelectQueryInfo &,
|
SelectQueryInfo & query_info,
|
||||||
ContextPtr /*context*/,
|
ContextPtr /*context*/,
|
||||||
QueryProcessingStage::Enum /*processed_stage*/,
|
QueryProcessingStage::Enum /*processed_stage*/,
|
||||||
size_t max_block_size,
|
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);
|
auto source = std::make_shared<ZerosSource>(max_block_size, limit ? *limit : 0, state);
|
||||||
|
|
||||||
if (limit && i == 0)
|
if (i == 0)
|
||||||
source->addTotalRowsApprox(*limit);
|
{
|
||||||
|
if (limit)
|
||||||
|
source->addTotalRowsApprox(*limit);
|
||||||
|
else if (query_info.limit)
|
||||||
|
source->addTotalRowsApprox(query_info.limit);
|
||||||
|
}
|
||||||
|
|
||||||
res.addSource(std::move(source));
|
res.addSource(std::move(source));
|
||||||
}
|
}
|
||||||
|
@ -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
|
@ -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 }
|
Loading…
Reference in New Issue
Block a user