Fix some tests

This commit is contained in:
avogar 2022-02-11 17:15:56 +03:00
parent 33c3b2d856
commit 880780c7a8
32 changed files with 73 additions and 14 deletions

View File

@ -333,6 +333,7 @@ std::string LocalServer::getInitialCreateTableQuery()
/// Use regular file
auto file_name = config().getString("table-file");
table_file = quoteString(file_name);
std::cerr << file_name << "\n";
format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name, false);
}

View File

@ -176,7 +176,7 @@
- users without password have readonly access.
See also: https://www.shodan.io/search?query=clickhouse
-->
<!-- <listen_host>::</listen_host> -->
<listen_host>::</listen_host>
<!-- Same for hosts without support for IPv6: -->

View File

@ -118,6 +118,8 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
if (!create->is_dictionary)
create->attach = true;
// create->columns_list->c
/// We remove everything that is not needed for ATTACH from the query.
assert(!create->temporary);
create->database.reset();

View File

@ -446,9 +446,15 @@ class TestCase:
return testcase_args
def configure_test_settings(self, client_options):
if self.tags and 'no-random-settings' in self.tags:
return client_options
return client_options + " --" + " --".join(self.random_settings)
def add_info_about_settings(self, description):
if self.tags and 'no-random-settings' in self.tags:
return description
return description + "\n" + "Settings used in the test: " + "--" + " --".join(self.random_settings) + "\n"
def __init__(self, suite, case: str, args, is_concurrent: bool):

View File

@ -2,13 +2,11 @@
"value": 4611686018427387904
"name": "value",
"value": "4611686018427387904"
value
value
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
OK
OK
"name": "value",
"value": "9223372036854775808"
"name": "value",
"value": 9223372036854775808
value
value
Cannot modify 'output_format_json_quote_64bit_integers' setting in readonly mode
OK
OK

View File

@ -9,13 +9,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=0 | grep value
$CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --output_format_json_quote_64bit_integers=1 | grep value
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
#${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode.' && echo "OK" || echo "FAIL"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode.'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o 'value\|Cannot modify .* setting in readonly mode'

View File

@ -1,4 +1,5 @@
SET send_logs_level = 'fatal';
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);

View File

@ -1,4 +1,6 @@
SET enable_optimize_predicate_expression = 0;
SET optimize_move_to_prewhere = 1;
SET convert_query_to_cnf = 0;
select * from system.one l cross join system.one r;

View File

@ -1,4 +1,5 @@
SET enable_optimize_predicate_expression = 0;
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;

View File

@ -1,5 +1,7 @@
-- Tags: shard
SET prefer_localhost_replica = 1;
SELECT count() FROM remote('127.0.0.1,localhos', system.one); -- { serverError 198 }
SELECT count() FROM remote('127.0.0.1|localhos', system.one);

View File

@ -7,6 +7,8 @@
-- sizeof(HLL) is (2^K * 6 / 8)
-- hence max_memory_usage for 100 rows = (96<<10)*100 = 9830400
SET use_uncompressed_cache = 0;
-- HashTable for UInt32 (used until (1<<13) elements), hence 8192 elements
SELECT 'UInt32';
SET max_memory_usage = 4000000;
@ -19,6 +21,8 @@ SELECT 'UInt64';
SET max_memory_usage = 4000000;
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k); -- { serverError 241 }
SET max_memory_usage = 9830400;
SELECT sum(u) FROM (SELECT intDiv(number, 4096) AS k, uniqCombined(reinterpretAsString(number % 4096)) u FROM numbers(4096 * 100) GROUP BY k);
SELECT 'K=16';

View File

@ -1,5 +1,7 @@
-- Tags: distributed
set allow_experimental_parallel_reading_from_replicas = 0;
drop table if exists sample_final;
create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID);
insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000);

View File

@ -1,5 +1,6 @@
SET enable_optimize_predicate_expression = 1;
SET joined_subquery_requires_alias = 0;
SET convert_query_to_cnf = 0;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485

View File

@ -1,3 +1,5 @@
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS n;
DROP TABLE IF EXISTS r;

View File

@ -2,6 +2,8 @@
-- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971
SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525
DROP TABLE IF EXISTS local_01099_a;
DROP TABLE IF EXISTS local_01099_b;
DROP TABLE IF EXISTS distributed_01099_a;

View File

@ -1,5 +1,7 @@
-- Tags: no-parallel
SET prefer_localhost_replica = 1;
DROP DATABASE IF EXISTS test_01155_ordinary;
DROP DATABASE IF EXISTS test_01155_atomic;

View File

