2021-01-25 14:10:18 +00:00
|
|
|
import pytest
|
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
import random
|
|
|
|
import string
|
|
|
|
import os
|
|
|
|
import time
|
|
|
|
from multiprocessing.dummy import Pool
|
2021-01-28 12:07:26 +00:00
|
|
|
from helpers.network import PartitionManager
|
2021-01-25 14:10:18 +00:00
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
|
|
|
node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'])
|
|
|
|
node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'])
|
|
|
|
node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'])
|
|
|
|
|
|
|
|
from kazoo.client import KazooClient
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
2021-01-28 12:07:26 +00:00
|
|
|
def smaller_exception(ex):
|
|
|
|
return '\n'.join(str(ex).split('\n')[0:2])
|
2021-01-25 14:10:18 +00:00
|
|
|
|
|
|
|
def test_simple_replicated_table(started_cluster):
|
|
|
|
|
|
|
|
for i, node in enumerate([node1, node2, node3]):
|
|
|
|
node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1))
|
|
|
|
|
|
|
|
node2.query("INSERT INTO t SELECT number FROM numbers(10)")
|
|
|
|
|
|
|
|
node1.query("SYSTEM SYNC REPLICA t", timeout=10)
|
|
|
|
node3.query("SYSTEM SYNC REPLICA t", timeout=10)
|
|
|
|
|
|
|
|
assert node1.query("SELECT COUNT() FROM t") == "10\n"
|
|
|
|
assert node2.query("SELECT COUNT() FROM t") == "10\n"
|
|
|
|
assert node3.query("SELECT COUNT() FROM t") == "10\n"
|
2021-01-28 12:07:26 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
def test_blocade_leader(started_cluster):
|
|
|
|
for i, node in enumerate([node1, node2, node3]):
|
|
|
|
node.query("CREATE TABLE t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1))
|
|
|
|
|
|
|
|
node2.query("INSERT INTO t1 SELECT number FROM numbers(10)")
|
|
|
|
|
|
|
|
node1.query("SYSTEM SYNC REPLICA t1", timeout=10)
|
|
|
|
node3.query("SYSTEM SYNC REPLICA t1", timeout=10)
|
|
|
|
|
|
|
|
assert node1.query("SELECT COUNT() FROM t1") == "10\n"
|
|
|
|
assert node2.query("SELECT COUNT() FROM t1") == "10\n"
|
|
|
|
assert node3.query("SELECT COUNT() FROM t1") == "10\n"
|
|
|
|
|
|
|
|
with PartitionManager() as pm:
|
|
|
|
pm.partition_instances(node2, node1)
|
|
|
|
pm.partition_instances(node3, node1)
|
|
|
|
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-01 07:51:10 +00:00
|
|
|
node2.query("SYSTEM RESTART REPLICA t1")
|
2021-01-28 12:07:26 +00:00
|
|
|
node2.query("INSERT INTO t1 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node2.query("ATTACH TABLE t1")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node2", smaller_exception(attach_ex))
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node2", smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot insert anything node2"
|
|
|
|
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-01 07:51:10 +00:00
|
|
|
node3.query("SYSTEM RESTART REPLICA t1")
|
2021-01-28 12:07:26 +00:00
|
|
|
node3.query("INSERT INTO t1 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node3.query("ATTACH TABLE t1")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node3", smaller_exception(attach_ex))
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node3", smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot insert anything node3"
|
|
|
|
|
|
|
|
for n, node in enumerate([node1, node2, node3]):
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
|
|
|
node.query("SYSTEM RESTART REPLICA t1")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node.query("ATTACH TABLE t1")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
|
|
|
|
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node{}".format(n + 1)
|
|
|
|
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
|
|
|
node1.query("INSERT INTO t1 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
|
|
|
print("Got exception node1", smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot insert anything node1"
|
|
|
|
|
|
|
|
for n, node in enumerate([node1, node2, node3]):
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-02 19:56:07 +00:00
|
|
|
node.query("SYSTEM RESTART REPLICA t1", timeout=10)
|
2021-01-28 12:07:26 +00:00
|
|
|
node.query("SYSTEM SYNC REPLICA t1", timeout=10)
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot sync replica node{}".format(n+1)
|
|
|
|
|
|
|
|
assert node1.query("SELECT COUNT() FROM t1") == "310\n"
|
|
|
|
assert node2.query("SELECT COUNT() FROM t1") == "310\n"
|
|
|
|
assert node3.query("SELECT COUNT() FROM t1") == "310\n"
|
|
|
|
|
|
|
|
|
|
|
|
def test_blocade_leader_twice(started_cluster):
|
|
|
|
for i, node in enumerate([node1, node2, node3]):
|
|
|
|
node.query("CREATE TABLE t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1))
|
|
|
|
|
|
|
|
node2.query("INSERT INTO t2 SELECT number FROM numbers(10)")
|
|
|
|
|
|
|
|
node1.query("SYSTEM SYNC REPLICA t2", timeout=10)
|
|
|
|
node3.query("SYSTEM SYNC REPLICA t2", timeout=10)
|
|
|
|
|
|
|
|
assert node1.query("SELECT COUNT() FROM t2") == "10\n"
|
|
|
|
assert node2.query("SELECT COUNT() FROM t2") == "10\n"
|
|
|
|
assert node3.query("SELECT COUNT() FROM t2") == "10\n"
|
|
|
|
|
|
|
|
with PartitionManager() as pm:
|
|
|
|
pm.partition_instances(node2, node1)
|
|
|
|
pm.partition_instances(node3, node1)
|
|
|
|
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-01 07:51:10 +00:00
|
|
|
node2.query("SYSTEM RESTART REPLICA t2")
|
2021-01-28 12:07:26 +00:00
|
|
|
node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node2.query("ATTACH TABLE t2")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node2", smaller_exception(attach_ex))
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node2", smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node2"
|
|
|
|
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-01 07:51:10 +00:00
|
|
|
node3.query("SYSTEM RESTART REPLICA t2")
|
2021-01-28 12:07:26 +00:00
|
|
|
node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node3.query("ATTACH TABLE t2")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node3", smaller_exception(attach_ex))
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node3", smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node3"
|
|
|
|
|
|
|
|
|
|
|
|
# Total network partition
|
|
|
|
pm.partition_instances(node3, node2)
|
|
|
|
|
2021-02-02 08:02:25 +00:00
|
|
|
for i in range(10):
|
2021-01-28 12:07:26 +00:00
|
|
|
try:
|
|
|
|
node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)")
|
|
|
|
assert False, "Node3 became leader?"
|
|
|
|
except Exception as ex:
|
|
|
|
time.sleep(0.5)
|
|
|
|
|
2021-02-02 08:02:25 +00:00
|
|
|
for i in range(10):
|
2021-01-28 12:07:26 +00:00
|
|
|
try:
|
|
|
|
node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)")
|
|
|
|
assert False, "Node2 became leader?"
|
|
|
|
except Exception as ex:
|
|
|
|
time.sleep(0.5)
|
|
|
|
|
|
|
|
|
|
|
|
for n, node in enumerate([node1, node2, node3]):
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
|
|
|
node.query("SYSTEM RESTART REPLICA t2")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
2021-02-01 07:51:10 +00:00
|
|
|
try:
|
|
|
|
node.query("ATTACH TABLE t2")
|
|
|
|
except Exception as attach_ex:
|
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(attach_ex))
|
|
|
|
|
2021-01-28 12:07:26 +00:00
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node{}".format(n + 1)
|
|
|
|
|
|
|
|
for n, node in enumerate([node1, node2, node3]):
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
|
|
|
node.query("INSERT INTO t2 SELECT rand() FROM numbers(100)")
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node{}".format(n + 1)
|
|
|
|
|
|
|
|
for node in [node1, node2, node3]:
|
|
|
|
for i in range(100):
|
|
|
|
try:
|
2021-02-02 19:33:57 +00:00
|
|
|
node.query("SYSTEM RESTART REPLICA t2", timeout=10)
|
2021-01-28 12:07:26 +00:00
|
|
|
node.query("SYSTEM SYNC REPLICA t2", timeout=10)
|
|
|
|
break
|
|
|
|
except Exception as ex:
|
|
|
|
print("Got exception node{}".format(n + 1), smaller_exception(ex))
|
|
|
|
time.sleep(0.5)
|
|
|
|
else:
|
|
|
|
assert False, "Cannot reconnect for node{}".format(n + 1)
|
|
|
|
|
|
|
|
assert node1.query("SELECT COUNT() FROM t2") == "510\n"
|
2021-02-02 12:41:09 +00:00
|
|
|
if node2.query("SELECT COUNT() FROM t2") != "510\n":
|
|
|
|
print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical"))
|
|
|
|
print("Replicas")
|
|
|
|
print(node2.query("SELECT * FROM system.replicas FORMAT Vertical"))
|
|
|
|
print("Replica 2 info")
|
|
|
|
print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2' FORMAT Vertical"))
|
|
|
|
print("Queue")
|
|
|
|
print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2/queue' FORMAT Vertical"))
|
|
|
|
print("Log")
|
|
|
|
print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/log' FORMAT Vertical"))
|
2021-01-28 12:07:26 +00:00
|
|
|
assert node2.query("SELECT COUNT() FROM t2") == "510\n"
|
|
|
|
assert node3.query("SELECT COUNT() FROM t2") == "510\n"
|