mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Trying to make some tests parallel
This commit is contained in:
parent
e105a6e9bf
commit
91e973350f
@ -1,6 +1,5 @@
|
||||
-- Tags: long, zookeeper, no-replicated-database, no-parallel
|
||||
-- Tags: long, zookeeper, no-replicated-database
|
||||
-- Tag no-replicated-database: Old syntax is not allowed
|
||||
-- Tag no-parallel: leftovers
|
||||
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: shard, no-parallel, no-fasttest
|
||||
-- Tags: shard, no-fasttest
|
||||
|
||||
SET max_rows_to_group_by = 100000;
|
||||
SET max_block_size = 100001;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS set;
|
||||
DROP TABLE IF EXISTS set2;
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS t_00180;
|
||||
DROP TABLE IF EXISTS mv_00180;
|
||||
DROP TABLE IF EXISTS `.inner.mv_00180`;
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
DROP TABLE IF EXISTS array_pk;
|
||||
CREATE TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel, no-fasttest
|
||||
# Tags: no-fasttest
|
||||
|
||||
set -e
|
||||
|
||||
@ -11,11 +11,11 @@ QUERY_FIELND_NUM=4
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 4" &>/dev/null &
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 5" &>/dev/null &
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query = 'SELECT sleep(1) FROM system.numbers LIMIT 5' ASYNC" | cut -f $QUERY_FIELND_NUM
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query = 'SELECT sleep(1) FROM system.numbers LIMIT 5' ASYNC" | cut -f $QUERY_FIELND_NUM
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 ASYNC"
|
||||
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated"
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS log;
|
||||
CREATE TABLE log (s String) ENGINE = Log;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-replicated-database, no-parallel
|
||||
# Tags: zookeeper, no-replicated-database
|
||||
# Tag no-replicated-database: Old syntax is not allowed
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: shard, no-parallel, no-fasttest
|
||||
# Tags: shard, no-fasttest
|
||||
|
||||
# set -x
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
-- Tags: zookeeper, no-parallel
|
||||
-- Tag no-parallel: leftovers
|
||||
-- Tags: zookeeper
|
||||
|
||||
SET optimize_on_insert = 0;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-tsan, no-parallel, no-debug
|
||||
# Tags: no-tsan, no-debug
|
||||
# Tag no-tsan: Too long for TSan
|
||||
|
||||
# shellcheck disable=SC2016
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SELECT
|
||||
0 OR NULL,
|
||||
1 OR NULL,
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
set optimize_on_insert = 0;
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS test_00599;
|
||||
DROP TABLE IF EXISTS test_view_00599;
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS union;
|
||||
|
||||
create view union as select 1 as test union all select 2;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
|
||||
DROP DATABASE IF EXISTS truncate_test;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS nested;
|
||||
CREATE TABLE nested
|
||||
(
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS src;
|
||||
DROP TABLE IF EXISTS dst;
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-parallel, no-s3-storage, no-upgrade-check
|
||||
# Tags: zookeeper, no-s3-storage, no-upgrade-check
|
||||
|
||||
# Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem
|
||||
# (it tries to do it as quick as possible, but it still performed in separate thread asynchronously)
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-parallel
|
||||
# Tags: zookeeper
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica, no-replicated-database, no-parallel, no-fasttest
|
||||
# Tags: replica, no-replicated-database, no-fasttest
|
||||
# Tag no-replicated-database: Old syntax is not allowed
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel, no-fasttest
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
drop table if exists lc_dict_reading;
|
||||
create table lc_dict_reading (val UInt64, str StringWithDictionary, pat String) engine = MergeTree order by val;
|
||||
insert into lc_dict_reading select number, if(number < 8192 * 4, number % 100, number) as s, s from system.numbers limit 1000000;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
select 'NativeReader';
|
||||
select toTypeName(dict), dict, lowCardinalityIndices(dict), lowCardinalityKeys(dict) from (select '123_' || toLowCardinality(v) as dict from (select arrayJoin(['a', 'bb', '', 'a', 'ccc', 'a', 'bb', '', 'dddd']) as v));
|
||||
select '-';
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
-- Tags: no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS rollup;
|
||||
CREATE TABLE rollup(a String, b Int32, s Int32) ENGINE = Memory;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: zookeeper, no-parallel
|
||||
# Tags: zookeeper
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_ddl = 0;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
-- Tags: no-fasttest
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test_00741; -- foo
|
||||
DROP DATABASE test_00741;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS alter_attach;
|
||||
CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p;
|
||||
INSERT INTO alter_attach VALUES (1, 1), (2, 1), (3, 1);
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
-- check ALTER MODIFY COLUMN with partitions
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-parallel
|
||||
# Tags: long
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
DROP TABLE IF EXISTS alter_compression_codec;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_suspicious_codecs = 1;
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET allow_suspicious_codecs = 1;
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-debug, no-parallel
|
||||
# Tags: no-debug
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: replica, no-debug, no-parallel
|
||||
# Tags: replica, no-debug
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
----- Group of very similar simple tests ------
|
||||
select '----HORIZONTAL MERGE TESTS----';
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
SELECT '----00489----';
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
----- Group of very similar simple tests ------
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set send_logs_level = 'fatal';
|
||||
|
||||
drop table if exists ttl;
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS reserved_word_table;
|
||||
CREATE TABLE reserved_word_table (`index` UInt8) ENGINE = MergeTree ORDER BY `index`;
|
||||
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
drop table if exists ttl_00933_2;
|
||||
|
||||
create table ttl_00933_2 (d DateTime, a Int default 111 ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel, no-random-merge-tree-settings
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel, no-random-merge-tree-settings
|
||||
# Tags: no-random-merge-tree-settings
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,5 +1,3 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
drop table if exists ttl;
|
||||
|
||||
create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) settings remove_empty_parts = 0;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-replicated-database, no-parallel
|
||||
-- Tags: no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
|
||||
DROP TABLE IF EXISTS log_for_alter;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: zookeeper, no-replicated-database, no-parallel
|
||||
-- Tags: zookeeper, no-replicated-database
|
||||
-- Tag no-replicated-database: Unsupported type of ALTER query
|
||||
|
||||
DROP TABLE IF EXISTS replicated_table_for_alter1;
|
||||
|
Loading…
Reference in New Issue
Block a user