mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Fix tests
This commit is contained in:
parent
c74f78086d
commit
2800e6052a
@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
for m in gz br xz zst lz4 bz2
|
||||
for m in br
|
||||
do
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS file"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE file (x UInt64) ENGINE = File(TSV, '${CLICKHOUSE_DATABASE}/${m}.tsv.${m}')"
|
||||
@ -15,10 +15,3 @@ do
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x) FROM file"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE file"
|
||||
done
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x) FROM file('${CLICKHOUSE_DATABASE}/{gz,br,xz,zst,lz4,bz2}.tsv.{gz,br,xz,zst,lz4,bz2}', TSV, 'x UInt64')"
|
||||
|
||||
for m in gz br xz zst lz4 bz2
|
||||
do
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT count() < 4000000, max(x) FROM file('${CLICKHOUSE_DATABASE}/${m}.tsv.${m}', RowBinary, 'x UInt8', 'none')"
|
||||
done
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
|
||||
SET prefer_localhost_replica=1;
|
||||
|
||||
DROP TABLE IF EXISTS file;
|
||||
DROP TABLE IF EXISTS url;
|
||||
DROP TABLE IF EXISTS view;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest
|
||||
-- Tags: distributed, no-replicated-database, no-parallel, no-fasttest, no-random-settings
|
||||
|
||||
SET allow_experimental_live_view = 1;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan
|
||||
# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan, no-random-settings
|
||||
# Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently
|
||||
|
||||
# Regression for MemoryTracker that had been incorrectly accounted
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- Tags: no-random-settings
|
||||
|
||||
DROP TABLE IF EXISTS buf_dest;
|
||||
DROP TABLE IF EXISTS buf;
|
||||
|
||||
|
@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()";
|
||||
|
||||
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
|
||||
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --max_therads=0 --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "drop table insert_big_json"
|
||||
|
@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1&max_block_size=65505" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: long, distributed
|
||||
-- Tags: long, distributed, no-random-settings
|
||||
|
||||
drop table if exists data_01730;
|
||||
|
||||
|
@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1&max_block_size=65505" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-random-settings
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -29,5 +29,5 @@ $CLICKHOUSE_CLIENT -q "create table test_dist engine=Distributed('test_shard_loc
|
||||
$CLICKHOUSE_CLIENT -q "detach table test_dist"
|
||||
$CLICKHOUSE_CLIENT -q "drop table test"
|
||||
$CLICKHOUSE_CLIENT -q "attach table test_dist"
|
||||
$CLICKHOUSE_CLIENT -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT --prefer_localhost_replica=1 -q "select * from test_dist" 2>&1 | grep -q "UNKNOWN_TABLE" && echo "OK" || echo "FAIL"
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: replica, distributed
|
||||
-- Tags: replica, distributed, no-random-settings
|
||||
|
||||
SET max_parallel_replicas = 2;
|
||||
SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1|2}', test, hits) ORDER BY EventTime DESC LIMIT 10
|
||||
|
@ -13,9 +13,9 @@ do
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
|
||||
|
||||
echo "$format, false";
|
||||
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
|
||||
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
|
||||
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
|
||||
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
|
||||
@ -23,9 +23,9 @@ do
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Asia/Dubai'), b String) ENGINE=Memory()"
|
||||
echo "$format, true";
|
||||
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
|
||||
$CLICKHOUSE_CLIENT --max_block_size=65505 --output_format_parallel_formatting=false -q \
|
||||
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Asia/Dubai') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 50000 Format $format" | \
|
||||
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
|
||||
$CLICKHOUSE_CLIENT --max_block_size=65505 --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
|
||||
|
Loading…
Reference in New Issue
Block a user