This commit is contained in:
MikhailBurdukov 2024-09-18 13:33:43 -07:00 committed by GitHub
commit 4f5c290bb8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 47 additions and 6 deletions

View File

@ -2102,7 +2102,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
part->version.setCreationTID(Tx::PrehistoricTID, nullptr);
renameTempPartAndReplace(part, transaction);
checkPartChecksumsAndCommit(transaction, part);
checkPartChecksumsAndCommit(transaction, part, /*hardlinked_files*/ {}, /*replace_zero_copy_lock*/ true);
writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */,
part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr,

View File

@ -6,6 +6,7 @@ import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import assert_eq_with_retry
logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler())
@ -145,9 +146,30 @@ def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_par
) == (3 * FILES_OVERHEAD) + (files_per_part * 3)
def remove_leftovers_from_zk(node_data, node_for_query, replica_name):
replicas = node_data.query_with_retry(
"select name from system.zookeeper where path='/test/drop_table/replicas'"
)
if replica_name in replicas and "test_drop_table" not in node_data.query(
"show tables"
):
node_for_query.query(
f"system drop replica '{replica_name}' from table test_drop_table"
)
def test_drop_table(cluster):
node = list(cluster.instances.values())[0]
node2 = list(cluster.instances.values())[1]
# We are checking log entries in this test, so it should be empty before the execution.
node.rotate_logs()
node2.rotate_logs()
# drop table .. sync, doesn't removes replica from zk immediately. Prevent race contition by removing old nodes from zk.
remove_leftovers_from_zk(node, node2, "1")
remove_leftovers_from_zk(node2, node, "2")
node.query(
"create table test_drop_table (n int) engine=ReplicatedMergeTree('/test/drop_table', '1') order by n partition by n % 99 settings storage_policy='s3'"
)
@ -195,11 +217,7 @@ def test_drop_table(cluster):
)
# It could leave some leftovers, remove them
replicas = node.query_with_retry(
"select name from system.zookeeper where path='/test/drop_table/replicas'"
)
if "1" in replicas and "test_drop_table" not in node.query("show tables"):
node2.query("system drop replica '1' from table test_drop_table")
remove_leftovers_from_zk(node, node2, "1")
# Just in case table was not created due to connection errors
node.query(
@ -222,3 +240,26 @@ def test_drop_table(cluster):
"select count(n), sum(n) from test_drop_table"
)
node.query("drop table test_drop_table sync")
def test_s3_check_restore(cluster):
create_table(cluster)
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
node1.query(
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-02", 2)),
)
node1.query("DETACH TABLE s3_test;")
node2.query("SYSTEM DROP REPLICA '1' FROM TABLE s3_test;")
node2.query(
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-02", 2)),
)
node1.query("ATTACH TABLE s3_test;")
node1.query("SYSTEM RESTORE REPLICA s3_test;")
assert_eq_with_retry(
node1,
"SELECT count() FROM system.replication_queue WHERE table='s3_test' and type='ATTACH_PART'",
"0\n",
)