ClickHouse/tests/integration/test_parallel_replicas_over_distributed/test.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

158 lines
6.3 KiB
Python
Raw Permalink Normal View History

2023-08-07 19:12:53 +00:00
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
nodes = [
2023-08-07 21:28:44 +00:00
cluster.add_instance(
f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True
)
for i in (1, 2, 3, 4, 5, 6)
2023-08-07 19:12:53 +00:00
]
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
2023-08-07 21:06:12 +00:00
def create_tables(cluster, table_name):
# create replicated tables
2023-08-07 19:12:53 +00:00
for node in nodes:
2023-08-07 21:06:12 +00:00
node.query(f"DROP TABLE IF EXISTS {table_name} SYNC")
if cluster == "test_single_shard_multiple_replicas":
nodes[0].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)"
)
nodes[1].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)"
)
nodes[2].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)"
)
nodes[3].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r4') ORDER BY (key)"
)
elif cluster == "test_multiple_shards_multiple_replicas":
# shard 1
nodes[0].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)"
)
nodes[1].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)"
)
nodes[2].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)"
2023-08-07 21:06:12 +00:00
)
# shard 2
2023-08-07 21:06:12 +00:00
nodes[3].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r1') ORDER BY (key)"
)
nodes[4].query(
2023-08-07 21:06:12 +00:00
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r2') ORDER BY (key)"
2023-08-07 19:12:53 +00:00
)
nodes[5].query(
f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r3') ORDER BY (key)"
)
2023-08-07 21:06:12 +00:00
else:
raise Exception(f"Unexpected cluster: {cluster}")
2023-08-07 19:12:53 +00:00
2023-08-07 21:06:12 +00:00
# create distributed table
nodes[0].query(f"DROP TABLE IF EXISTS {table_name}_d SYNC")
2023-08-07 19:12:53 +00:00
nodes[0].query(
f"""
2023-08-07 21:06:12 +00:00
CREATE TABLE {table_name}_d AS {table_name}
2023-08-07 19:12:53 +00:00
Engine=Distributed(
{cluster},
currentDatabase(),
2023-08-07 21:06:12 +00:00
{table_name},
key
2023-08-07 19:12:53 +00:00
)
"""
)
2023-08-07 21:06:12 +00:00
# populate data
2023-08-07 21:28:44 +00:00
nodes[0].query(
f"INSERT INTO {table_name}_d SELECT number, number FROM numbers(1000)",
Rename directory monitor concept into background INSERT (#55978) * Limit log frequence for "Skipping send data over distributed table" message After SYSTEM STOP DISTRIBUTED SENDS it will constantly print this message. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename directory monitor concept into async INSERT Rename the following query settings (with preserving backward compatiblity, by keeping old name as an alias): - distributed_directory_monitor_sleep_time_ms -> distributed_async_insert_sleep_time_ms - distributed_directory_monitor_max_sleep_time_ms -> distributed_async_insert_max_sleep_time_ms - distributed_directory_monitor_batch -> distributed_async_insert_batch_inserts - distributed_directory_monitor_split_batch_on_failure -> distributed_async_insert_split_batch_on_failure Rename the following table settings (with preserving backward compatiblity, by keeping old name as an alias): - monitor_batch_inserts -> async_insert_batch - monitor_split_batch_on_failure -> async_insert_split_batch_on_failure - directory_monitor_sleep_time_ms -> async_insert_sleep_time_ms - directory_monitor_max_sleep_time_ms -> async_insert_max_sleep_time_ms And also update all the references: $ gg -e directory_monitor_ -e monitor_ tests docs | cut -d: -f1 | sort -u | xargs sed -e 's/distributed_directory_monitor_sleep_time_ms/distributed_async_insert_sleep_time_ms/g' -e 's/distributed_directory_monitor_max_sleep_time_ms/distributed_async_insert_max_sleep_time_ms/g' -e 's/distributed_directory_monitor_batch_inserts/distributed_async_insert_batch/g' -e 's/distributed_directory_monitor_split_batch_on_failure/distributed_async_insert_split_batch_on_failure/g' -e 's/monitor_batch_inserts/async_insert_batch/g' -e 's/monitor_split_batch_on_failure/async_insert_split_batch_on_failure/g' -e 's/monitor_sleep_time_ms/async_insert_sleep_time_ms/g' -e 's/monitor_max_sleep_time_ms/async_insert_max_sleep_time_ms/g' -i Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename async_insert for Distributed into background_insert This will avoid amigibuity between general async INSERT's and INSERT into Distributed, which are indeed background, so new term express it even better. Mostly done with: $ git di HEAD^ --name-only | xargs sed -i -e 's/distributed_async_insert/distributed_background_insert/g' -e 's/async_insert_batch/background_insert_batch/g' -e 's/async_insert_split_batch_on_failure/background_insert_split_batch_on_failure/g' -e 's/async_insert_sleep_time_ms/background_insert_sleep_time_ms/g' -e 's/async_insert_max_sleep_time_ms/background_insert_max_sleep_time_ms/g' Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Mark 02417_opentelemetry_insert_on_distributed_table as long CI: https://s3.amazonaws.com/clickhouse-test-reports/55978/7a6abb03a0b507e29e999cb7e04f246a119c6f28/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> --------- Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-01 14:09:39 +00:00
settings={"distributed_foreground_insert": 1},
2023-08-07 21:28:44 +00:00
)
nodes[0].query(
f"INSERT INTO {table_name}_d SELECT number, number FROM numbers(2000)",
Rename directory monitor concept into background INSERT (#55978) * Limit log frequence for "Skipping send data over distributed table" message After SYSTEM STOP DISTRIBUTED SENDS it will constantly print this message. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename directory monitor concept into async INSERT Rename the following query settings (with preserving backward compatiblity, by keeping old name as an alias): - distributed_directory_monitor_sleep_time_ms -> distributed_async_insert_sleep_time_ms - distributed_directory_monitor_max_sleep_time_ms -> distributed_async_insert_max_sleep_time_ms - distributed_directory_monitor_batch -> distributed_async_insert_batch_inserts - distributed_directory_monitor_split_batch_on_failure -> distributed_async_insert_split_batch_on_failure Rename the following table settings (with preserving backward compatiblity, by keeping old name as an alias): - monitor_batch_inserts -> async_insert_batch - monitor_split_batch_on_failure -> async_insert_split_batch_on_failure - directory_monitor_sleep_time_ms -> async_insert_sleep_time_ms - directory_monitor_max_sleep_time_ms -> async_insert_max_sleep_time_ms And also update all the references: $ gg -e directory_monitor_ -e monitor_ tests docs | cut -d: -f1 | sort -u | xargs sed -e 's/distributed_directory_monitor_sleep_time_ms/distributed_async_insert_sleep_time_ms/g' -e 's/distributed_directory_monitor_max_sleep_time_ms/distributed_async_insert_max_sleep_time_ms/g' -e 's/distributed_directory_monitor_batch_inserts/distributed_async_insert_batch/g' -e 's/distributed_directory_monitor_split_batch_on_failure/distributed_async_insert_split_batch_on_failure/g' -e 's/monitor_batch_inserts/async_insert_batch/g' -e 's/monitor_split_batch_on_failure/async_insert_split_batch_on_failure/g' -e 's/monitor_sleep_time_ms/async_insert_sleep_time_ms/g' -e 's/monitor_max_sleep_time_ms/async_insert_max_sleep_time_ms/g' -i Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename async_insert for Distributed into background_insert This will avoid amigibuity between general async INSERT's and INSERT into Distributed, which are indeed background, so new term express it even better. Mostly done with: $ git di HEAD^ --name-only | xargs sed -i -e 's/distributed_async_insert/distributed_background_insert/g' -e 's/async_insert_batch/background_insert_batch/g' -e 's/async_insert_split_batch_on_failure/background_insert_split_batch_on_failure/g' -e 's/async_insert_sleep_time_ms/background_insert_sleep_time_ms/g' -e 's/async_insert_max_sleep_time_ms/background_insert_max_sleep_time_ms/g' Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Mark 02417_opentelemetry_insert_on_distributed_table as long CI: https://s3.amazonaws.com/clickhouse-test-reports/55978/7a6abb03a0b507e29e999cb7e04f246a119c6f28/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> --------- Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-01 14:09:39 +00:00
settings={"distributed_foreground_insert": 1},
2023-08-07 21:28:44 +00:00
)
nodes[0].query(
f"INSERT INTO {table_name}_d SELECT -number, -number FROM numbers(1000)",
Rename directory monitor concept into background INSERT (#55978) * Limit log frequence for "Skipping send data over distributed table" message After SYSTEM STOP DISTRIBUTED SENDS it will constantly print this message. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename directory monitor concept into async INSERT Rename the following query settings (with preserving backward compatiblity, by keeping old name as an alias): - distributed_directory_monitor_sleep_time_ms -> distributed_async_insert_sleep_time_ms - distributed_directory_monitor_max_sleep_time_ms -> distributed_async_insert_max_sleep_time_ms - distributed_directory_monitor_batch -> distributed_async_insert_batch_inserts - distributed_directory_monitor_split_batch_on_failure -> distributed_async_insert_split_batch_on_failure Rename the following table settings (with preserving backward compatiblity, by keeping old name as an alias): - monitor_batch_inserts -> async_insert_batch - monitor_split_batch_on_failure -> async_insert_split_batch_on_failure - directory_monitor_sleep_time_ms -> async_insert_sleep_time_ms - directory_monitor_max_sleep_time_ms -> async_insert_max_sleep_time_ms And also update all the references: $ gg -e directory_monitor_ -e monitor_ tests docs | cut -d: -f1 | sort -u | xargs sed -e 's/distributed_directory_monitor_sleep_time_ms/distributed_async_insert_sleep_time_ms/g' -e 's/distributed_directory_monitor_max_sleep_time_ms/distributed_async_insert_max_sleep_time_ms/g' -e 's/distributed_directory_monitor_batch_inserts/distributed_async_insert_batch/g' -e 's/distributed_directory_monitor_split_batch_on_failure/distributed_async_insert_split_batch_on_failure/g' -e 's/monitor_batch_inserts/async_insert_batch/g' -e 's/monitor_split_batch_on_failure/async_insert_split_batch_on_failure/g' -e 's/monitor_sleep_time_ms/async_insert_sleep_time_ms/g' -e 's/monitor_max_sleep_time_ms/async_insert_max_sleep_time_ms/g' -i Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename async_insert for Distributed into background_insert This will avoid amigibuity between general async INSERT's and INSERT into Distributed, which are indeed background, so new term express it even better. Mostly done with: $ git di HEAD^ --name-only | xargs sed -i -e 's/distributed_async_insert/distributed_background_insert/g' -e 's/async_insert_batch/background_insert_batch/g' -e 's/async_insert_split_batch_on_failure/background_insert_split_batch_on_failure/g' -e 's/async_insert_sleep_time_ms/background_insert_sleep_time_ms/g' -e 's/async_insert_max_sleep_time_ms/background_insert_max_sleep_time_ms/g' Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Mark 02417_opentelemetry_insert_on_distributed_table as long CI: https://s3.amazonaws.com/clickhouse-test-reports/55978/7a6abb03a0b507e29e999cb7e04f246a119c6f28/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> --------- Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-01 14:09:39 +00:00
settings={"distributed_foreground_insert": 1},
2023-08-07 21:28:44 +00:00
)
nodes[0].query(
f"INSERT INTO {table_name}_d SELECT -number, -number FROM numbers(2000)",
Rename directory monitor concept into background INSERT (#55978) * Limit log frequence for "Skipping send data over distributed table" message After SYSTEM STOP DISTRIBUTED SENDS it will constantly print this message. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename directory monitor concept into async INSERT Rename the following query settings (with preserving backward compatiblity, by keeping old name as an alias): - distributed_directory_monitor_sleep_time_ms -> distributed_async_insert_sleep_time_ms - distributed_directory_monitor_max_sleep_time_ms -> distributed_async_insert_max_sleep_time_ms - distributed_directory_monitor_batch -> distributed_async_insert_batch_inserts - distributed_directory_monitor_split_batch_on_failure -> distributed_async_insert_split_batch_on_failure Rename the following table settings (with preserving backward compatiblity, by keeping old name as an alias): - monitor_batch_inserts -> async_insert_batch - monitor_split_batch_on_failure -> async_insert_split_batch_on_failure - directory_monitor_sleep_time_ms -> async_insert_sleep_time_ms - directory_monitor_max_sleep_time_ms -> async_insert_max_sleep_time_ms And also update all the references: $ gg -e directory_monitor_ -e monitor_ tests docs | cut -d: -f1 | sort -u | xargs sed -e 's/distributed_directory_monitor_sleep_time_ms/distributed_async_insert_sleep_time_ms/g' -e 's/distributed_directory_monitor_max_sleep_time_ms/distributed_async_insert_max_sleep_time_ms/g' -e 's/distributed_directory_monitor_batch_inserts/distributed_async_insert_batch/g' -e 's/distributed_directory_monitor_split_batch_on_failure/distributed_async_insert_split_batch_on_failure/g' -e 's/monitor_batch_inserts/async_insert_batch/g' -e 's/monitor_split_batch_on_failure/async_insert_split_batch_on_failure/g' -e 's/monitor_sleep_time_ms/async_insert_sleep_time_ms/g' -e 's/monitor_max_sleep_time_ms/async_insert_max_sleep_time_ms/g' -i Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename async_insert for Distributed into background_insert This will avoid amigibuity between general async INSERT's and INSERT into Distributed, which are indeed background, so new term express it even better. Mostly done with: $ git di HEAD^ --name-only | xargs sed -i -e 's/distributed_async_insert/distributed_background_insert/g' -e 's/async_insert_batch/background_insert_batch/g' -e 's/async_insert_split_batch_on_failure/background_insert_split_batch_on_failure/g' -e 's/async_insert_sleep_time_ms/background_insert_sleep_time_ms/g' -e 's/async_insert_max_sleep_time_ms/background_insert_max_sleep_time_ms/g' Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Mark 02417_opentelemetry_insert_on_distributed_table as long CI: https://s3.amazonaws.com/clickhouse-test-reports/55978/7a6abb03a0b507e29e999cb7e04f246a119c6f28/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> --------- Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-01 14:09:39 +00:00
settings={"distributed_foreground_insert": 1},
2023-08-07 21:28:44 +00:00
)
nodes[0].query(
f"INSERT INTO {table_name}_d SELECT number, number FROM numbers(3)",
Rename directory monitor concept into background INSERT (#55978) * Limit log frequence for "Skipping send data over distributed table" message After SYSTEM STOP DISTRIBUTED SENDS it will constantly print this message. Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename directory monitor concept into async INSERT Rename the following query settings (with preserving backward compatiblity, by keeping old name as an alias): - distributed_directory_monitor_sleep_time_ms -> distributed_async_insert_sleep_time_ms - distributed_directory_monitor_max_sleep_time_ms -> distributed_async_insert_max_sleep_time_ms - distributed_directory_monitor_batch -> distributed_async_insert_batch_inserts - distributed_directory_monitor_split_batch_on_failure -> distributed_async_insert_split_batch_on_failure Rename the following table settings (with preserving backward compatiblity, by keeping old name as an alias): - monitor_batch_inserts -> async_insert_batch - monitor_split_batch_on_failure -> async_insert_split_batch_on_failure - directory_monitor_sleep_time_ms -> async_insert_sleep_time_ms - directory_monitor_max_sleep_time_ms -> async_insert_max_sleep_time_ms And also update all the references: $ gg -e directory_monitor_ -e monitor_ tests docs | cut -d: -f1 | sort -u | xargs sed -e 's/distributed_directory_monitor_sleep_time_ms/distributed_async_insert_sleep_time_ms/g' -e 's/distributed_directory_monitor_max_sleep_time_ms/distributed_async_insert_max_sleep_time_ms/g' -e 's/distributed_directory_monitor_batch_inserts/distributed_async_insert_batch/g' -e 's/distributed_directory_monitor_split_batch_on_failure/distributed_async_insert_split_batch_on_failure/g' -e 's/monitor_batch_inserts/async_insert_batch/g' -e 's/monitor_split_batch_on_failure/async_insert_split_batch_on_failure/g' -e 's/monitor_sleep_time_ms/async_insert_sleep_time_ms/g' -e 's/monitor_max_sleep_time_ms/async_insert_max_sleep_time_ms/g' -i Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Rename async_insert for Distributed into background_insert This will avoid amigibuity between general async INSERT's and INSERT into Distributed, which are indeed background, so new term express it even better. Mostly done with: $ git di HEAD^ --name-only | xargs sed -i -e 's/distributed_async_insert/distributed_background_insert/g' -e 's/async_insert_batch/background_insert_batch/g' -e 's/async_insert_split_batch_on_failure/background_insert_split_batch_on_failure/g' -e 's/async_insert_sleep_time_ms/background_insert_sleep_time_ms/g' -e 's/async_insert_max_sleep_time_ms/background_insert_max_sleep_time_ms/g' Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> * Mark 02417_opentelemetry_insert_on_distributed_table as long CI: https://s3.amazonaws.com/clickhouse-test-reports/55978/7a6abb03a0b507e29e999cb7e04f246a119c6f28/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> --------- Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-11-01 14:09:39 +00:00
settings={"distributed_foreground_insert": 1},
2023-08-07 21:28:44 +00:00
)
2023-08-07 19:12:53 +00:00
@pytest.mark.parametrize(
"cluster,max_parallel_replicas,prefer_localhost_replica",
[
pytest.param("test_single_shard_multiple_replicas", 2, 0),
pytest.param("test_single_shard_multiple_replicas", 3, 0),
pytest.param("test_single_shard_multiple_replicas", 4, 0),
pytest.param("test_single_shard_multiple_replicas", 10, 0),
pytest.param("test_single_shard_multiple_replicas", 2, 1),
pytest.param("test_single_shard_multiple_replicas", 3, 1),
pytest.param("test_single_shard_multiple_replicas", 4, 1),
pytest.param("test_single_shard_multiple_replicas", 10, 1),
pytest.param("test_multiple_shards_multiple_replicas", 2, 0),
pytest.param("test_multiple_shards_multiple_replicas", 3, 0),
pytest.param("test_multiple_shards_multiple_replicas", 4, 0),
pytest.param("test_multiple_shards_multiple_replicas", 10, 0),
pytest.param("test_multiple_shards_multiple_replicas", 2, 1),
pytest.param("test_multiple_shards_multiple_replicas", 3, 1),
pytest.param("test_multiple_shards_multiple_replicas", 4, 1),
pytest.param("test_multiple_shards_multiple_replicas", 10, 1),
2023-08-17 11:42:56 +00:00
],
2023-08-07 19:12:53 +00:00
)
2023-08-17 11:42:56 +00:00
def test_parallel_replicas_over_distributed(
start_cluster, cluster, max_parallel_replicas, prefer_localhost_replica
):
2023-08-07 21:06:12 +00:00
table_name = "test_table"
create_tables(cluster, table_name)
2023-08-07 19:12:53 +00:00
node = nodes[0]
expected_result = f"6003\t-1999\t1999\t3\n"
2023-08-07 21:06:12 +00:00
# sync all replicas to get consistent result
node.query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster} {table_name}")
2023-08-07 21:06:12 +00:00
# parallel replicas
2023-08-07 19:12:53 +00:00
assert (
node.query(
2023-08-07 21:06:12 +00:00
f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d",
2023-08-07 19:12:53 +00:00
settings={
2023-08-07 21:06:12 +00:00
"allow_experimental_parallel_reading_from_replicas": 2,
"prefer_localhost_replica": prefer_localhost_replica,
"max_parallel_replicas": max_parallel_replicas,
2023-08-07 19:12:53 +00:00
},
)
== expected_result
)
# w/o parallel replicas
assert (
2023-12-13 11:38:41 +00:00
node.query(
f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d",
settings={
"allow_experimental_parallel_reading_from_replicas": 0,
2023-12-13 11:38:41 +00:00
},
)
== expected_result
)