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.

147 lines
4.5 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,
2023-06-15 13:05:17 +00:00
main_configs=[
"configs/compat.xml",
],
)
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,
2023-06-15 13:05:17 +00:00
main_configs=[
"configs/compat.xml",
],
)
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,
2023-06-15 13:05:17 +00:00
main_configs=[
"configs/compat.xml",
],
)
2020-03-13 17:23:36 +00:00
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop Since there can be some leftovers: 2023.07.24 07:08:25.238066 [ 140 ] {} <Error> Application: Code: 219. DB::Exception: Cannot drop: filesystem error: in remove: Directory not empty ["/var/lib/clickhouse/data/system/"]. Probably database contain some detached tables or metadata leftovers from Ordinary engine. If you want to remove all data anyway, try to attach database back and drop it again with enabled force_remove_data_recursively_on_drop setting: Exception while trying to convert database system from Ordinary to Atomic. It may be in some intermediate state. You can finish conversion manually by moving the rest tables from system to .tmp_convert.system.9396432095832455195 (using RENAME TABLE) and executing DROP DATABASE system and RENAME DATABASE .tmp_convert.system.9396432095832455195 TO system. (DATABASE_NOT_EMPTY), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e68af57 in /usr/bin/clickhouse 1. ? @ 0x000000000cab443c in /usr/bin/clickhouse 2. DB::DatabaseOnDisk::drop(std::shared_ptr<DB::Context const>) @ 0x000000001328d617 in /usr/bin/clickhouse 3. DB::DatabaseCatalog::detachDatabase(std::shared_ptr<DB::Context const>, String const&, bool, bool) @ 0x0000000013524a6c in /usr/bin/clickhouse 4. DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::shared_ptr<DB::IDatabase>&, std::vector<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>, std::allocator<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>>>&) @ 0x0000000013bc05e4 in /usr/bin/clickhouse 5. DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) @ 0x0000000013bbc6b8 in /usr/bin/clickhouse 6. DB::InterpreterDropQuery::execute() @ 0x0000000013bbba22 in /usr/bin/clickhouse 7. ? @ 0x00000000140b13a5 in /usr/bin/clickhouse 8. DB::executeQuery(String const&, std::shared_ptr<DB::Context>, bool, DB::QueryProcessingStage::Enum) @ 0x00000000140ad20e in /usr/bin/clickhouse 9. ? @ 0x00000000140d2ef0 in /usr/bin/clickhouse 10. DB::maybeConvertSystemDatabase(std::shared_ptr<DB::Context>) @ 0x00000000140d0aaf in /usr/bin/clickhouse 11. DB::Server::main(std::vector<String, std::allocator<String>> const&) @ 0x000000000e724e55 in /usr/bin/clickhouse 12. Poco::Util::Application::run() @ 0x0000000017ead086 in /usr/bin/clickhouse 13. DB::Server::run() @ 0x000000000e714a5d in /usr/bin/clickhouse 14. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000017ec07b9 in /usr/bin/clickhouse 15. mainEntryClickHouseServer(int, char**) @ 0x000000000e711a26 in /usr/bin/clickhouse 16. main @ 0x0000000008cf13cf in /usr/bin/clickhouse 17. __libc_start_main @ 0x0000000000021b97 in /lib/x86_64-linux-gnu/libc-2.27.so 18. _start @ 0x00000000080705ae in /usr/bin/clickhouse (version 23.7.1.2012) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-07-24 08:14:23 +00:00
def restart_node(node):
# set force_remove_data_recursively_on_drop (cannot be done before, because the version is too old)
node.put_users_config("configs/force_remove_data_recursively_on_drop.xml")
node.restart_with_latest_version(signal=9, fix_metadata=True)
2020-03-13 17:23:36 +00:00
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
Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop Since there can be some leftovers: 2023.07.24 07:08:25.238066 [ 140 ] {} <Error> Application: Code: 219. DB::Exception: Cannot drop: filesystem error: in remove: Directory not empty ["/var/lib/clickhouse/data/system/"]. Probably database contain some detached tables or metadata leftovers from Ordinary engine. If you want to remove all data anyway, try to attach database back and drop it again with enabled force_remove_data_recursively_on_drop setting: Exception while trying to convert database system from Ordinary to Atomic. It may be in some intermediate state. You can finish conversion manually by moving the rest tables from system to .tmp_convert.system.9396432095832455195 (using RENAME TABLE) and executing DROP DATABASE system and RENAME DATABASE .tmp_convert.system.9396432095832455195 TO system. (DATABASE_NOT_EMPTY), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e68af57 in /usr/bin/clickhouse 1. ? @ 0x000000000cab443c in /usr/bin/clickhouse 2. DB::DatabaseOnDisk::drop(std::shared_ptr<DB::Context const>) @ 0x000000001328d617 in /usr/bin/clickhouse 3. DB::DatabaseCatalog::detachDatabase(std::shared_ptr<DB::Context const>, String const&, bool, bool) @ 0x0000000013524a6c in /usr/bin/clickhouse 4. DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::shared_ptr<DB::IDatabase>&, std::vector<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>, std::allocator<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>>>&) @ 0x0000000013bc05e4 in /usr/bin/clickhouse 5. DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) @ 0x0000000013bbc6b8 in /usr/bin/clickhouse 6. DB::InterpreterDropQuery::execute() @ 0x0000000013bbba22 in /usr/bin/clickhouse 7. ? @ 0x00000000140b13a5 in /usr/bin/clickhouse 8. DB::executeQuery(String const&, std::shared_ptr<DB::Context>, bool, DB::QueryProcessingStage::Enum) @ 0x00000000140ad20e in /usr/bin/clickhouse 9. ? @ 0x00000000140d2ef0 in /usr/bin/clickhouse 10. DB::maybeConvertSystemDatabase(std::shared_ptr<DB::Context>) @ 0x00000000140d0aaf in /usr/bin/clickhouse 11. DB::Server::main(std::vector<String, std::allocator<String>> const&) @ 0x000000000e724e55 in /usr/bin/clickhouse 12. Poco::Util::Application::run() @ 0x0000000017ead086 in /usr/bin/clickhouse 13. DB::Server::run() @ 0x000000000e714a5d in /usr/bin/clickhouse 14. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000017ec07b9 in /usr/bin/clickhouse 15. mainEntryClickHouseServer(int, char**) @ 0x000000000e711a26 in /usr/bin/clickhouse 16. main @ 0x0000000008cf13cf in /usr/bin/clickhouse 17. __libc_start_main @ 0x0000000000021b97 in /lib/x86_64-linux-gnu/libc-2.27.so 18. _start @ 0x00000000080705ae in /usr/bin/clickhouse (version 23.7.1.2012) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-07-24 08:14:23 +00:00
restart_node(node1)
restart_node(node2)
# 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 ORDER BY id") == "1\n4\n"
assert node2.query("SELECT id FROM mt ORDER BY id") == "1\n4\n"
2020-03-13 17:23:36 +00:00
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
# Flush logs before restart to avoid trash from system tables which are on database ordindary
2023-04-14 13:35:39 +00:00
# (We could be in process of creating some system table, which will leave empty directory on restart,
# so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files)
node3.query("SYSTEM FLUSH LOGS")
Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop Since there can be some leftovers: 2023.07.24 07:08:25.238066 [ 140 ] {} <Error> Application: Code: 219. DB::Exception: Cannot drop: filesystem error: in remove: Directory not empty ["/var/lib/clickhouse/data/system/"]. Probably database contain some detached tables or metadata leftovers from Ordinary engine. If you want to remove all data anyway, try to attach database back and drop it again with enabled force_remove_data_recursively_on_drop setting: Exception while trying to convert database system from Ordinary to Atomic. It may be in some intermediate state. You can finish conversion manually by moving the rest tables from system to .tmp_convert.system.9396432095832455195 (using RENAME TABLE) and executing DROP DATABASE system and RENAME DATABASE .tmp_convert.system.9396432095832455195 TO system. (DATABASE_NOT_EMPTY), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e68af57 in /usr/bin/clickhouse 1. ? @ 0x000000000cab443c in /usr/bin/clickhouse 2. DB::DatabaseOnDisk::drop(std::shared_ptr<DB::Context const>) @ 0x000000001328d617 in /usr/bin/clickhouse 3. DB::DatabaseCatalog::detachDatabase(std::shared_ptr<DB::Context const>, String const&, bool, bool) @ 0x0000000013524a6c in /usr/bin/clickhouse 4. DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::shared_ptr<DB::IDatabase>&, std::vector<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>, std::allocator<StrongTypedef<wide::integer<128ul, unsigned int>, DB::UUIDTag>>>&) @ 0x0000000013bc05e4 in /usr/bin/clickhouse 5. DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) @ 0x0000000013bbc6b8 in /usr/bin/clickhouse 6. DB::InterpreterDropQuery::execute() @ 0x0000000013bbba22 in /usr/bin/clickhouse 7. ? @ 0x00000000140b13a5 in /usr/bin/clickhouse 8. DB::executeQuery(String const&, std::shared_ptr<DB::Context>, bool, DB::QueryProcessingStage::Enum) @ 0x00000000140ad20e in /usr/bin/clickhouse 9. ? @ 0x00000000140d2ef0 in /usr/bin/clickhouse 10. DB::maybeConvertSystemDatabase(std::shared_ptr<DB::Context>) @ 0x00000000140d0aaf in /usr/bin/clickhouse 11. DB::Server::main(std::vector<String, std::allocator<String>> const&) @ 0x000000000e724e55 in /usr/bin/clickhouse 12. Poco::Util::Application::run() @ 0x0000000017ead086 in /usr/bin/clickhouse 13. DB::Server::run() @ 0x000000000e714a5d in /usr/bin/clickhouse 14. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000017ec07b9 in /usr/bin/clickhouse 15. mainEntryClickHouseServer(int, char**) @ 0x000000000e711a26 in /usr/bin/clickhouse 16. main @ 0x0000000008cf13cf in /usr/bin/clickhouse 17. __libc_start_main @ 0x0000000000021b97 in /lib/x86_64-linux-gnu/libc-2.27.so 18. _start @ 0x00000000080705ae in /usr/bin/clickhouse (version 23.7.1.2012) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-07-24 08:14:23 +00:00
restart_node(node3)
# 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")