More pytest fixes (#23538)

This commit is contained in:
Ivan 2021-06-07 20:29:19 +03:00 committed by GitHub
parent e01850c02b
commit 71f1a101c1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 94 additions and 10 deletions

2
contrib/avro vendored

@ -1 +1 @@
Subproject commit 1ee16d8c5a7808acff5cf0475f771195d9aa3faa
Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8

View File

@ -1,2 +0,0 @@
*.stderr
*.stdout

View File

@ -25,3 +25,5 @@ $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(y, x), sum(x - y) as s from test
$CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1"
$CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT -q "drop table test_agg_proj"

View File

@ -77,3 +77,4 @@ echo "optimize_move_to_prewhere = 1, allow_experimental_projection_optimization
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_sort_proj WHERE y > 4294967286 order by x FORMAT JSON
SETTINGS optimize_move_to_prewhere = 1, allow_experimental_projection_optimization = 1" | grep rows_read
$CLICKHOUSE_CLIENT -q "DROP TABLE test_sort_proj"

View File

@ -12,3 +12,6 @@ SELECT 1025 == count(n) FROM foo AS t1 ANY LEFT JOIN foo_lc AS t2 ON t1.n == t2.
SELECT 1025 == count(n) FROM foo_lc AS t1 ANY LEFT JOIN foo AS t2 ON t1.n == t2.n;
SELECT 1025 == count(n) FROM foo_lc AS t1 ALL LEFT JOIN foo_lc AS t2 ON t1.n == t2.n;
DROP TABLE foo;
DROP TABLE foo_lc;

View File

@ -41,3 +41,5 @@ SELECT 'after row policy with plaintext_password';
psql "postgresql://postgresql_user:qwerty@localhost:${CLICKHOUSE_PORT_POSTGRESQL}/db01802" -c "SELECT * FROM postgresql;"
$CLICKHOUSE_CLIENT -q "DROP TABLE db01802.postgresql"
$CLICKHOUSE_CLIENT -q "DROP DATABASE db01802"

View File

@ -5,3 +5,5 @@ drop table if exists data_01832;
create table data_01832 (key Int) Engine=Memory;
insert into data_01832 values (1);
select * from data_01832;
drop table data_01832;

View File

@ -24,3 +24,6 @@ LAYOUT(COMPLEX_KEY_DIRECT());
SELECT 'complex key';
SELECT name, key FROM system.dictionaries WHERE name='example_complex_key_dictionary' AND database=currentDatabase();
DROP DICTIONARY example_complex_key_dictionary;
DROP DICTIONARY example_simple_key_dictionary;

View File

@ -28,3 +28,5 @@ CREATE TABLE test_null_as_default (a Int8, b Int64 DEFAULT c - 500, c Int32 DEFA
INSERT INTO test_null_as_default(a, c) SELECT 1, NULL UNION ALL SELECT 2, NULL;
SELECT * FROM test_null_as_default ORDER BY a;
DROP TABLE test_null_as_default;

View File

@ -4,5 +4,4 @@
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | curl -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||:
${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | ${CLICKHOUSE_CURL} -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||:

View File

@ -13,3 +13,6 @@ insert into dist_01850 values (1); -- { serverError 60 }
drop table if exists dist_01850;
drop table shard_0.data_01850;
drop database shard_0;
drop database shard_1;

View File

@ -307,3 +307,4 @@ SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabas
DROP TABLE polygons_01862;
DROP TABLE points_01862;
DROP DICTIONARY polygon_dictionary_01862;

View File

@ -14,3 +14,6 @@ INSERT INTO test_jit_nullable VALUES (0), (1), (NULL);
SELECT 'test_jit_nullable';
SELECT value, multiIf(value = 1, 2, value, 1, 0), if (value, 1, 0) FROM test_jit_nullable;
DROP TABLE test_jit_nonnull;
DROP TABLE test_jit_nullable;

View File

@ -36,6 +36,7 @@ function run_test_once()
$CLICKHOUSE_CLIENT -nm -q "
DROP DICTIONARY simple_key_cache_dictionary_01863;
DROP TABLE simple_key_source_table_01863;
"
if [ "$prev" == "$curr" ]; then

View File

@ -5,3 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
python3 "$CURDIR"/01854_HTTP_dict_decompression.python
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY test_table_select"

View File

@ -38,3 +38,5 @@ SELECT 'dt64 != const dt' FROM dt64test WHERE dt64_column != toDateTime('2020-01
SELECT 'dt64 != dt' FROM dt64test WHERE dt64_column != materialize(toDateTime('2020-01-13 13:37:00'));
SELECT 'dt != const dt64' FROM dt64test WHERE dt_column != toDateTime64('2020-01-13 13:37:00', 3);
SELECT 'dt != dt64' FROM dt64test WHERE dt_column != materialize(toDateTime64('2020-01-13 13:37:00', 3));
DROP TABLE dt64test;

View File

@ -48,3 +48,7 @@ SELECT count() FROM table4 WHERE id % 10 = 7;
SELECT 'comparison:';
SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v;
DROP TABLE table1;
DROP TABLE table2;
DROP TABLE table3;
DROP TABLE table4;

View File

@ -14,3 +14,6 @@ select
/
(select count() from nums_in_mem_dist where rand() > 0)
from system.one; -- { serverError 158 }
drop table nums_in_mem;
drop table nums_in_mem_dist;

View File

@ -51,3 +51,5 @@ check_background_query &
$CLICKHOUSE_CLIENT -q "select *, sleepEachRow(1) from data_01882" --max_threads=1 --format Null --query_id="$QUERY_ID" --max_block_size=1
wait
$CLICKHOUSE_CLIENT -q "drop table data_01882"

View File

@ -1,18 +1,35 @@
import pytest
import os
import sys
import tempfile
import re
import pytest
from .server import ServerThread
# Command-line arguments
def pytest_addoption(parser):
parser.addoption(
"--builddir", action="store", default=None, help="Path to build directory to use binaries from",
)
# HTML report hooks
def pytest_html_report_title(report):
report.title = "ClickHouse Functional Stateless Tests (PyTest)"
RE_TEST_NAME = re.compile(r"\[(.*)\]")
def pytest_itemcollected(item):
match = RE_TEST_NAME.search(item.name)
if match:
item._nodeid = match.group(1)
# Fixtures
@pytest.fixture(scope='module')
def cmdopts(request):
return {

2
tests/queries/pytest.ini Normal file
View File

@ -0,0 +1,2 @@
[pytest]
render_collapsed = True

View File

@ -15,6 +15,7 @@ SKIP_LIST = [
# just fail
"00133_long_shard_memory_tracker_and_exception_safety",
"00463_long_sessions_in_http_interface",
"00505_secure",
"00505_shard_secure",
"00646_url_engine",
@ -56,8 +57,10 @@ SKIP_LIST = [
"01098_temporary_and_external_tables",
"01099_parallel_distributed_insert_select",
"01103_check_cpu_instructions_at_startup",
"01107_atomic_db_detach_attach",
"01114_database_atomic",
"01148_zookeeper_path_macros_unfolding",
"01152_cross_replication", # tcp port in reference
"01175_distributed_ddl_output_mode_long",
"01181_db_atomic_drop_on_cluster", # tcp port in reference
"01280_ssd_complex_key_dictionary",
@ -99,19 +102,30 @@ SKIP_LIST = [
"01601_custom_tld",
"01606_git_import",
"01610_client_spawn_editor", # expect-test
"01654_test_writer_block_sequence", # No module named 'pandas'
"01658_read_file_to_stringcolumn",
"01666_merge_tree_max_query_limit",
"01674_unicode_asan",
"01676_clickhouse_client_autocomplete", # expect-test (partially)
"01676_long_clickhouse_client_autocomplete",
"01683_text_log_deadlock", # secure tcp
"01684_ssd_cache_dictionary_simple_key",
"01685_ssd_cache_dictionary_complex_key",
"01737_clickhouse_server_wait_server_pool_long",
"01746_executable_pool_dictionary",
"01747_executable_pool_dictionary_implicit_key",
"01747_join_view_filter_dictionary",
"01748_dictionary_table_dot",
"01754_cluster_all_replicas_shard_num",
"01759_optimize_skip_unused_shards_zero_shards",
"01763_max_distributed_depth", # BROKEN
"01780_clickhouse_dictionary_source_loop",
"01801_s3_cluster",
"01802_test_postgresql_protocol_with_row_policy",
"01804_dictionary_decimal256_type", # hardcoded path
"01848_http_insert_segfault",
"01875_ssd_cache_dictionary_decimal256_type",
"01880_remote_ipv6",
]
@ -154,6 +168,7 @@ def run_shell(bin_prefix, server, database, path, reference, replace_map=None):
'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port),
'CLICKHOUSE_PORT_HTTP': str(server.http_port),
'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port),
'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port),
'CLICKHOUSE_TMP': server.tmp_dir,
'CLICKHOUSE_CONFIG_CLIENT': server.client_config
}

View File

@ -38,6 +38,7 @@ class ServerThread(threading.Thread):
self.https_port = port_base + 5
self.odbc_port = port_base + 6
self.proxy_port = port_base + 7
self.postgresql_port = port_base + 8
self._args = [
'--config-file={config_path}'.format(config_path=self.server_config),
@ -46,6 +47,7 @@ class ServerThread(threading.Thread):
'--http_port={http_port}'.format(http_port=self.http_port),
'--interserver_http_port={inter_port}'.format(inter_port=self.inter_port),
'--tcp_with_proxy_port={proxy_port}'.format(proxy_port=self.proxy_port),
'--postgresql_port={psql_port}'.format(psql_port=self.postgresql_port),
# TODO: SSL certificate is not specified '--tcp_port_secure={tcps_port}'.format(tcps_port=self.tcps_port),
]
@ -75,7 +77,7 @@ class ServerThread(threading.Thread):
time.sleep(ServerThread.DEFAULT_SERVER_DELAY)
s = socket.create_connection(('localhost', self.tcp_port), ServerThread.DEFAULT_CONNECTION_TIMEOUT)
s.sendall(b'G') # trigger expected "bad" HELLO response
print('Successful server response:', s.recv(1024)) # FIXME: read whole buffered response
s.recv(1024) # FIXME: read whole buffered response
s.shutdown(socket.SHUT_RDWR)
s.close()
except Exception:
@ -116,7 +118,7 @@ class ServerThread(threading.Thread):
if self._proc.returncode is None:
self._proc.terminate()
self.join()
print('Stop clickhouse-server')
print('Stopped clickhouse-server')
ServerThread.DEFAULT_SERVER_CONFIG = \
@ -289,6 +291,21 @@ ServerThread.DEFAULT_SERVER_CONFIG = \
</replica>
</shard>
</test_cluster_with_incorrect_pw>
<test_cluster_two_replicas_different_databases>
<shard>
<replica>
<default_database>shard_0</default_database>
<host>localhost</host>
<port>{tcp_port}</port>
</replica>
<replica>
<default_database>shard_1</default_database>
<host>localhost</host>
<port>{tcp_port}</port>
</replica>
</shard>
</test_cluster_two_replicas_different_databases>
</remote_servers>
<storage_configuration>