mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Fix inconsistent parts remove from local FS without dropping them from ZK
This commit is contained in:
parent
62e18d89ea
commit
dfe214b37c
@ -4616,17 +4616,19 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(
|
||||
zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retried)
|
||||
{
|
||||
std::vector<std::future<Coordination::ExistsResponse>> exists_futures;
|
||||
exists_futures.reserve(part_names.size());
|
||||
for (const String & part_name : part_names)
|
||||
{
|
||||
String part_path = replica_path + "/parts/" + part_name;
|
||||
exists_futures.emplace_back(zookeeper->asyncExists(part_path));
|
||||
}
|
||||
|
||||
std::vector<std::future<Coordination::MultiResponse>> remove_futures;
|
||||
exists_futures.reserve(part_names.size());
|
||||
remove_futures.reserve(part_names.size());
|
||||
try
|
||||
{
|
||||
/// Exception can be thrown from loop
|
||||
/// if zk session will be dropped
|
||||
for (const String & part_name : part_names)
|
||||
{
|
||||
String part_path = replica_path + "/parts/" + part_name;
|
||||
exists_futures.emplace_back(zookeeper->asyncExists(part_path));
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < part_names.size(); ++i)
|
||||
{
|
||||
Coordination::ExistsResponse exists_resp = exists_futures[i].get();
|
||||
|
@ -0,0 +1,14 @@
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<default_database>shard_0</default_database>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
61
dbms/tests/integration/test_parts_delete_zookeeper/test.py
Normal file
61
dbms/tests/integration/test_parts_delete_zookeeper/test.py
Normal file
@ -0,0 +1,61 @@
|
||||
import time
|
||||
import pytest
|
||||
|
||||
from helpers.network import PartitionManager
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
node1.query(
|
||||
'''
|
||||
CREATE DATABASE test;
|
||||
CREATE TABLE test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/replicated', 'node1')
|
||||
ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS old_parts_lifetime=4, cleanup_delay_period=1;
|
||||
'''
|
||||
)
|
||||
|
||||
yield cluster
|
||||
|
||||
except Exception as ex:
|
||||
print ex
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
# Test that outdated parts are not removed when they cannot be removed from zookeeper
|
||||
def test_merge_doesnt_work_without_zookeeper(start_cluster):
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)")
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 5), ('2018-10-03', 6)")
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "2\n"
|
||||
|
||||
node1.query("OPTIMIZE TABLE test_table FINAL")
|
||||
assert node1.query("SELECT count(*) from system.parts") == "3\n"
|
||||
|
||||
assert_eq_with_retry(node1, "SELECT count(*) from system.parts", "1")
|
||||
|
||||
node1.query("TRUNCATE TABLE test_table")
|
||||
|
||||
assert node1.query("SELECT count(*) from system.parts") == "0\n"
|
||||
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)")
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 5), ('2018-10-03', 6)")
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "2\n"
|
||||
|
||||
with PartitionManager() as pm:
|
||||
node1.query("OPTIMIZE TABLE test_table FINAL")
|
||||
pm.drop_instance_zk_connections(node1)
|
||||
time.sleep(10) # > old_parts_lifetime
|
||||
assert node1.query("SELECT count(*) from system.parts") == "3\n"
|
||||
|
||||
assert_eq_with_retry(node1, "SELECT count(*) from system.parts", "1")
|
Loading…
Reference in New Issue
Block a user