ClickHouse/tests/integration/test_version_update_after_mutation/test.py

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

127 lines
3.8 KiB
Python
Raw Normal View History

2020-03-13 17:23:36 +00:00
import pytest
import time
2020-03-13 17:23:36 +00:00
from helpers.cluster import ClickHouseCluster
2021-05-15 12:33:01 +00:00
from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry
2020-03-13 17:23:36 +00:00
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
with_zookeeper=True,
image="yandex/clickhouse-server",
2022-08-26 13:33:04 +00:00
tag="20.4.9.110",
with_installed_binary=True,
stay_alive=True,
)
node2 = cluster.add_instance(
"node2",
with_zookeeper=True,
image="yandex/clickhouse-server",
2022-08-26 13:33:04 +00:00
tag="20.4.9.110",
with_installed_binary=True,
stay_alive=True,
)
node3 = cluster.add_instance(
"node3",
with_zookeeper=True,
image="yandex/clickhouse-server",
2022-08-26 13:33:04 +00:00
tag="20.4.9.110",
with_installed_binary=True,
stay_alive=True,
)
2020-03-13 17:23:36 +00:00
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_mutate_and_upgrade(start_cluster):
for node in [node1, node2]:
2021-07-15 12:49:52 +00:00
node.query("DROP TABLE IF EXISTS mt")
node.query(
"CREATE TABLE mt (EventDate Date, id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{}') ORDER BY tuple()".format(
node.name
)
)
2020-03-13 17:23:36 +00:00
node1.query("INSERT INTO mt VALUES ('2020-02-13', 1), ('2020-02-13', 2);")
node1.query("ALTER TABLE mt DELETE WHERE id = 2", settings={"mutations_sync": "2"})
2021-04-29 11:57:48 +00:00
node2.query("SYSTEM SYNC REPLICA mt", timeout=15)
2020-03-13 17:23:36 +00:00
node2.query("DETACH TABLE mt") # stop being leader
node1.query("DETACH TABLE mt") # stop being leader
node1.restart_with_latest_version(signal=9, fix_metadata=True)
node2.restart_with_latest_version(signal=9, fix_metadata=True)
# After hard restart table can be in readonly mode
exec_query_with_retry(
node2, "INSERT INTO mt VALUES ('2020-02-13', 3)", retry_count=60
)
exec_query_with_retry(node1, "SYSTEM SYNC REPLICA mt", retry_count=60)
2020-03-13 17:23:36 +00:00
assert node1.query("SELECT COUNT() FROM mt") == "2\n"
assert node2.query("SELECT COUNT() FROM mt") == "2\n"
node1.query("INSERT INTO mt VALUES ('2020-02-13', 4);")
2021-04-29 11:57:48 +00:00
node2.query("SYSTEM SYNC REPLICA mt", timeout=15)
2020-03-13 17:23:36 +00:00
assert node1.query("SELECT COUNT() FROM mt") == "3\n"
assert node2.query("SELECT COUNT() FROM mt") == "3\n"
node2.query("ALTER TABLE mt DELETE WHERE id = 3", settings={"mutations_sync": "2"})
2021-04-29 11:57:48 +00:00
node1.query("SYSTEM SYNC REPLICA mt", timeout=15)
2020-03-13 17:23:36 +00:00
assert node1.query("SELECT COUNT() FROM mt") == "2\n"
assert node2.query("SELECT COUNT() FROM mt") == "2\n"
node1.query(
"ALTER TABLE mt MODIFY COLUMN id Int32 DEFAULT 0",
settings={"replication_alter_partitions_sync": "2"},
)
2020-03-13 17:23:36 +00:00
node2.query("OPTIMIZE TABLE mt FINAL")
assert node1.query("SELECT id FROM mt") == "1\n4\n"
assert node2.query("SELECT id FROM mt") == "1\n4\n"
2021-07-15 12:49:52 +00:00
for node in [node1, node2]:
node.query("DROP TABLE mt")
2020-03-13 17:23:36 +00:00
def test_upgrade_while_mutation(start_cluster):
2021-07-15 12:49:52 +00:00
node3.query("DROP TABLE IF EXISTS mt1")
node3.query(
"CREATE TABLE mt1 (EventDate Date, id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', 'node3') ORDER BY tuple()"
)
2020-03-13 17:23:36 +00:00
node3.query("INSERT INTO mt1 select '2020-02-13', number from numbers(100000)")
node3.query("SYSTEM STOP MERGES mt1")
2020-03-13 17:23:36 +00:00
node3.query("ALTER TABLE mt1 DELETE WHERE id % 2 == 0")
node3.query("DETACH TABLE mt1") # stop being leader
node3.restart_with_latest_version(signal=9, fix_metadata=True)
# checks for readonly
2021-07-28 10:21:59 +00:00
exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60)
2021-06-28 08:28:45 +00:00
node3.query(
"ALTER TABLE mt1 DELETE WHERE id > 100000", settings={"mutations_sync": "2"}
)
2020-08-27 07:16:07 +00:00
# will delete nothing, but previous async mutation will finish with this query
2020-03-13 17:23:36 +00:00
assert_eq_with_retry(node3, "SELECT COUNT() from mt1", "50000\n")
2021-07-15 12:49:52 +00:00
node3.query("DROP TABLE mt1")