2021-01-27 10:07:18 +00:00
|
|
|
import pytest
|
|
|
|
|
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
from multiprocessing.dummy import Pool
|
2021-03-01 13:42:31 +00:00
|
|
|
from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk
|
2021-01-27 10:07:18 +00:00
|
|
|
import time
|
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
|
|
|
|
|
|
|
node1 = cluster.add_instance("node1", with_zookeeper=True)
|
|
|
|
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
|
|
|
|
|
|
|
def test_merge_and_part_corruption(started_cluster):
|
2021-10-19 17:39:44 +00:00
|
|
|
node1.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2021-10-19 17:39:44 +00:00
|
|
|
CREATE TABLE replicated_mt(date Date, id UInt32, value Int32)
|
2023-08-14 10:06:57 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') ORDER BY id
|
2023-05-22 17:07:18 +00:00
|
|
|
SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0;
|
2021-10-19 17:39:44 +00:00
|
|
|
""".format(
|
|
|
|
replica=node1.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2021-10-19 17:39:44 +00:00
|
|
|
|
2021-01-27 10:07:18 +00:00
|
|
|
node1.query("SYSTEM STOP REPLICATION QUEUES replicated_mt")
|
|
|
|
for i in range(4):
|
|
|
|
node1.query(
|
|
|
|
"INSERT INTO replicated_mt SELECT toDate('2019-10-01'), number, number * number FROM numbers ({f}, 100000)".format(
|
|
|
|
f=i * 100000
|
|
|
|
)
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-01-27 10:07:18 +00:00
|
|
|
assert (
|
|
|
|
node1.query(
|
|
|
|
"SELECT COUNT() FROM system.parts WHERE table='replicated_mt' AND active=1"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2021-01-27 10:07:18 +00:00
|
|
|
== "4\n"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2021-01-27 10:07:18 +00:00
|
|
|
|
|
|
|
# Need to corrupt "border part" (left or right). If we will corrupt something in the middle
|
|
|
|
# clickhouse will not consider merge as broken, because we have parts with the same min and max
|
|
|
|
# block numbers.
|
2021-03-01 13:42:31 +00:00
|
|
|
corrupt_part_data_on_disk(node1, "replicated_mt", "all_3_3_0")
|
2021-01-27 10:07:18 +00:00
|
|
|
|
|
|
|
with Pool(1) as p:
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-01-27 10:07:18 +00:00
|
|
|
def optimize_with_delay(x):
|
|
|
|
node1.query("OPTIMIZE TABLE replicated_mt FINAL", timeout=30)
|
|
|
|
|
|
|
|
# corrupt part after merge already assigned, but not started
|
|
|
|
res_opt = p.apply_async(optimize_with_delay, (1,))
|
|
|
|
node1.query(
|
2023-08-14 10:06:57 +00:00
|
|
|
"CHECK TABLE replicated_mt",
|
|
|
|
settings={"check_query_single_value_result": 0, "max_threads": 1},
|
2021-01-27 10:07:18 +00:00
|
|
|
)
|
|
|
|
# start merge
|
|
|
|
node1.query("SYSTEM START REPLICATION QUEUES replicated_mt")
|
|
|
|
res_opt.get()
|
|
|
|
|
|
|
|
# will hung if checked bug not fixed
|
|
|
|
node1.query(
|
|
|
|
"ALTER TABLE replicated_mt UPDATE value = 7 WHERE 1",
|
|
|
|
settings={"mutations_sync": 2},
|
|
|
|
timeout=30,
|
|
|
|
)
|
|
|
|
assert node1.query("SELECT sum(value) FROM replicated_mt") == "2100000\n"
|
2021-10-19 17:39:44 +00:00
|
|
|
|
|
|
|
node1.query("DROP TABLE replicated_mt SYNC")
|