ClickHouse/tests/integration/test_quorum_inserts/test.py

298 lines
13 KiB
Python
Raw Normal View History

2020-04-06 20:34:04 +00:00
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
2020-04-06 20:34:04 +00:00
cluster = ClickHouseCluster(__file__)
zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"],
2020-09-21 21:09:50 +00:00
main_configs=["configs/config.d/remote_servers.xml"],
2020-04-10 21:29:54 +00:00
macros={"cluster": "anime", "shard": "0", "replica": "zero"},
with_zookeeper=True)
first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"],
2020-09-21 21:09:50 +00:00
main_configs=["configs/config.d/remote_servers.xml"],
2020-04-10 21:29:54 +00:00
macros={"cluster": "anime", "shard": "0", "replica": "first"},
with_zookeeper=True)
second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"],
2020-09-21 21:09:50 +00:00
main_configs=["configs/config.d/remote_servers.xml"],
2020-04-10 21:29:54 +00:00
macros={"cluster": "anime", "shard": "0", "replica": "second"},
with_zookeeper=True)
2020-04-06 20:34:04 +00:00
@pytest.fixture(scope="module")
def started_cluster():
global cluster
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
2020-04-10 21:29:54 +00:00
def test_simple_add_replica(started_cluster):
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
create_query = "CREATE TABLE test_simple " \
2020-04-06 20:34:04 +00:00
"(a Int8, d Date) " \
2020-04-10 21:29:54 +00:00
"Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \
2020-04-06 20:34:04 +00:00
"PARTITION BY d ORDER BY a"
2020-04-10 21:29:54 +00:00
zero.query(create_query)
first.query(create_query)
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
first.query("SYSTEM STOP FETCHES test_simple")
2020-04-06 20:34:04 +00:00
zero.query("INSERT INTO test_simple VALUES (1, '2011-01-01')", settings={'insert_quorum': 1})
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple")
assert '' == first.query("SELECT * from test_simple")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
first.query("SYSTEM START FETCHES test_simple")
2020-04-06 20:34:04 +00:00
2020-04-14 18:05:10 +00:00
first.query("SYSTEM SYNC REPLICA test_simple", timeout=20)
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple")
assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
second.query(create_query)
second.query("SYSTEM SYNC REPLICA test_simple", timeout=20)
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
assert '1\t2011-01-01\n' == zero.query("SELECT * from test_simple")
assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple")
assert '1\t2011-01-01\n' == second.query("SELECT * from test_simple")
2020-04-06 20:34:04 +00:00
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
def test_drop_replica_and_achieve_quorum(started_cluster):
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster")
2020-04-10 21:29:54 +00:00
create_query = "CREATE TABLE test_drop_replica_and_achieve_quorum " \
"(a Int8, d Date) " \
"Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \
"PARTITION BY d ORDER BY a"
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
print("Create Replicated table with two replicas")
zero.query(create_query)
first.query(create_query)
print("Stop fetches on one replica. Since that, it will be isolated.")
first.query("SYSTEM STOP FETCHES test_drop_replica_and_achieve_quorum")
print("Insert to other replica. This query will fail.")
quorum_timeout = zero.query_and_get_error(
"INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (1, '2011-01-01')",
settings={'insert_quorum_timeout': 5000})
2020-04-10 21:29:54 +00:00
assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail."
assert TSV("1\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum",
settings={'select_sequential_consistency': 0}))
2020-04-10 21:29:54 +00:00
assert TSV("") == TSV(zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum",
settings={'select_sequential_consistency': 1}))
2020-04-10 21:29:54 +00:00
# TODO:(Mikhaylov) begin; maybe delete this lines. I want clickhouse to fetch parts and update quorum.
2020-04-10 21:29:54 +00:00
print("START FETCHES first replica")
first.query("SYSTEM START FETCHES test_drop_replica_and_achieve_quorum")
2020-04-06 20:34:04 +00:00
print("SYNC first replica")
2020-04-10 21:29:54 +00:00
first.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20)
# TODO:(Mikhaylov) end
2020-04-10 21:29:54 +00:00
print("Add second replica")
second.query(create_query)
2020-04-06 20:34:04 +00:00
print("SYNC second replica")
2020-04-10 21:29:54 +00:00
second.query("SYSTEM SYNC REPLICA test_drop_replica_and_achieve_quorum", timeout=20)
2020-04-06 20:34:04 +00:00
print("Quorum for previous insert achieved.")
2020-04-10 21:29:54 +00:00
assert TSV("1\t2011-01-01\n") == TSV(second.query("SELECT * FROM test_drop_replica_and_achieve_quorum",
settings={'select_sequential_consistency': 1}))
2020-04-06 20:34:04 +00:00
print("Now we can insert some other data.")
2020-04-10 21:29:54 +00:00
zero.query("INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')")
2020-04-06 20:34:04 +00:00
assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(
zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a"))
assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(
first.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a"))
assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV(
second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a"))
2020-04-06 20:34:04 +00:00
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
@pytest.mark.parametrize(
('add_new_data'),
[
False,
True
]
)
def test_insert_quorum_with_drop_partition(started_cluster, add_new_data):
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-09-21 21:09:50 +00:00
create_query = "CREATE TABLE test_quorum_insert_with_drop_partition ON CLUSTER cluster " \
2020-04-10 21:29:54 +00:00
"(a Int8, d Date) " \
2020-09-21 21:09:50 +00:00
"Engine = ReplicatedMergeTree " \
2020-04-10 21:29:54 +00:00
"PARTITION BY d ORDER BY a "
2020-04-06 20:34:04 +00:00
2020-04-13 15:21:05 +00:00
print("Create Replicated table with three replicas")
2020-04-10 21:29:54 +00:00
zero.query(create_query)
print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.")
first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition")
2020-04-06 20:34:04 +00:00
print("Insert with quorum. (zero and second)")
2020-04-10 21:29:54 +00:00
zero.query("INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(1, '2011-01-01')")
2020-04-06 20:34:04 +00:00
print("Drop partition.")
2020-04-10 21:29:54 +00:00
zero.query("ALTER TABLE test_quorum_insert_with_drop_partition DROP PARTITION '2011-01-01'")
if (add_new_data):
print("Insert to deleted partition")
zero.query("INSERT INTO test_quorum_insert_with_drop_partition(a,d) VALUES(2, '2011-01-01')")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
print("Resume fetches for test_quorum_insert_with_drop_partition at first replica.")
first.query("SYSTEM START FETCHES test_quorum_insert_with_drop_partition")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
print("Sync first replica with others.")
first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition")
2020-09-21 21:09:50 +00:00
assert "20110101" not in first.query("""
WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_quorum_insert_with_drop_partition') AS uuid,
'/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p
SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical
""")
2020-04-06 20:34:04 +00:00
print("Select from updated partition.")
2020-04-10 21:29:54 +00:00
if (add_new_data):
assert TSV("2\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
assert TSV("2\t2011-01-01\n") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
else:
assert TSV("") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
assert TSV("") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition"))
2020-04-06 20:34:04 +00:00
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-04-13 15:21:05 +00:00
@pytest.mark.parametrize(
('add_new_data'),
[
False,
True
]
)
def test_insert_quorum_with_move_partition(started_cluster, add_new_data):
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster")
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster")
2020-04-13 15:21:05 +00:00
2020-09-21 21:09:50 +00:00
create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source ON CLUSTER cluster " \
2020-04-13 15:21:05 +00:00
"(a Int8, d Date) " \
2020-09-21 21:09:50 +00:00
"Engine = ReplicatedMergeTree " \
2020-04-13 15:21:05 +00:00
"PARTITION BY d ORDER BY a "
2020-09-21 21:09:50 +00:00
create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination ON CLUSTER cluster " \
"(a Int8, d Date) " \
2020-09-21 21:09:50 +00:00
"Engine = ReplicatedMergeTree " \
"PARTITION BY d ORDER BY a "
2020-04-13 15:21:05 +00:00
print("Create source Replicated table with three replicas")
zero.query(create_source)
print("Create destination Replicated table with three replicas")
zero.query(create_destination)
print("Stop fetches for test_insert_quorum_with_move_partition_source at first replica.")
first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source")
print("Insert with quorum. (zero and second)")
zero.query("INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(1, '2011-01-01')")
print("Drop partition.")
zero.query(
"ALTER TABLE test_insert_quorum_with_move_partition_source MOVE PARTITION '2011-01-01' TO TABLE test_insert_quorum_with_move_partition_destination")
2020-04-13 15:21:05 +00:00
if (add_new_data):
print("Insert to deleted partition")
zero.query("INSERT INTO test_insert_quorum_with_move_partition_source(a,d) VALUES(2, '2011-01-01')")
print("Resume fetches for test_insert_quorum_with_move_partition_source at first replica.")
first.query("SYSTEM START FETCHES test_insert_quorum_with_move_partition_source")
print("Sync first replica with others.")
first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source")
2020-09-21 21:09:50 +00:00
assert "20110101" not in first.query("""
WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_insert_quorum_with_move_partition_source') AS uuid,
'/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p
SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical
""")
2020-04-13 15:21:05 +00:00
print("Select from updated partition.")
if (add_new_data):
assert TSV("2\t2011-01-01\n") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
assert TSV("2\t2011-01-01\n") == TSV(
second.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
2020-04-13 15:21:05 +00:00
else:
assert TSV("") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
assert TSV("") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source"))
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster")
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster")
2020-04-13 15:21:05 +00:00
2020-04-06 20:34:04 +00:00
def test_insert_quorum_with_ttl(started_cluster):
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster")
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
create_query = "CREATE TABLE test_insert_quorum_with_ttl " \
"(a Int8, d Date) " \
"Engine = ReplicatedMergeTree('/clickhouse/tables/{table}', '{replica}') " \
"PARTITION BY d ORDER BY a " \
2020-06-16 19:10:25 +00:00
"TTL d + INTERVAL 5 second DELETE WHERE toYear(d) = 2011 " \
2020-04-10 21:29:54 +00:00
"SETTINGS merge_with_ttl_timeout=2 "
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
print("Create Replicated table with two replicas")
zero.query(create_query)
first.query(create_query)
2020-04-06 20:34:04 +00:00
2020-04-10 21:29:54 +00:00
print("Stop fetches for test_insert_quorum_with_ttl at first replica.")
first.query("SYSTEM STOP FETCHES test_insert_quorum_with_ttl")
2020-04-06 20:34:04 +00:00
print("Insert should fail since it can not reach the quorum.")
2020-04-10 21:29:54 +00:00
quorum_timeout = zero.query_and_get_error("INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')",
settings={'insert_quorum_timeout': 5000})
2020-04-06 20:34:04 +00:00
assert "Timeout while waiting for quorum" in quorum_timeout, "Query must fail."
2020-04-10 21:29:54 +00:00
print("Wait 10 seconds and TTL merge have to be executed. But it won't delete data.")
2020-04-06 20:34:04 +00:00
time.sleep(10)
assert TSV("1\t2011-01-01\n") == TSV(
zero.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 0}))
2020-04-10 21:29:54 +00:00
print("Resume fetches for test_insert_quorum_with_ttl at first replica.")
first.query("SYSTEM START FETCHES test_insert_quorum_with_ttl")
print("Sync first replica.")
first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl")
zero.query("INSERT INTO test_insert_quorum_with_ttl(a,d) VALUES(1, '2011-01-01')",
settings={'insert_quorum_timeout': 5000})
2020-04-06 20:34:04 +00:00
print("Inserts should resume.")
2020-04-10 21:29:54 +00:00
zero.query("INSERT INTO test_insert_quorum_with_ttl(a, d) VALUES(2, '2012-02-02')")
2020-06-16 02:56:50 +00:00
first.query("OPTIMIZE TABLE test_insert_quorum_with_ttl")
first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl")
zero.query("SYSTEM SYNC REPLICA test_insert_quorum_with_ttl")
assert TSV("2\t2012-02-02\n") == TSV(
first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 0}))
assert TSV("2\t2012-02-02\n") == TSV(
first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 1}))
2020-06-16 02:56:50 +00:00
2020-09-21 21:09:50 +00:00
zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster")