ClickHouse/tests/integration/test_sharding_key_from_default_column/test.py

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

195 lines
7.9 KiB
Python
Raw Permalink Normal View History

2021-09-06 03:11:16 +00:00
import pytest
import itertools
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", main_configs=["configs/test_cluster.xml"], with_zookeeper=True
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/test_cluster.xml"], with_zookeeper=True
)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def cleanup_after_test():
try:
yield
finally:
node1.query("DROP TABLE IF EXISTS dist ON CLUSTER 'test_cluster'")
node1.query("DROP TABLE IF EXISTS local ON CLUSTER 'test_cluster'")
# A default column is used in the sharding key expression.
def test_default_column():
node1.query(
"CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 DEFAULT x + 100, z Int32 DEFAULT x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)"
)
node1.query(
"CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 DEFAULT x + 200, z Int32 DEFAULT x - y) ENGINE = MergeTree() ORDER BY y"
)
for insert_sync in [0, 1]:
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": insert_sync}
2021-09-06 03:11:16 +00:00
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[2, 102, 104], [4, 104, 108]]
)
2023-03-23 12:48:02 +00:00
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 101, 102], [3, 103, 106]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 101, 102], [2, 102, 104], [3, 103, 106], [4, 104, 108]]
2021-09-06 03:11:16 +00:00
)
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)",
settings=settings,
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV([[2, 22, 24]])
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 11, 12], [3, 33, 36]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 11, 12], [2, 22, 24], [3, 33, 36]]
)
2021-09-06 03:11:16 +00:00
# A materialized column is used in the sharding key expression and `insert_allow_materialized_columns` set to 1.
def test_materialized_column_allow_insert_materialized():
node1.query(
"CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 100, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)"
)
node1.query(
"CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y"
)
for insert_sync in [0, 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": insert_sync,
2021-09-06 03:11:16 +00:00
"insert_allow_materialized_columns": 1,
}
2021-09-06 03:11:16 +00:00
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[2, 102, 104], [4, 104, 108]]
)
2023-03-23 12:48:02 +00:00
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 101, 102], [3, 103, 106]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 101, 102], [2, 102, 104], [3, 103, 106], [4, 104, 108]]
2021-09-06 03:11:16 +00:00
)
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)",
settings=settings,
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV([[2, 22, 24]])
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 11, 12], [3, 33, 36]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 11, 12], [2, 22, 24], [3, 33, 36]]
)
2021-09-06 03:11:16 +00:00
# A materialized column is used in the sharding key expression and `insert_allow_materialized_columns` set to 0.
def test_materialized_column_disallow_insert_materialized():
node1.query(
"CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 100, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)"
)
node1.query(
"CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y"
)
for insert_sync in [0, 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": insert_sync,
2021-09-06 03:11:16 +00:00
"insert_allow_materialized_columns": 0,
}
2021-09-06 03:11:16 +00:00
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[2, 202, -200], [4, 204, -200]]
)
2023-03-23 12:48:02 +00:00
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 201, -200], [3, 203, -200]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 201, -200], [2, 202, -200], [3, 203, -200], [4, 204, -200]]
2021-09-06 03:11:16 +00:00
)
# INSERT INTO TABLE dist (x, y)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
expected_error = "Cannot insert column y, because it is MATERIALIZED column"
assert expected_error in node1.query_and_get_error(
"INSERT INTO TABLE dist (x, y) VALUES (1, 11), (2, 22), (3, 33)",
settings=settings,
)
# Almost the same as the previous test `test_materialized_column_disallow_insert_materialized`, but the sharding key has different values.
def test_materialized_column_disallow_insert_materialized_different_shards():
node1.query(
"CREATE TABLE dist ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 101, z Int32 MATERIALIZED x + y) ENGINE = Distributed('test_cluster', currentDatabase(), local, y)"
)
node1.query(
"CREATE TABLE local ON CLUSTER 'test_cluster' (x Int32, y Int32 MATERIALIZED x + 200, z Int32 MATERIALIZED x - y) ENGINE = MergeTree() ORDER BY y"
)
for insert_sync in [0, 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": insert_sync,
2021-09-06 03:11:16 +00:00
"insert_allow_materialized_columns": 0,
}
2021-09-06 03:11:16 +00:00
# INSERT INTO TABLE dist (x)
node1.query("TRUNCATE TABLE local ON CLUSTER 'test_cluster'")
node1.query(
"INSERT INTO TABLE dist (x) VALUES (1), (2), (3), (4)", settings=settings
)
node1.query("SYSTEM FLUSH DISTRIBUTED dist")
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[1, 201, -200], [3, 203, -200]]
)
2023-03-23 12:48:02 +00:00
assert node2.query("SELECT x, y, z FROM local ORDER BY x") == TSV(
2021-09-06 03:11:16 +00:00
[[2, 202, -200], [4, 204, -200]]
)
2023-03-23 12:48:02 +00:00
assert node1.query("SELECT x, y, z FROM dist ORDER BY x") == TSV(
[[1, 201, -200], [2, 202, -200], [3, 203, -200], [4, 204, -200]]
2021-09-06 03:11:16 +00:00
)