mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #10268 from ClickHouse/max-rows-to-sort
Added failing tests about "max_rows_to_sort" setting.
This commit is contained in:
commit
f7b1263d41
@ -2067,10 +2067,9 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
|
|||||||
|
|
||||||
const Settings & settings = context->getSettingsRef();
|
const Settings & settings = context->getSettingsRef();
|
||||||
|
|
||||||
/// TODO: Limits on sorting
|
IBlockInputStream::LocalLimits limits;
|
||||||
// IBlockInputStream::LocalLimits limits;
|
limits.mode = IBlockInputStream::LIMITS_CURRENT;
|
||||||
// limits.mode = IBlockInputStream::LIMITS_TOTAL;
|
limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode);
|
||||||
// limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode);
|
|
||||||
|
|
||||||
if (input_sorting_info)
|
if (input_sorting_info)
|
||||||
{
|
{
|
||||||
@ -2107,6 +2106,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
|
|||||||
return std::make_shared<PartialSortingTransform>(header, output_order_descr, limit);
|
return std::make_shared<PartialSortingTransform>(header, output_order_descr, limit);
|
||||||
});
|
});
|
||||||
|
|
||||||
|
/// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform
|
||||||
|
|
||||||
pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr
|
pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr
|
||||||
{
|
{
|
||||||
return std::make_shared<FinishSortingTransform>(
|
return std::make_shared<FinishSortingTransform>(
|
||||||
@ -2126,6 +2127,15 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
|
|||||||
return std::make_shared<PartialSortingTransform>(header, output_order_descr, limit);
|
return std::make_shared<PartialSortingTransform>(header, output_order_descr, limit);
|
||||||
});
|
});
|
||||||
|
|
||||||
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||||
|
{
|
||||||
|
if (stream_type == QueryPipeline::StreamType::Totals)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
auto transform = std::make_shared<LimitsCheckingTransform>(header, limits);
|
||||||
|
return transform;
|
||||||
|
});
|
||||||
|
|
||||||
/// Merge the sorted blocks.
|
/// Merge the sorted blocks.
|
||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||||
{
|
{
|
||||||
|
@ -56,11 +56,6 @@ void ISource::work()
|
|||||||
finished = true;
|
finished = true;
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
// {
|
|
||||||
// current_chunk = std::current_exception();
|
|
||||||
// has_input = true;
|
|
||||||
// got_exception = true;
|
|
||||||
// }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -15,7 +15,9 @@ namespace ErrorCodes
|
|||||||
void SourceWithProgress::work()
|
void SourceWithProgress::work()
|
||||||
{
|
{
|
||||||
if (!limits.speed_limits.checkTimeLimit(total_stopwatch.elapsed(), limits.timeout_overflow_mode))
|
if (!limits.speed_limits.checkTimeLimit(total_stopwatch.elapsed(), limits.timeout_overflow_mode))
|
||||||
|
{
|
||||||
cancel();
|
cancel();
|
||||||
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
was_progress_called = false;
|
was_progress_called = false;
|
||||||
@ -57,7 +59,13 @@ void SourceWithProgress::progress(const Progress & value)
|
|||||||
/// The total amount of data processed or intended for processing in all sources, possibly on remote servers.
|
/// The total amount of data processed or intended for processing in all sources, possibly on remote servers.
|
||||||
|
|
||||||
ProgressValues progress = process_list_elem->getProgressIn();
|
ProgressValues progress = process_list_elem->getProgressIn();
|
||||||
size_t total_rows_estimate = std::max(progress.read_rows, progress.total_rows_to_read);
|
|
||||||
|
/// If the mode is "throw" and estimate of total rows is known, then throw early if an estimate is too high.
|
||||||
|
/// If the mode is "break", then allow to read before limit even if estimate is very high.
|
||||||
|
|
||||||
|
size_t rows_to_check_limit = progress.read_rows;
|
||||||
|
if (limits.size_limits.overflow_mode == OverflowMode::THROW && progress.total_rows_to_read > progress.read_rows)
|
||||||
|
rows_to_check_limit = progress.total_rows_to_read;
|
||||||
|
|
||||||
/// Check the restrictions on the
|
/// Check the restrictions on the
|
||||||
/// * amount of data to read
|
/// * amount of data to read
|
||||||
@ -67,10 +75,12 @@ void SourceWithProgress::progress(const Progress & value)
|
|||||||
|
|
||||||
if (limits.mode == LimitsMode::LIMITS_TOTAL)
|
if (limits.mode == LimitsMode::LIMITS_TOTAL)
|
||||||
{
|
{
|
||||||
if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read",
|
if (!limits.size_limits.check(rows_to_check_limit, progress.read_bytes, "rows or bytes to read",
|
||||||
ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES))
|
ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES))
|
||||||
|
{
|
||||||
cancel();
|
cancel();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
size_t total_rows = progress.total_rows_to_read;
|
size_t total_rows = progress.total_rows_to_read;
|
||||||
|
|
||||||
|
@ -44,7 +44,9 @@ void LimitsCheckingTransform::transform(Chunk & chunk)
|
|||||||
|
|
||||||
if (limits.mode == LimitsMode::LIMITS_CURRENT &&
|
if (limits.mode == LimitsMode::LIMITS_CURRENT &&
|
||||||
!limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES))
|
!limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES))
|
||||||
|
{
|
||||||
stopReading();
|
stopReading();
|
||||||
|
}
|
||||||
|
|
||||||
if (quota)
|
if (quota)
|
||||||
checkQuota(chunk);
|
checkQuota(chunk);
|
||||||
@ -56,13 +58,16 @@ void LimitsCheckingTransform::checkQuota(Chunk & chunk)
|
|||||||
switch (limits.mode)
|
switch (limits.mode)
|
||||||
{
|
{
|
||||||
case LimitsMode::LIMITS_TOTAL:
|
case LimitsMode::LIMITS_TOTAL:
|
||||||
/// Checked in `progress` method.
|
/// Checked in SourceWithProgress::progress method.
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case LimitsMode::LIMITS_CURRENT:
|
case LimitsMode::LIMITS_CURRENT:
|
||||||
{
|
{
|
||||||
UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds();
|
UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds();
|
||||||
quota->used({Quota::RESULT_ROWS, chunk.getNumRows()}, {Quota::RESULT_BYTES, chunk.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed});
|
quota->used(
|
||||||
|
{Quota::RESULT_ROWS, chunk.getNumRows()},
|
||||||
|
{Quota::RESULT_BYTES, chunk.bytes()},
|
||||||
|
{Quota::EXECUTION_TIME, total_elapsed - prev_elapsed});
|
||||||
prev_elapsed = total_elapsed;
|
prev_elapsed = total_elapsed;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -38,6 +38,9 @@ private:
|
|||||||
VolumePtr tmp_volume;
|
VolumePtr tmp_volume;
|
||||||
size_t min_free_disk_space;
|
size_t min_free_disk_space;
|
||||||
|
|
||||||
|
size_t sum_rows_in_blocks = 0;
|
||||||
|
size_t sum_bytes_in_blocks = 0;
|
||||||
|
|
||||||
Logger * log = &Logger::get("MergeSortingTransform");
|
Logger * log = &Logger::get("MergeSortingTransform");
|
||||||
|
|
||||||
/// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore.
|
/// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore.
|
||||||
|
@ -83,9 +83,6 @@ protected:
|
|||||||
size_t max_merged_block_size;
|
size_t max_merged_block_size;
|
||||||
UInt64 limit;
|
UInt64 limit;
|
||||||
|
|
||||||
size_t sum_rows_in_blocks = 0;
|
|
||||||
size_t sum_bytes_in_blocks = 0;
|
|
||||||
|
|
||||||
/// Before operation, will remove constant columns from blocks. And after, place constant columns back.
|
/// Before operation, will remove constant columns from blocks. And after, place constant columns back.
|
||||||
/// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files)
|
/// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files)
|
||||||
/// Save original block structure here.
|
/// Save original block structure here.
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
1
|
7
tests/queries/0_stateless/01131_max_rows_to_sort.sql
Normal file
7
tests/queries/0_stateless/01131_max_rows_to_sort.sql
Normal file
@ -0,0 +1,7 @@
|
|||||||
|
SET max_rows_to_sort = 100;
|
||||||
|
SELECT * FROM system.numbers ORDER BY number; -- { serverError 396 }
|
||||||
|
|
||||||
|
SET sort_overflow_mode = 'break';
|
||||||
|
SET max_block_size = 1000;
|
||||||
|
|
||||||
|
SELECT count() >= 100 AND count() <= 1000 FROM (SELECT * FROM system.numbers ORDER BY number);
|
57
tests/queries/0_stateless/01132_max_rows_to_read.reference
Normal file
57
tests/queries/0_stateless/01132_max_rows_to_read.reference
Normal file
@ -0,0 +1,57 @@
|
|||||||
|
19
|
||||||
|
20
|
||||||
|
19
|
||||||
|
20
|
||||||
|
20
|
||||||
|
20
|
||||||
|
20
|
||||||
|
20
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
30
tests/queries/0_stateless/01132_max_rows_to_read.sql
Normal file
30
tests/queries/0_stateless/01132_max_rows_to_read.sql
Normal file
@ -0,0 +1,30 @@
|
|||||||
|
SET max_block_size = 10;
|
||||||
|
SET max_rows_to_read = 20;
|
||||||
|
SET read_overflow_mode = 'throw';
|
||||||
|
|
||||||
|
SELECT count() FROM numbers(30); -- { serverError 158 }
|
||||||
|
SELECT count() FROM numbers(19);
|
||||||
|
SELECT count() FROM numbers(20);
|
||||||
|
SELECT count() FROM numbers(21); -- { serverError 158 }
|
||||||
|
|
||||||
|
-- check early exception if the estimated number of rows is high
|
||||||
|
SELECT * FROM numbers(30); -- { serverError 158 }
|
||||||
|
|
||||||
|
SET read_overflow_mode = 'break';
|
||||||
|
|
||||||
|
SELECT count() FROM numbers(19);
|
||||||
|
SELECT count() FROM numbers(20);
|
||||||
|
SELECT count() FROM numbers(21);
|
||||||
|
SELECT count() FROM numbers(29);
|
||||||
|
SELECT count() FROM numbers(30);
|
||||||
|
SELECT count() FROM numbers(31);
|
||||||
|
|
||||||
|
-- check that partial result is returned even if the estimated number of rows is high
|
||||||
|
SELECT * FROM numbers(30);
|
||||||
|
|
||||||
|
-- the same for uneven block sizes
|
||||||
|
-- NOTE: currently it outputs less amount of data; it will be better to output the latest block also
|
||||||
|
SET max_block_size = 11;
|
||||||
|
SELECT * FROM numbers(30);
|
||||||
|
SET max_block_size = 9;
|
||||||
|
SELECT * FROM numbers(30);
|
173
tests/queries/0_stateless/01133_max_result_rows.reference
Normal file
173
tests/queries/0_stateless/01133_max_result_rows.reference
Normal file
@ -0,0 +1,173 @@
|
|||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
20
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
10
|
||||||
|
11
|
||||||
|
12
|
||||||
|
13
|
||||||
|
14
|
||||||
|
15
|
||||||
|
16
|
||||||
|
17
|
||||||
|
18
|
||||||
|
19
|
||||||
|
20
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
5
|
||||||
|
6
|
||||||
|
7
|
||||||
|
8
|
||||||
|
9
|
||||||
|
100
|
||||||
|
100
|
||||||
|
200
|
24
tests/queries/0_stateless/01133_max_result_rows.sql
Normal file
24
tests/queries/0_stateless/01133_max_result_rows.sql
Normal file
@ -0,0 +1,24 @@
|
|||||||
|
SET max_block_size = 10;
|
||||||
|
SET max_result_rows = 20;
|
||||||
|
SET result_overflow_mode = 'throw';
|
||||||
|
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(300); -- { serverError 396 }
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(190);
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(200);
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(210); -- { serverError 396 }
|
||||||
|
|
||||||
|
SET result_overflow_mode = 'break';
|
||||||
|
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(300);
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(190);
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(200);
|
||||||
|
SELECT DISTINCT intDiv(number, 10) FROM numbers(210);
|
||||||
|
|
||||||
|
SET max_block_size = 10;
|
||||||
|
SET max_result_rows = 1;
|
||||||
|
SELECT number FROM system.numbers;
|
||||||
|
SELECT count() FROM numbers(100);
|
||||||
|
-- subquery result is not the total result
|
||||||
|
SELECT count() FROM (SELECT * FROM numbers(100));
|
||||||
|
-- remote query result is not the total result
|
||||||
|
SELECT count() FROM remote('127.0.0.{1,2}', numbers(100));
|
13
tests/queries/0_stateless/01134_set_overflow_mode.reference
Normal file
13
tests/queries/0_stateless/01134_set_overflow_mode.reference
Normal file
@ -0,0 +1,13 @@
|
|||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
---
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
0
|
17
tests/queries/0_stateless/01134_set_overflow_mode.sql
Normal file
17
tests/queries/0_stateless/01134_set_overflow_mode.sql
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
SET max_block_size = 10;
|
||||||
|
SET max_rows_in_set = 20;
|
||||||
|
SET set_overflow_mode = 'throw';
|
||||||
|
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(300)); -- { serverError 191 }
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(190));
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(200));
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(210)); -- { serverError 191 }
|
||||||
|
|
||||||
|
SET set_overflow_mode = 'break';
|
||||||
|
|
||||||
|
SELECT '---';
|
||||||
|
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(300));
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(190));
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(200));
|
||||||
|
SELECT arrayJoin([5, 25]) IN (SELECT DISTINCT toUInt8(intDiv(number, 10)) FROM numbers(210));
|
Loading…
Reference in New Issue
Block a user