ClickHouse/tests/integration/test_insert_into_distributed/test.py

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

366 lines
12 KiB
Python
Raw Normal View History

2017-06-14 14:38:08 +00:00
import time
import pytest
2021-04-28 06:43:56 +00:00
from helpers.client import QueryRuntimeException
2017-06-14 14:38:08 +00:00
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
2017-07-11 11:44:16 +00:00
from helpers.test_tools import TSV
2017-06-14 14:38:08 +00:00
cluster = ClickHouseCluster(__file__)
2017-07-11 11:44:16 +00:00
instance_test_reconnect = cluster.add_instance(
"instance_test_reconnect", main_configs=["configs/remote_servers.xml"]
)
instance_test_inserts_batching = cluster.add_instance(
"instance_test_inserts_batching",
main_configs=["configs/remote_servers.xml"],
user_configs=["configs/enable_distributed_inserts_batching.xml"],
)
2020-08-03 11:13:43 +00:00
remote = cluster.add_instance(
"remote", main_configs=["configs/forbid_background_merges.xml"]
)
2017-06-14 14:38:08 +00:00
2017-08-23 10:45:23 +00:00
instance_test_inserts_local_cluster = cluster.add_instance(
"instance_test_inserts_local_cluster", main_configs=["configs/remote_servers.xml"]
)
node1 = cluster.add_instance(
"node1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True
)
2018-12-03 13:11:26 +00:00
shard1 = cluster.add_instance(
"shard1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True
)
shard2 = cluster.add_instance(
"shard2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True
)
2017-08-23 10:45:23 +00:00
2017-06-14 14:38:08 +00:00
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
2017-07-11 11:44:16 +00:00
remote.query("CREATE TABLE local1 (x UInt32) ENGINE = Log")
instance_test_reconnect.query(
"""
2017-07-11 11:44:16 +00:00
CREATE TABLE distributed (x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local1')
"""
2017-07-11 11:44:16 +00:00
)
2017-06-14 14:38:08 +00:00
2017-07-21 11:58:18 +00:00
remote.query(
2022-06-23 08:37:52 +00:00
"CREATE TABLE local2 (d Date, x UInt32, s String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"
2017-07-21 11:58:18 +00:00
)
2017-07-11 11:44:16 +00:00
instance_test_inserts_batching.query(
"""
CREATE TABLE distributed (d Date, x UInt32) ENGINE = Distributed('test_cluster', 'default', 'local2') SETTINGS fsync_after_insert=1, fsync_directories=1
"""
2017-08-23 10:45:23 +00:00
)
instance_test_inserts_local_cluster.query(
2022-06-23 08:37:52 +00:00
"CREATE TABLE local (d Date, x UInt32) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"
)
2017-08-23 10:45:23 +00:00
instance_test_inserts_local_cluster.query(
"""
2017-08-23 10:45:23 +00:00
CREATE TABLE distributed_on_local (d Date, x UInt32) ENGINE = Distributed('test_local_cluster', 'default', 'local')
"""
2017-06-14 14:38:08 +00:00
)
node1.query(
"""
2022-06-23 08:37:52 +00:00
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node1') PARTITION BY toYYYYMM(date) ORDER BY id
"""
)
node2.query(
"""
2022-06-23 08:37:52 +00:00
CREATE TABLE replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id
"""
)
node1.query(
"""
CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'replicated')
"""
)
node2.query(
"""
CREATE TABLE distributed (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'replicated')
"""
2018-12-03 13:11:26 +00:00
)
shard1.query(
"""
2022-06-23 08:37:52 +00:00
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"""
)
2018-12-03 13:11:26 +00:00
shard2.query(
"""
2022-06-23 08:37:52 +00:00
CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY x"""
)
2018-12-03 13:11:26 +00:00
shard1.query(
"""
CREATE TABLE low_cardinality_all (d Date, x UInt32, s LowCardinality(String)) ENGINE = Distributed('shard_with_low_cardinality', 'default', 'low_cardinality', sipHash64(s))"""
)
2019-11-13 18:35:35 +00:00
node1.query(
"""
2019-11-13 18:35:35 +00:00
CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n"""
)
node2.query(
"""
2019-11-13 18:35:35 +00:00
CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n"""
)
2021-04-20 14:07:19 +00:00
node1.query(
"""
2021-04-28 06:43:56 +00:00
CREATE TABLE distributed_one_replica_internal_replication (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica_internal_replication', 'default', 'single_replicated')
"""
2021-04-20 14:07:19 +00:00
)
node2.query(
"""
2021-04-28 06:43:56 +00:00
CREATE TABLE distributed_one_replica_internal_replication (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica_internal_replication', 'default', 'single_replicated')
"""
2021-04-28 06:43:56 +00:00
)
node1.query(
"""
2021-04-28 06:43:56 +00:00
CREATE TABLE distributed_one_replica_no_internal_replication (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'single_replicated')
"""
2021-04-28 06:43:56 +00:00
)
node2.query(
"""
2021-04-28 06:43:56 +00:00
CREATE TABLE distributed_one_replica_no_internal_replication (date Date, id UInt32) ENGINE = Distributed('shard_with_local_replica', 'default', 'single_replicated')
"""
2021-04-20 14:07:19 +00:00
)
node2.query(
"""
2022-06-23 08:37:52 +00:00
CREATE TABLE single_replicated(date Date, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/single_replicated', 'node2') PARTITION BY toYYYYMM(date) ORDER BY id
"""
2021-04-20 14:07:19 +00:00
)
2017-06-14 14:38:08 +00:00
yield cluster
finally:
cluster.shutdown()
def test_reconnect(started_cluster):
2017-07-11 11:44:16 +00:00
instance = instance_test_reconnect
2017-06-14 14:38:08 +00:00
with PartitionManager() as pm:
# Open a connection for insertion.
2017-07-11 11:44:16 +00:00
instance.query("INSERT INTO distributed VALUES (1)")
2019-02-05 10:15:14 +00:00
time.sleep(1)
2017-07-11 11:44:16 +00:00
assert remote.query("SELECT count(*) FROM local1").strip() == "1"
2017-06-14 14:38:08 +00:00
# Now break the connection.
2017-07-11 11:44:16 +00:00
pm.partition_instances(
instance, remote, action="REJECT --reject-with tcp-reset"
)
instance.query("INSERT INTO distributed VALUES (2)")
2019-02-05 10:15:14 +00:00
time.sleep(1)
2017-06-14 14:38:08 +00:00
# Heal the partition and insert more data.
# The connection must be reestablished and after some time all data must be inserted.
pm.heal_all()
2019-02-05 10:15:14 +00:00
time.sleep(1)
2017-07-11 11:44:16 +00:00
instance.query("INSERT INTO distributed VALUES (3)")
2021-03-16 10:00:49 +00:00
time.sleep(5)
2017-07-11 11:44:16 +00:00
assert remote.query("SELECT count(*) FROM local1").strip() == "3"
def test_inserts_batching(started_cluster):
instance = instance_test_inserts_batching
with PartitionManager() as pm:
pm.partition_instances(instance, remote)
instance.query("INSERT INTO distributed(d, x) VALUES ('2000-01-01', 1)")
# Sleep a bit so that this INSERT forms a batch of its own.
time.sleep(0.1)
instance.query("INSERT INTO distributed(x, d) VALUES (2, '2000-01-01')")
for i in range(3, 7):
instance.query(
"INSERT INTO distributed(d, x) VALUES ('2000-01-01', {})".format(i)
)
2017-07-11 11:44:16 +00:00
for i in range(7, 9):
instance.query(
"INSERT INTO distributed(x, d) VALUES ({}, '2000-01-01')".format(i)
)
2017-07-11 11:44:16 +00:00
instance.query("INSERT INTO distributed(d, x) VALUES ('2000-01-01', 9)")
2017-07-21 11:58:18 +00:00
# After ALTER the structure of the saved blocks will be different
instance.query("ALTER TABLE distributed ADD COLUMN s String")
for i in range(10, 13):
instance.query(
"INSERT INTO distributed(d, x) VALUES ('2000-01-01', {})".format(i)
)
2017-07-21 11:58:18 +00:00
2020-04-20 03:15:29 +00:00
instance.query("SYSTEM FLUSH DISTRIBUTED distributed")
2017-07-11 11:44:16 +00:00
time.sleep(1.0)
result = remote.query(
"SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part"
)
# Explanation: as merges are turned off on remote instance, active parts in local2 table correspond 1-to-1
# to inserted blocks.
# Batches of max 3 rows are formed as min_insert_block_size_rows = 3.
# Blocks:
# 1. Failed batch that is retried with the same contents.
# 2. Full batch of inserts before ALTER.
# 3. Full batch of inserts before ALTER.
2017-07-21 11:58:18 +00:00
# 4. Full batch of inserts after ALTER (that have different block structure).
# 5. What was left to insert with the column structure before ALTER.
2017-07-11 11:44:16 +00:00
expected = """\
2022-06-24 17:10:33 +00:00
200001_1_1_0\t[1]
200001_2_2_0\t[2,3,4]
200001_3_3_0\t[5,6,7]
200001_4_4_0\t[10,11,12]
200001_5_5_0\t[8,9]
"""
2017-07-11 11:44:16 +00:00
assert TSV(result) == TSV(expected)
2017-08-23 10:45:23 +00:00
def test_inserts_local(started_cluster):
instance = instance_test_inserts_local_cluster
instance.query("INSERT INTO distributed_on_local VALUES ('2000-01-01', 1)")
time.sleep(0.5)
assert instance.query("SELECT count(*) FROM local").strip() == "1"
2021-04-28 06:43:56 +00:00
def test_inserts_single_replica_local_internal_replication(started_cluster):
with pytest.raises(
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
2023-08-11 10:24:16 +00:00
QueryRuntimeException, match="Table default.single_replicated does not exist"
2021-04-28 06:43:56 +00:00
):
node1.query(
"INSERT INTO distributed_one_replica_internal_replication VALUES ('2000-01-01', 1)",
settings={
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
"distributed_foreground_insert": "1",
2021-04-28 06:43:56 +00:00
"prefer_localhost_replica": "1",
# to make the test more deterministic
"load_balancing": "first_or_random",
},
)
assert node2.query("SELECT count(*) FROM single_replicated").strip() == "0"
def test_inserts_single_replica_internal_replication(started_cluster):
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
try:
2021-04-28 06:43:56 +00:00
node1.query(
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
"INSERT INTO distributed_one_replica_internal_replication VALUES ('2000-01-01', 1)",
2021-04-28 06:43:56 +00:00
settings={
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
"distributed_foreground_insert": "1",
2021-04-28 06:43:56 +00:00
"prefer_localhost_replica": "0",
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
# to make the test more deterministic
"load_balancing": "first_or_random",
2021-04-28 06:43:56 +00:00
},
)
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
assert node2.query("SELECT count(*) FROM single_replicated").strip() == "1"
finally:
node2.query("TRUNCATE TABLE single_replicated")
def test_inserts_single_replica_no_internal_replication(started_cluster):
try:
with pytest.raises(
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
2023-08-11 10:24:16 +00:00
QueryRuntimeException,
match="Table default.single_replicated does not exist",
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
):
node1.query(
"INSERT INTO distributed_one_replica_no_internal_replication VALUES ('2000-01-01', 1)",
settings={
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
"distributed_foreground_insert": "1",
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
"prefer_localhost_replica": "0",
},
)
2023-02-25 00:18:34 +00:00
assert node2.query("SELECT count(*) FROM single_replicated").strip() == "0"
Fix test_insert_into_distributed flaps In [1] the order of the tests was different: Running pytest container as: 'docker run --net=host --rm --name clickhouse_integration_tests --privileged --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-odbc-bridge:/clickhouse-odbc-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse-library-bridge:/clickhouse-library-bridge --volume=/place/sandbox-data/tasks/7/4/962738347/clickhouse:/clickhouse --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/programs/server:/clickhouse-config --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/tests/integration:/ClickHouse/tests/integration --volume=/place/sandbox-data/tasks/7/4/962738347/ClickHouse/src/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos --volume=clickhouse_integration_tests_volume:/var/lib/docker -e DOCKER_MYSQL_GOLANG_CLIENT_TAG=latest -e DOCKER_MYSQL_JAVA_CLIENT_TAG=latest -e DOCKER_MYSQL_JS_CLIENT_TAG=latest -e DOCKER_MYSQL_PHP_CLIENT_TAG=latest -e DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=latest -e DOCKER_BASE_TAG=latest -e DOCKER_KERBEROS_KDC_TAG=latest -e PYTEST_OPTS='-ss test_insert_into_distributed/test.py::test_inserts_batching test_insert_into_distributed/test.py::test_inserts_local test_insert_into_distributed/test.py::test_inserts_low_cardinality test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication test_insert_into_distributed/test.py::test_prefer_localhost_replica test_insert_into_distributed/test.py::test_reconnect test_insert_into_distributed/test.py::test_table_function -rfEp --color=no --durations=0 ' yandex/clickhouse-integration-tests-runner:latest '. [1]: https://clickhouse-test-reports.s3.yandex.net/23874/80065b08f8182936ac57408bd84492d95f2322c1/integration_tests_(asan).html#fail1 So the table should be TRUNCATEd to avoid failures in this case.
2021-05-05 18:00:46 +00:00
finally:
node2.query("TRUNCATE TABLE single_replicated")
2021-04-28 06:43:56 +00:00
def test_prefer_localhost_replica(started_cluster):
2020-01-28 18:54:25 +00:00
test_query = "SELECT * FROM distributed ORDER BY id"
node1.query("INSERT INTO distributed VALUES (toDate('2017-06-17'), 11)")
node2.query("INSERT INTO distributed VALUES (toDate('2017-06-17'), 22)")
time.sleep(1.0)
2020-01-28 18:54:25 +00:00
expected_distributed = """\
2018-08-10 01:31:55 +00:00
2017-06-17\t11
2017-06-17\t22
"""
2020-01-28 18:54:25 +00:00
expected_from_node2 = """\
2018-08-10 01:31:55 +00:00
2017-06-17\t11
2017-06-17\t22
2017-06-17\t44
"""
2020-01-28 18:54:25 +00:00
expected_from_node1 = """\
2018-08-10 01:31:55 +00:00
2017-06-17\t11
2017-06-17\t22
2017-06-17\t33
"""
2020-01-28 18:54:25 +00:00
assert TSV(node1.query(test_query)) == TSV(expected_distributed)
assert TSV(node2.query(test_query)) == TSV(expected_distributed)
# Make replicas inconsistent by disabling merges and fetches
# for possibility of determining to which replica the query was send
node1.query("SYSTEM STOP MERGES")
node1.query("SYSTEM STOP FETCHES")
node2.query("SYSTEM STOP MERGES")
node2.query("SYSTEM STOP FETCHES")
node1.query("INSERT INTO replicated VALUES (toDate('2017-06-17'), 33)")
node2.query("INSERT INTO replicated VALUES (toDate('2017-06-17'), 44)")
time.sleep(1.0)
# Query is sent to node2, as it local and prefer_localhost_replica=1
assert TSV(node2.query(test_query)) == TSV(expected_from_node2)
# Now query is sent to node1, as it higher in order
assert TSV(
node2.query(
test_query
+ " SETTINGS load_balancing='in_order', prefer_localhost_replica=0"
)
) == TSV(expected_from_node1)
2018-12-03 13:11:26 +00:00
def test_inserts_low_cardinality(started_cluster):
2018-12-03 13:22:29 +00:00
instance = shard1
2018-12-03 13:11:26 +00:00
instance.query(
"INSERT INTO low_cardinality_all (d,x,s) VALUES ('2018-11-12',1,'123')"
)
time.sleep(0.5)
assert instance.query("SELECT count(*) FROM low_cardinality_all").strip() == "1"
2019-11-13 18:35:35 +00:00
2019-11-13 18:35:35 +00:00
def test_table_function(started_cluster):
node1.query(
"insert into table function cluster('shard_with_local_replica', 'default', 'table_function') select number, concat('str_', toString(number)) from numbers(100000)"
)
assert (
node1.query(
"select count() from cluster('shard_with_local_replica', 'default', 'table_function')"
).rstrip()
== "100000"
)