@ -1,4 +1,5 @@
set optimize_arithmetic_operations_in_aggregate_functions = 1;
SET convert_query_to_cnf = 0;
explain syntax select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0;
select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0;

View File

@ -1,3 +1,5 @@
-- Tags: no-random-settings
show settings like 'send_timeout';
SHOW SETTINGS ILIKE '%CONNECT_timeout%';
SHOW CHANGED SETTINGS ILIKE '%MEMORY%';

View File

@ -1,4 +1,5 @@
-- Tags: no-parallel
set prefer_localhost_replica = 1;
drop table if exists null_01293;
drop table if exists dist_01293;

View File

@ -1,5 +1,7 @@
-- Tags: distributed
SET allow_experimental_parallel_reading_from_replicas = 0;
DROP TABLE IF EXISTS test5346;
CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime)

View File

@ -1,5 +1,7 @@
-- Tags: replica
SET allow_experimental_parallel_reading_from_replicas=0;
DROP TABLE IF EXISTS t;
CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x;
INSERT INTO t VALUES ('Hello');

View File

@ -1,3 +1,6 @@
SET optimize_move_to_prewhere = 1;
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS prewhere_move;
CREATE TABLE prewhere_move (x Int, y String) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO prewhere_move SELECT number, toString(number) FROM numbers(1000);

View File

@ -1,3 +1,5 @@
SET use_uncompressed_cache = 0;
DROP TABLE IF EXISTS adaptive_table;
--- If granularity of consequent blocks differs a lot, then adaptive

View File

@ -1,3 +1,5 @@
SET group_by_two_level_threshold = 10000;
CREATE TABLE group_bitmap_data_test
(
`pickup_date` Date,

View File

@ -1,3 +1,6 @@
SET optimize_move_to_prewhere = 1;
SET convert_query_to_cnf = 1;
DROP TABLE IF EXISTS prewhere_move_select_final;
CREATE TABLE prewhere_move_select_final (x Int, y Int, z Int) ENGINE = ReplacingMergeTree() ORDER BY (x, y);

View File

@ -1,5 +1,7 @@
-- Tags: distributed
SET prefer_localhost_replica = 0;
DROP TABLE IF EXISTS tt6;
CREATE TABLE tt6
@ -13,6 +15,8 @@ CREATE TABLE tt6
)
ENGINE = Distributed('test_shard_localhost', '', 'tt7', rand());
DROP TABLE IF EXISTS tt7;
CREATE TABLE tt7 as tt6 ENGINE = Distributed('test_shard_localhost', '', 'tt6', rand());
INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 581 }
@ -28,3 +32,4 @@ INSERT INTO tt6 VALUES (1, 1, 1, 1, 'ok'); -- { serverError 306}
SELECT * FROM tt6; -- { serverError 306 }
DROP TABLE tt6;
DROP TABLE tt7;

View File

@ -1,3 +1,6 @@
SET optimize_move_to_prewhere = 1;
SET convert_query_to_cnf = 0;
DROP TABLE IF EXISTS t_move_to_prewhere;
CREATE TABLE t_move_to_prewhere (id UInt32, a UInt8, b UInt8, c UInt8, fat_string String)

View File

@ -1,6 +1,8 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: Different query_id
SET prefer_localhost_replica = 1;
DROP TABLE IF EXISTS tmp;
CREATE TABLE tmp ENGINE = TinyLog AS SELECT queryID();

View File

@ -5,6 +5,7 @@ CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=M
INSERT INTO test_tuple_filter VALUES (1,'A','2021-01-01'),(2,'B','2021-01-01'),(3,'C','2021-01-01'),(4,'D','2021-01-02'),(5,'E','2021-01-02');
SET force_primary_key = 1;
SET optimize_move_to_prewhere = 1;
SELECT * FROM test_tuple_filter WHERE (id, value) = (1, 'A');
SELECT * FROM test_tuple_filter WHERE (1, 'A') = (id, value);

View File

@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000));" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress"
$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress"

View File

@ -1,3 +1,5 @@
SET optimize_move_to_prewhere = 1;
DROP TABLE IF EXISTS t_02156_mt1;
DROP TABLE IF EXISTS t_02156_mt2;
DROP TABLE IF EXISTS t_02156_log;

View File

@ -2,7 +2,7 @@
DROP TABLE IF EXISTS test.avro;
SET max_threads = 1, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage
SET max_threads = 1, max_insert_threads = 0, max_block_size = 8192, min_insert_block_size_rows = 8192, min_insert_block_size_bytes = 1048576; -- lower memory usage
CREATE TABLE test.avro AS test.hits ENGINE = File(Avro);
INSERT INTO test.avro SELECT * FROM test.hits LIMIT 10000;