Merge pull request #10268 from ClickHouse/max-rows-to-sort

Added failing tests about "max_rows_to_sort" setting.
This commit is contained in:
alexey-milovidov 2020-05-17 22:21:30 +03:00 committed by GitHub
commit f7b1263d41
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 360 additions and 18 deletions

View File

@ -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
{ {

View File

@ -56,11 +56,6 @@ void ISource::work()
finished = true; finished = true;
throw; throw;
} }
// {
// current_chunk = std::current_exception();
// has_input = true;
// got_exception = true;
// }
} }
} }

View File

@ -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;

View File

@ -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;
} }

View File

@ -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.

View File

@ -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.

View File

@ -0,0 +1 @@
1

View 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);

View 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

View 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);

View 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

View 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));

View File

@ -0,0 +1,13 @@
1
0
1
0
---
1
0
1
0
1
0
1
0

View 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));