mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Adjust some tests
This commit is contained in:
parent
e50e28ac62
commit
b42069cfa8
@ -1,10 +1,10 @@
|
|||||||
-- Tags: distributed
|
-- Tags: distributed
|
||||||
|
|
||||||
SET max_memory_usage = 300000000;
|
SET max_memory_usage = 150000000;
|
||||||
SET max_bytes_before_external_sort = 20000000;
|
SET max_bytes_before_external_sort = 10000000;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS numbers10m;
|
DROP TABLE IF EXISTS numbers10m;
|
||||||
CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 10000000;
|
CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 5000000;
|
||||||
|
|
||||||
SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20;
|
SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20;
|
||||||
|
|
||||||
|
@ -74,7 +74,7 @@ ${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE t"
|
|||||||
|
|
||||||
echo "A session cannot be used by concurrent connections:"
|
echo "A session cannot be used by concurrent connections:"
|
||||||
|
|
||||||
${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9&query_id=${CLICKHOUSE_DATABASE}_9" --data-binary "SELECT count() FROM system.numbers" >/dev/null &
|
${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&session_id=${CLICKHOUSE_DATABASE}_9&query_id=${CLICKHOUSE_DATABASE}_9&max_rows_to_read=0" --data-binary "SELECT count() FROM system.numbers" >/dev/null &
|
||||||
|
|
||||||
# An infinite loop is required to make the test reliable. We will ensure that at least once the query on the line above has started before this check
|
# An infinite loop is required to make the test reliable. We will ensure that at least once the query on the line above has started before this check
|
||||||
while true
|
while true
|
||||||
|
@ -11,7 +11,7 @@ function wait_for_query_to_start()
|
|||||||
while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done
|
while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done
|
||||||
}
|
}
|
||||||
|
|
||||||
${CLICKHOUSE_CURL_COMMAND} -q --max-time 30 -sS "$CLICKHOUSE_URL&query_id=test_00601_$CLICKHOUSE_DATABASE" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k)' > /dev/null &
|
${CLICKHOUSE_CURL_COMMAND} -q --max-time 30 -sS "$CLICKHOUSE_URL&query_id=test_00601_$CLICKHOUSE_DATABASE" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(50000000) GROUP BY k) SETTINGS max_rows_to_read = 0' > /dev/null &
|
||||||
wait_for_query_to_start "test_00601_$CLICKHOUSE_DATABASE"
|
wait_for_query_to_start "test_00601_$CLICKHOUSE_DATABASE"
|
||||||
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'test_00601_$CLICKHOUSE_DATABASE'"
|
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'test_00601_$CLICKHOUSE_DATABASE'"
|
||||||
wait
|
wait
|
||||||
|
@ -1,2 +1,2 @@
|
|||||||
SET max_execution_speed = 1, max_execution_time = 3;
|
SET max_execution_speed = 1, max_execution_time = 3, max_rows_to_read = 0;
|
||||||
SELECT count() FROM system.numbers; -- { serverError TIMEOUT_EXCEEDED }
|
SELECT count() FROM system.numbers; -- { serverError TIMEOUT_EXCEEDED }
|
||||||
|
@ -3,6 +3,7 @@ create table t(n int, a Int64, s String) engine = MergeTree() order by a;
|
|||||||
|
|
||||||
set enable_positional_arguments = 0;
|
set enable_positional_arguments = 0;
|
||||||
set optimize_trivial_insert_select = 1;
|
set optimize_trivial_insert_select = 1;
|
||||||
|
set max_rows_to_read = 0;
|
||||||
|
|
||||||
-- due to aggregate functions, optimize_trivial_insert_select will not be applied
|
-- due to aggregate functions, optimize_trivial_insert_select will not be applied
|
||||||
insert into t select 1, sum(number) as c, getSetting('max_threads') from numbers_mt(100000000) settings max_insert_threads=4, max_threads=2;
|
insert into t select 1, sum(number) as c, getSetting('max_threads') from numbers_mt(100000000) settings max_insert_threads=4, max_threads=2;
|
||||||
|
@ -9,3 +9,4 @@ FROM
|
|||||||
)
|
)
|
||||||
WHERE number = 1
|
WHERE number = 1
|
||||||
LIMIT 1
|
LIMIT 1
|
||||||
|
SETTINGS max_rows_to_read = 0;
|
||||||
|
@ -14,10 +14,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
|
|
||||||
function test()
|
function test()
|
||||||
{
|
{
|
||||||
timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --query "
|
timeout 5 ${CLICKHOUSE_LOCAL} --max_execution_time 10 --max_rows_to_read 0 --query "
|
||||||
SELECT DISTINCT number % 5 FROM system.numbers" ||:
|
SELECT DISTINCT number % 5 FROM system.numbers" ||:
|
||||||
echo -e '---'
|
echo -e '---'
|
||||||
timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10" --data-binary "
|
timeout 5 ${CLICKHOUSE_CURL} -sS --no-buffer "${CLICKHOUSE_URL}&max_execution_time=10&max_rows_to_read=0" --data-binary "
|
||||||
SELECT DISTINCT number % 5 FROM system.numbers" ||:
|
SELECT DISTINCT number % 5 FROM system.numbers" ||:
|
||||||
echo -e '---'
|
echo -e '---'
|
||||||
}
|
}
|
||||||
|
@ -5,5 +5,6 @@ connect_timeout_with_failover_secure_ms Milliseconds 3000
|
|||||||
external_storage_connect_timeout_sec UInt64 10
|
external_storage_connect_timeout_sec UInt64 10
|
||||||
s3_connect_timeout_ms UInt64 1000
|
s3_connect_timeout_ms UInt64 1000
|
||||||
filesystem_prefetch_max_memory_usage UInt64 1073741824
|
filesystem_prefetch_max_memory_usage UInt64 1073741824
|
||||||
|
max_memory_usage UInt64 5000000000
|
||||||
max_untracked_memory UInt64 1048576
|
max_untracked_memory UInt64 1048576
|
||||||
memory_profiler_step UInt64 1048576
|
memory_profiler_step UInt64 1048576
|
||||||
|
@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
start=$SECONDS
|
start=$SECONDS
|
||||||
# If the memory leak exists, it will lead to OOM fairly quickly.
|
# If the memory leak exists, it will lead to OOM fairly quickly.
|
||||||
for _ in {1..1000}; do
|
for _ in {1..1000}; do
|
||||||
$CLICKHOUSE_CLIENT --max_memory_usage 1G <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10";
|
$CLICKHOUSE_CLIENT --max_memory_usage 1G --max_rows_to_read 0 <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10";
|
||||||
|
|
||||||
# NOTE: we cannot use timeout here since this will not guarantee that the query will be executed at least once.
|
# NOTE: we cannot use timeout here since this will not guarantee that the query will be executed at least once.
|
||||||
# (since graceful wait of clickhouse-client had been reverted)
|
# (since graceful wait of clickhouse-client had been reverted)
|
||||||
|
@ -9,7 +9,7 @@ create table t (x UInt64, s String) engine = MergeTree order by x SETTINGS index
|
|||||||
INSERT INTO t SELECT
|
INSERT INTO t SELECT
|
||||||
number,
|
number,
|
||||||
if(number < (8129 * 1024), arrayStringConcat(arrayMap(x -> toString(x), range(number % 128)), ' '), '')
|
if(number < (8129 * 1024), arrayStringConcat(arrayMap(x -> toString(x), range(number % 128)), ' '), '')
|
||||||
FROM numbers_mt((8129 * 1024) * 3) settings max_insert_threads=8;
|
FROM numbers_mt((8129 * 1024) * 3) settings max_insert_threads=8, max_rows_to_read=0;
|
||||||
|
|
||||||
-- optimize table t final;
|
-- optimize table t final;
|
||||||
|
|
||||||
|
@ -2,4 +2,3 @@
|
|||||||
0.009775171065493644
|
0.009775171065493644
|
||||||
0.009775171065493644
|
0.009775171065493644
|
||||||
0.009775171065493644
|
0.009775171065493644
|
||||||
0.009775171065493644
|
|
||||||
|
@ -3,4 +3,3 @@ WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)
|
|||||||
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000));
|
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000));
|
||||||
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(100000));
|
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(100000));
|
||||||
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(1000000));
|
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(1000000));
|
||||||
WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000000));
|
|
||||||
|
@ -10,7 +10,7 @@ function wait_for_query_to_start()
|
|||||||
}
|
}
|
||||||
|
|
||||||
QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1"
|
QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1"
|
||||||
(${CLICKHOUSE_CLIENT} --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." || echo 'FAIL') &
|
(${CLICKHOUSE_CLIENT} --max_rows_to_read 0 --query_id="${QUERY_1_ID}" --query='select (SELECT max(number) from system.numbers) + 1;' 2>&1 | grep -q "Code: 394." || echo 'FAIL') &
|
||||||
wait_for_query_to_start "${QUERY_1_ID}"
|
wait_for_query_to_start "${QUERY_1_ID}"
|
||||||
${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC"
|
${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC"
|
||||||
|
|
||||||
|
@ -15,6 +15,8 @@ SELECT
|
|||||||
FROM system.numbers limit 500000));
|
FROM system.numbers limit 500000));
|
||||||
|
|
||||||
|
|
||||||
|
SET max_rows_to_read = 0;
|
||||||
|
|
||||||
SELECT roundBankers(result.1, 5), roundBankers(result.2, 5 ) FROM (
|
SELECT roundBankers(result.1, 5), roundBankers(result.2, 5 ) FROM (
|
||||||
SELECT
|
SELECT
|
||||||
studentTTest(sample, variant) as result
|
studentTTest(sample, variant) as result
|
||||||
|
@ -23,16 +23,16 @@ function check_output() {
|
|||||||
|
|
||||||
# TCP CLIENT
|
# TCP CLIENT
|
||||||
echo "TCP CLIENT"
|
echo "TCP CLIENT"
|
||||||
OUTPUT=$($CLICKHOUSE_CLIENT --max_execution_time $MAX_TIMEOUT -q "SELECT count() FROM system.numbers" 2>&1 || true)
|
OUTPUT=$($CLICKHOUSE_CLIENT --max_rows_to_read 0 --max_execution_time $MAX_TIMEOUT -q "SELECT count() FROM system.numbers" 2>&1 || true)
|
||||||
check_output "${OUTPUT}"
|
check_output "${OUTPUT}"
|
||||||
|
|
||||||
echo "TCP CLIENT WITH SETTINGS IN QUERY"
|
echo "TCP CLIENT WITH SETTINGS IN QUERY"
|
||||||
OUTPUT=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.numbers SETTINGS max_execution_time=$MAX_TIMEOUT" 2>&1 || true)
|
OUTPUT=$($CLICKHOUSE_CLIENT --max_rows_to_read 0 -q "SELECT count() FROM system.numbers SETTINGS max_execution_time=$MAX_TIMEOUT" 2>&1 || true)
|
||||||
check_output "${OUTPUT}"
|
check_output "${OUTPUT}"
|
||||||
|
|
||||||
# HTTP CLIENT
|
# HTTP CLIENT
|
||||||
echo "HTTP CLIENT"
|
echo "HTTP CLIENT"
|
||||||
OUTPUT=$(${CLICKHOUSE_CURL_COMMAND} -q -sS "$CLICKHOUSE_URL&max_execution_time=$MAX_TIMEOUT" -d \
|
OUTPUT=$(${CLICKHOUSE_CURL_COMMAND} -q -sS "$CLICKHOUSE_URL&max_execution_time=${MAX_TIMEOUT}&max_rows_to_read=0" -d \
|
||||||
"SELECT count() FROM system.numbers" || true)
|
"SELECT count() FROM system.numbers" || true)
|
||||||
check_output "${OUTPUT}"
|
check_output "${OUTPUT}"
|
||||||
|
|
||||||
|
@ -13,11 +13,9 @@ set allow_prefetched_read_pool_for_local_filesystem = 0;
|
|||||||
|
|
||||||
-- { echoOn }
|
-- { echoOn }
|
||||||
|
|
||||||
explain pipeline select * from (select * from numbers(1e8) group by number) group by number;
|
explain pipeline select * from (select * from numbers(1e8) group by number) group by number settings max_rows_to_read = 0;
|
||||||
|
explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number settings max_rows_to_read = 0;
|
||||||
explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number;
|
explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number settings max_rows_to_read = 0;
|
||||||
|
|
||||||
explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number;
|
|
||||||
|
|
||||||
explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 1;
|
explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 1;
|
||||||
|
|
||||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
|
|
||||||
QUERY_ID="${CLICKHOUSE_DATABASE}_read_with_cancel"
|
QUERY_ID="${CLICKHOUSE_DATABASE}_read_with_cancel"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -n --query_id="$QUERY_ID" --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true;" &
|
$CLICKHOUSE_CLIENT --max_rows_to_read 0 -n --query_id="$QUERY_ID" --query="SELECT sum(number * 0) FROM numbers(10000000000) SETTINGS partial_result_on_first_cancel=true;" &
|
||||||
pid=$!
|
pid=$!
|
||||||
|
|
||||||
for _ in {0..60}
|
for _ in {0..60}
|
||||||
|
@ -10,7 +10,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
# NOTE: .sh test is used over .sql because it needs $CLICKHOUSE_DATABASE to
|
# NOTE: .sh test is used over .sql because it needs $CLICKHOUSE_DATABASE to
|
||||||
# avoid truncation, since seems that the version of MinIO that is used on CI
|
# avoid truncation, since seems that the version of MinIO that is used on CI
|
||||||
# too slow with this.
|
# too slow with this.
|
||||||
$CLICKHOUSE_CLIENT -nm -q "
|
#
|
||||||
|
# Unfortunately, the test has to buffer it in memory.
|
||||||
|
$CLICKHOUSE_CLIENT --max_memory_usage 10G -nm -q "
|
||||||
INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV')
|
INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV')
|
||||||
SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024)
|
SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024)
|
||||||
SETTINGS s3_max_single_part_upload_size = '5Gi';
|
SETTINGS s3_max_single_part_upload_size = '5Gi';
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
-- Tags: no-fasttest
|
-- Tags: no-fasttest
|
||||||
|
|
||||||
|
SET max_rows_to_read = 0;
|
||||||
|
|
||||||
-- Query stops after timeout without an error
|
-- Query stops after timeout without an error
|
||||||
SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=2, timeout_overflow_mode='break' FORMAT Null;
|
SELECT * FROM numbers(100000000) SETTINGS max_block_size=1, max_execution_time=2, timeout_overflow_mode='break' FORMAT Null;
|
||||||
|
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
SELECT sleep(3.40282e+44); -- { serverError BAD_ARGUMENTS }
|
SELECT sleep(3.40282e+44); -- { serverError BAD_ARGUMENTS }
|
||||||
SELECT sleep((pow(2, 64) / 1000000) - 1); -- { serverError BAD_ARGUMENTS }
|
SELECT sleep((pow(2, 64) / 1000000) - 1); -- { serverError BAD_ARGUMENTS }
|
||||||
SELECT sleepEachRow(184467440737095516) from numbers(10000); -- { serverError BAD_ARGUMENTS }
|
SELECT sleepEachRow(184467440737095516) from numbers(10000); -- { serverError BAD_ARGUMENTS }
|
||||||
|
SET max_rows_to_read = 0;
|
||||||
SELECT sleepEachRow(pow(2, 31)) from numbers(9007199254740992) settings function_sleep_max_microseconds_per_block = 8589934592000000000; -- { serverError TOO_SLOW }
|
SELECT sleepEachRow(pow(2, 31)) from numbers(9007199254740992) settings function_sleep_max_microseconds_per_block = 8589934592000000000; -- { serverError TOO_SLOW }
|
||||||
|
|
||||||
-- Another corner case, but it requires lots of memory to run (huge block size)
|
-- Another corner case, but it requires lots of memory to run (huge block size)
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SET max_rows_to_read = 0;
|
||||||
create table test (number UInt64) engine=MergeTree order by number;
|
create table test (number UInt64) engine=MergeTree order by number;
|
||||||
insert into test select * from numbers(50000000);
|
insert into test select * from numbers(50000000);
|
||||||
select ignore(number) from test where RAND() > 4292390314 limit 10;
|
select ignore(number) from test where RAND() > 4292390314 limit 10;
|
||||||
|
Loading…
Reference in New Issue
Block a user