test: table metadata load failure should shutdowns the server properly

This commit is contained in:
serxa 2024-01-24 12:01:32 +00:00
parent c36090ec99
commit 0badcffdc1
2 changed files with 72 additions and 1 deletions

View File

@ -3776,7 +3776,7 @@ class ClickHouseInstance:
except Exception as e:
logging.warning(f"Stop ClickHouse raised an error {e}")
def start_clickhouse(self, start_wait_sec=60, retry_start=True):
def start_clickhouse(self, start_wait_sec=60, retry_start=True, expected_to_fail=False):
if not self.stay_alive:
raise Exception(
"ClickHouse can be started again only with stay_alive=True instance"
@ -3794,10 +3794,15 @@ class ClickHouseInstance:
["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)],
user=str(os.getuid()),
)
if expected_to_fail:
self.wait_start_failed(start_wait_sec + start_time - time.time())
return
time.sleep(1)
continue
else:
logging.debug("Clickhouse process running.")
if expected_to_fail:
raise Exception("ClickHouse was expected not to be running.")
try:
self.wait_start(start_wait_sec + start_time - time.time())
return
@ -3849,6 +3854,28 @@ class ClickHouseInstance:
if last_err is not None:
raise last_err
def wait_start_failed(self, start_wait_sec):
start_time = time.time()
while time.time() <= start_time + start_wait_sec:
pid = self.get_process_pid("clickhouse")
if pid is None:
return
time.sleep(1)
logging.error(
f"No time left to shutdown. Process is still running. Will dump threads."
)
ps_clickhouse = self.exec_in_container(
["bash", "-c", "ps -C clickhouse"], nothrow=True, user="root"
)
logging.info(f"PS RESULT:\n{ps_clickhouse}")
pid = self.get_process_pid("clickhouse")
if pid is not None:
self.exec_in_container(
["bash", "-c", f"gdb -batch -ex 'thread apply all bt full' -p {pid}"],
user="root",
)
raise Exception("ClickHouse server is still running, but was expected to shutdown. Check logs.")
def restart_clickhouse(self, stop_start_wait_sec=60, kill=False):
self.stop_clickhouse(stop_start_wait_sec, kill)
self.start_clickhouse(stop_start_wait_sec)

View File

@ -1398,3 +1398,47 @@ def test_modify_comment(started_cluster):
main_node.query("DROP DATABASE modify_comment_db SYNC")
dummy_node.query("DROP DATABASE modify_comment_db SYNC")
def test_table_metadata_corruption(started_cluster):
main_node.query("DROP DATABASE IF EXISTS table_metadata_corruption")
dummy_node.query("DROP DATABASE IF EXISTS table_metadata_corruption")
main_node.query(
"CREATE DATABASE table_metadata_corruption ENGINE = Replicated('/clickhouse/databases/table_metadata_corruption', 'shard1', 'replica1');"
)
dummy_node.query(
"CREATE DATABASE table_metadata_corruption ENGINE = Replicated('/clickhouse/databases/table_metadata_corruption', 'shard1', 'replica2');"
)
create_some_tables("table_metadata_corruption")
main_node.query("SYSTEM SYNC DATABASE REPLICA table_metadata_corruption")
dummy_node.query("SYSTEM SYNC DATABASE REPLICA table_metadata_corruption")
# Server should handle this by throwing an exception during table loading, which should lead to server shutdown
corrupt = "sed --follow-symlinks -i 's/ReplicatedMergeTree/CorruptedMergeTree/' /var/lib/clickhouse/metadata/table_metadata_corruption/rmt1.sql"
print(f"Corrupting metadata using `{corrupt}`")
dummy_node.stop_clickhouse(kill=True)
dummy_node.exec_in_container(["bash", "-c", corrupt])
query = (
"SELECT name, uuid, create_table_query FROM system.tables WHERE database='table_metadata_corruption' AND name NOT LIKE '.inner_id.%' "
"ORDER BY name SETTINGS show_table_uuid_in_table_create_query_if_not_nil=1"
)
expected = main_node.query(query)
# We expect clickhouse server to shutdown without LOGICAL_ERRORs or deadlocks
dummy_node.start_clickhouse(expected_to_fail=True)
assert not dummy_node.contains_in_log("LOGICAL_ERROR")
fix_corrupt = "sed --follow-symlinks -i 's/CorruptedMergeTree/ReplicatedMergeTree/' /var/lib/clickhouse/metadata/table_metadata_corruption/rmt1.sql"
print(f"Fix corrupted metadata using `{fix_corrupt}`")
dummy_node.exec_in_container(["bash", "-c", fix_corrupt])
dummy_node.start_clickhouse()
assert_eq_with_retry(dummy_node, query, expected)
main_node.query("DROP DATABASE IF EXISTS table_metadata_corruption")
dummy_node.query("DROP DATABASE IF EXISTS table_metadata_corruption")