fix tests

This commit is contained in:
Alexander Tokmakov 2021-03-18 15:49:31 +03:00
parent 710c491f63
commit fdae70df6c
15 changed files with 88 additions and 70 deletions

View File

@ -17,16 +17,16 @@ function start()
# NOTE We run "clickhouse server" instead of "clickhouse-server"
# to make "pidof clickhouse-server" return single pid of the main instance.
# We wil run main instance using "service clickhouse-server start"
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \
-- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \
--logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \
-- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \
--logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \
--tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \
--mysql_port 19004 \
--test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \
-- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \
--logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \
-- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \
--logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \
--tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \
--mysql_port 29004 \
--test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3
@ -118,8 +118,8 @@ if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then
tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||:
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||:
pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||:
pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||:
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
mv /var/log/clickhouse-server/stderr2.log /test_output/ ||:
mv /var/log/clickhouse-server/stderr3.log /test_output/ ||:
fi

View File

@ -40,17 +40,17 @@ fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \
-- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \
--logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \
-- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \
--logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \
--tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \
--mysql_port 19004 \
--test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \
--macros.replica r2 # It doesn't work :(
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \
-- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \
--logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \
-- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \
--logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \
--tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \
--mysql_port 29004 \
--test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \
@ -102,8 +102,8 @@ tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log
tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||:
pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||:
pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||:
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
mv /var/log/clickhouse-server/stderr2.log /test_output/ ||:
mv /var/log/clickhouse-server/stderr3.log /test_output/ ||:
fi

View File

@ -355,20 +355,21 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q
bool maybe_replica_macros = info.expanded_other;
bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros");
if (!enable_functional_tests_helper)
LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments");
if (maybe_shard_macros && maybe_replica_macros)
return;
if (enable_functional_tests_helper)
{
if (maybe_path.empty() || maybe_path.back() != '/')
maybe_path += '/';
arg1->value = maybe_path + "{shard}";
arg2->value = maybe_replica + "{replica}";
arg1->value = maybe_path + "auto_{shard}";
arg2->value = maybe_replica + "auto_{replica}";
return;
}
LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments");
if (maybe_shard_macros && maybe_replica_macros)
return;
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. "
"If you really want to specify it explicitly, then you should use some macros "

View File

@ -194,6 +194,9 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file))
if not args.show_db_name:
os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file))
if args.replicated_database:
os.system("LC_ALL=C sed -i -e 's|/auto_{{shard}}||g' {file}".format(file=stdout_file))
os.system("LC_ALL=C sed -i -e 's|auto_{{replica}}||g' {file}".format(file=stdout_file))
stdout = open(stdout_file, 'rb').read() if os.path.exists(stdout_file) else b''
stdout = str(stdout, errors='replace', encoding='utf-8')
@ -209,8 +212,12 @@ def need_retry(stderr):
def get_processlist(args):
try:
query = b"SHOW PROCESSLIST FORMAT Vertical"
if args.replicated_database:
query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \
b"FROM clusterAllReplicas('r', system.processes) WHERE query NOT LIKE '%system.processes%' FORMAT Vertical"
clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10)
(stdout, _) = clickhouse_proc.communicate(query, timeout=10)
return False, stdout.decode('utf-8')
except Exception as ex:
print("Exception", ex)

View File

@ -71,25 +71,25 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
# There is a bug in config reloading, so we cannot override macros using --macros.replica r2
# And we have to copy configs...
mkdir /etc/clickhouse-server1
mkdir /etc/clickhouse-server2
mkdir /etc/clickhouse-server3
chown clickhouse /etc/clickhouse-server1
chown clickhouse /etc/clickhouse-server2
chown clickhouse /etc/clickhouse-server3
chgrp clickhouse /etc/clickhouse-server1
chgrp clickhouse /etc/clickhouse-server2
chgrp clickhouse /etc/clickhouse-server3
sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server1
sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2
sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server3
rm /etc/clickhouse-server1/config.d/macros.xml
rm /etc/clickhouse-server2/config.d/macros.xml
rm /etc/clickhouse-server3/config.d/macros.xml
sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|<replica>r1</replica>|<replica>r2</replica>|" > /etc/clickhouse-server2/config.d/macros.xml
sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|<shard>s1</shard>|<shard>s2</shard>|" > /etc/clickhouse-server3/config.d/macros.xml
sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|<replica>r1</replica>|<replica>r2</replica>|" > /etc/clickhouse-server1/config.d/macros.xml
sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|<shard>s1</shard>|<shard>s2</shard>|" > /etc/clickhouse-server2/config.d/macros.xml
sudo mkdir /var/lib/clickhouse1
sudo mkdir /var/lib/clickhouse2
sudo mkdir /var/lib/clickhouse3
sudo chown clickhouse /var/lib/clickhouse1
sudo chown clickhouse /var/lib/clickhouse2
sudo chown clickhouse /var/lib/clickhouse3
sudo chgrp clickhouse /var/lib/clickhouse1
sudo chgrp clickhouse /var/lib/clickhouse2
sudo chgrp clickhouse /var/lib/clickhouse3
fi
ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml

View File

@ -4,13 +4,13 @@ DROP TABLE IF EXISTS part_header_r2;
SET replication_alter_partitions_sync = 2;
CREATE TABLE part_header_r1(x UInt32, y UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '1') ORDER BY x
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '1{replica}') ORDER BY x
SETTINGS use_minimalistic_part_header_in_zookeeper = 0,
old_parts_lifetime = 1,
cleanup_delay_period = 0,
cleanup_delay_period_random_add = 0;
CREATE TABLE part_header_r2(x UInt32, y UInt32)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '2') ORDER BY x
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '2{replica}') ORDER BY x
SETTINGS use_minimalistic_part_header_in_zookeeper = 1,
old_parts_lifetime = 1,
cleanup_delay_period = 0,
@ -39,10 +39,10 @@ SELECT sleep(3) FORMAT Null;
SELECT '*** Test part removal ***';
SELECT '*** replica 1 ***';
SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1';
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts';
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts';
SELECT '*** replica 2 ***';
SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2';
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts';
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts';
SELECT '*** Test ALTER ***';
ALTER TABLE part_header_r1 MODIFY COLUMN y String;

View File

@ -15,7 +15,7 @@ CREATE TABLE elog (
engine_id UInt32,
referrer String
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog', 'test')
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog/{shard}', '{replica}')
PARTITION BY date
ORDER BY (engine_id)
SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;"
@ -28,35 +28,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 3, 'h
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 3 rows
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
while [[ $count != 2 ]]
do
sleep 1
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
done
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')"
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 4 rows
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
while [[ $count != 2 ]]
do
sleep 1
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
done
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')"
$CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 5 rows
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
while [[ $count != 2 ]]
do
sleep 1
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'")
count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'")
done
$CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')"

View File

@ -8,13 +8,13 @@ CREATE TABLE versioned_collapsing_table(
sign Int8,
version UInt16
)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table', '1', sign, version)
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table/{shard}', '{replica}', sign, version)
PARTITION BY d
ORDER BY (key1, key2);
INSERT INTO versioned_collapsing_table VALUES (toDate('2019-10-10'), 1, 1, 'Hello', -1, 1);
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table' and name = 'metadata';
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table/s1' and name = 'metadata';
SELECT COUNT() FROM versioned_collapsing_table;

View File

@ -2,4 +2,4 @@
10
10
24
CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions\', \'1\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192
CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192

View File

@ -5,7 +5,7 @@ CREATE TABLE replicated_mutations_empty_partitions
key UInt64,
value String
)
ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions', '1')
ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}', '{replica}')
ORDER BY key
PARTITION by key;
@ -13,7 +13,7 @@ INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number
SELECT count(distinct value) FROM replicated_mutations_empty_partitions;
SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers';
SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers';
ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3';
ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4';
@ -21,7 +21,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5';
ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9';
-- still ten records
SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers';
SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers';
ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2;

View File

@ -4,13 +4,13 @@
499999500000
499999500000
Metadata version on replica 1 equal with first replica, OK
CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
Metadata version on replica 2 equal with first replica, OK
CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
Metadata version on replica 3 equal with first replica, OK
CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
Metadata version on replica 4 equal with first replica, OK
CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
Metadata version on replica 5 equal with first replica, OK
CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192
499999500000

View File

@ -11,7 +11,9 @@ for i in $(seq $REPLICAS); do
done
for i in $(seq $REPLICAS); do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill/{shard}', '{replica}$i') ORDER BY key
SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_kill_1 SELECT number, toString(number) FROM numbers(1000000)"
@ -77,9 +79,9 @@ while true; do
done
metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'")
metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'")
for i in $(seq $REPLICAS); do
replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'")
replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'")
if [ "$metadata_version" != "$replica_metadata_version" ]; then
echo "Metadata version on replica $i differs from the first replica, FAIL"
else

View File

@ -1,16 +1,14 @@
block_numbers
blocks
1
r1
========
block_numbers
blocks
1
r1
========
block_numbers
blocks
========
1
failed_parts
last_part
leader_election-0000000000
parallel

View File

@ -3,17 +3,19 @@ DROP TABLE IF EXISTS sample_table;
CREATE TABLE sample_table (
key UInt64
)
ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in', '1')
ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in/{shard}', '{replica}')
ORDER BY tuple();
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in' AND name like 'block%' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/replicas' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name;
SELECT '========';
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in') AND name LIKE 'block%' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/replicas') ORDER BY name;
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name;
SELECT '========';
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in','/clickhouse/01700_system_zookeeper_path_in/replicas') AND name LIKE 'block%' ORDER BY name;
SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1',
'/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name;
SELECT '========';
SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/', name) FROM system.zookeeper WHERE (path = '/clickhouse/01700_system_zookeeper_path_in')) ORDER BY name;
SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/s1/', name)
FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/01700_system_zookeeper_path_in/s1')) ORDER BY name;
DROP TABLE IF EXISTS sample_table;

View File

@ -117,6 +117,14 @@
"01148_zookeeper_path_macros_unfolding",
"01294_system_distributed_on_cluster",
"01269_create_with_null",
"01451_replicated_detach_drop_and_quorum",
"01188_attach_table_from_path",
/// user_files
"01721_engine_file_truncate_on_insert",
/// Fails due to additional replicas or shards
"01650_drop_part_and_deduplication_zookeeper",
"01532_execute_merges_on_single_replica",
"01509_parallel_quorum_insert_no_replicas",
/// grep -c
"01018_ddl_dictionaries_bad_queries",
"00908_bloom_filter_index",