diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 5cb27d90b62..2a9f6cddc59 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -127,17 +127,17 @@ EOL function stop() { + local max_tries="" + if [ -n "$1" ] + then + max_tries="--max-tries $1" + fi + local pid # Preserve the pid, since the server can hung after the PID will be deleted. pid="$(cat /var/run/clickhouse-server/clickhouse-server.pid)" - # --max-tries is supported only since 22.12 - if dpkg --compare-versions "$(clickhouse local -q 'select version()')" ge "22.12"; then - # Increase default waiting timeout for sanitizers and debug builds - clickhouse stop --max-tries 180 --do-not-kill && return - else - clickhouse stop --do-not-kill && return - fi + clickhouse stop $max_tries --do-not-kill && return # We failed to stop the server with SIGTERM. Maybe it hang, let's collect stacktraces. kill -TERM "$(pidof gdb)" ||: @@ -431,7 +431,7 @@ else clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" - stop + stop 180 mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log # Start new server diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bd57579505a..0f36c93ea4f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -118,7 +118,6 @@ StorageMergeTree::StorageMergeTree( void StorageMergeTree::startup() { - clearOldPartsFromFilesystem(); clearOldWriteAheadLogs(); clearEmptyParts(); @@ -185,26 +184,6 @@ void StorageMergeTree::shutdown() if (deduplication_log) deduplication_log->shutdown(); - - try - { - /// We clear all old parts after stopping all background operations. - /// It's important, because background operations can produce temporary - /// parts which will remove themselves in their destructors. If so, we - /// may have race condition between our remove call and background - /// process. - /// Do not clear old parts in case when server is shutting down because it failed to start due to some exception. - - if (Context::getGlobalContextInstance()->getApplicationType() == Context::ApplicationType::SERVER - && Context::getGlobalContextInstance()->isServerCompletelyStarted()) - clearOldPartsFromFilesystem(true); - } - catch (...) - { - /// Example: the case of readonly filesystem, we have failure removing old parts. - /// Should not prevent table shutdown. - tryLogCurrentException(log); - } } diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 6bd224851e7..eaad8080674 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -470,11 +470,20 @@ def test_detached_part_dir_exists(started_cluster): ) # will move all_1_1_0 to detached/all_1_1_0 and create all_1_1_1 wait_for_delete_empty_parts(instance, "detached_part_dir_exists") + wait_for_delete_inactive_parts(instance, "detached_part_dir_exists") q("detach table detached_part_dir_exists") q("attach table detached_part_dir_exists") + q("insert into detached_part_dir_exists select 1") # will create all_1_1_0 q("insert into detached_part_dir_exists select 1") # will create all_2_2_0 - q("insert into detached_part_dir_exists select 1") # will create all_3_3_0 + + assert ( + q( + "select name from system.parts where table='detached_part_dir_exists' and active order by name" + ) + == "all_1_1_0\nall_2_2_0\n" + ) + instance.exec_in_container( [ "bash", @@ -511,7 +520,7 @@ def test_make_clone_in_detached(started_cluster): path = path_to_data + "data/default/clone_in_detached/" # broken part already detached - q("insert into clone_in_detached values (42, '¯\_(ツ)_/¯')") + q("insert into clone_in_detached values (42, '¯-_(ツ)_-¯')") instance.exec_in_container(["rm", path + "all_0_0_0/data.bin"]) instance.exec_in_container( ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] @@ -522,7 +531,7 @@ def test_make_clone_in_detached(started_cluster): ) # there's a directory with the same name, but different content - q("insert into clone_in_detached values (43, '¯\_(ツ)_/¯')") + q("insert into clone_in_detached values (43, '¯-_(ツ)_-¯')") instance.exec_in_container(["rm", path + "all_1_1_0/data.bin"]) instance.exec_in_container( ["cp", "-r", path + "all_1_1_0", path + "detached/broken_all_1_1_0"] @@ -551,7 +560,7 @@ def test_make_clone_in_detached(started_cluster): ) # there are directories with the same name, but different content, and part already detached - q("insert into clone_in_detached values (44, '¯\_(ツ)_/¯')") + q("insert into clone_in_detached values (44, '¯-_(ツ)_-¯')") instance.exec_in_container(["rm", path + "all_2_2_0/data.bin"]) instance.exec_in_container( ["cp", "-r", path + "all_2_2_0", path + "detached/broken_all_2_2_0"] diff --git a/tests/integration/test_parts_removal/__init__.py b/tests/integration/test_parts_removal/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_parts_removal/test.py b/tests/integration/test_parts_removal/test.py deleted file mode 100644 index 4772178d63b..00000000000 --- a/tests/integration/test_parts_removal/test.py +++ /dev/null @@ -1,71 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name -# pylint: disable=line-too-long - -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance("node", stay_alive=True) - - -def query_split(node, query): - return list( - map(lambda x: x.strip().split("\t"), node.query(query).strip().split("\n")) - ) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_parts_removal_on_abnormal_exit(start_cluster): - node.query( - """ - create table test_parts_removal (key Int) engine=MergeTree order by key; - insert into test_parts_removal values (1); -- all_1_1_0 - insert into test_parts_removal values (2); -- all_1_2_0 - optimize table test_parts_removal; -- all_2_2_0 - """ - ) - - parts = query_split( - node, "select name, _state from system.parts where table = 'test_parts_removal'" - ) - assert parts == [ - ["all_1_1_0", "Outdated"], - ["all_1_2_1", "Active"], - ["all_2_2_0", "Outdated"], - ] - - node.restart_clickhouse(kill=True) - - parts = query_split( - node, "select name, _state from system.parts where table = 'test_parts_removal'" - ) - assert parts == [ - ["all_1_1_0", "Outdated"], - ["all_1_2_1", "Active"], - ["all_2_2_0", "Outdated"], - ] - - node.query( - """ - detach table test_parts_removal; - attach table test_parts_removal; - """ - ) - - parts = query_split( - node, "select name, _state from system.parts where table = 'test_parts_removal'" - ) - assert parts == [ - ["all_1_2_1", "Active"], - ] diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index 3099fae4a42..d088d747ee8 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -23,6 +23,7 @@ 3 1 Begin 1 1 1 1 3 1 AddPart 1 1 1 1 all_3_3_0 3 1 Commit 1 1 1 0 +1 1 LockPart 1 1 1 1 all_2_2_0 4 1 Begin 1 1 1 1 4 1 AddPart 1 1 1 1 all_4_4_0 4 1 Commit 1 1 1 0 diff --git a/tests/queries/0_stateless/01172_transaction_counters.sql b/tests/queries/0_stateless/01172_transaction_counters.sql index ffe2d08e2d8..a809e4196e9 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.sql +++ b/tests/queries/0_stateless/01172_transaction_counters.sql @@ -2,7 +2,7 @@ drop table if exists txn_counters; -create table txn_counters (n Int64, creation_tid DEFAULT transactionID()) engine=MergeTree order by n; +create table txn_counters (n Int64, creation_tid DEFAULT transactionID()) engine=MergeTree order by n SETTINGS old_parts_lifetime=3600; insert into txn_counters(n) values (1); select transactionID(); @@ -31,7 +31,7 @@ attach table txn_counters; begin transaction; insert into txn_counters(n) values (4); select 6, system.parts.name, txn_counters.creation_tid = system.parts.creation_tid from txn_counters join system.parts on txn_counters._part = system.parts.name where database=currentDatabase() and table='txn_counters' order by system.parts.name; -select 7, name, removal_tid, removal_csn from system.parts where database=currentDatabase() and table='txn_counters' order by system.parts.name; +select 7, name, removal_tid, removal_csn from system.parts where database=currentDatabase() and table='txn_counters' and active order by system.parts.name; select 8, transactionID().3 == serverUUID(); commit; diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sh b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sh new file mode 100755 index 00000000000..b65e6019a2a --- /dev/null +++ b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +set -euo pipefail + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./parts.lib +. "$CURDIR"/parts.lib + +${CLICKHOUSE_CLIENT} -q 'DROP TABLE IF EXISTS ttl_empty_parts' + +${CLICKHOUSE_CLIENT} -q ' + CREATE TABLE ttl_empty_parts (id UInt32, d Date) ENGINE = MergeTree ORDER BY tuple() PARTITION BY id SETTINGS old_parts_lifetime=5 +' + +${CLICKHOUSE_CLIENT} -q "INSERT INTO ttl_empty_parts SELECT 0, toDate('2005-01-01') + number from numbers(500);" +${CLICKHOUSE_CLIENT} -q "INSERT INTO ttl_empty_parts SELECT 1, toDate('2050-01-01') + number from numbers(500);" + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ttl_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active;" + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE ttl_empty_parts MODIFY TTL d SETTINGS mutations_sync = 1;" + +# To be sure, that task, which clears outdated parts executed. +timeout 60 bash -c 'wait_for_delete_empty_parts ttl_empty_parts' + +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ttl_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active;" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE ttl_empty_parts;" diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql deleted file mode 100644 index f40ed70caef..00000000000 --- a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE IF EXISTS ttl_empty_parts; - -CREATE TABLE ttl_empty_parts (id UInt32, d Date) ENGINE = MergeTree ORDER BY tuple() PARTITION BY id; - -INSERT INTO ttl_empty_parts SELECT 0, toDate('2005-01-01') + number from numbers(500); -INSERT INTO ttl_empty_parts SELECT 1, toDate('2050-01-01') + number from numbers(500); - -SELECT count() FROM ttl_empty_parts; -SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active; - -ALTER TABLE ttl_empty_parts MODIFY TTL d; - --- To be sure, that task, which clears outdated parts executed. -DETACH TABLE ttl_empty_parts; -ATTACH TABLE ttl_empty_parts; - -SELECT count() FROM ttl_empty_parts; -SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active; - -DROP TABLE ttl_empty_parts; diff --git a/tests/queries/0_stateless/02421_type_json_empty_parts.sh b/tests/queries/0_stateless/02421_type_json_empty_parts.sh new file mode 100755 index 00000000000..b6cf5995bfa --- /dev/null +++ b/tests/queries/0_stateless/02421_type_json_empty_parts.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +set -euo pipefail + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +# shellcheck source=./parts.lib +. "$CURDIR"/parts.lib + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'Collapsing';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data JSON) ENGINE = CollapsingMergeTree(s) ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, 1, '{\"k1\": \"aaa\"}') (1, -1, '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'DELETE all';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, data JSON) ENGINE = MergeTree ORDER BY id SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '{\"k1\": \"aaa\"}') (1, '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;" +${CLICKHOUSE_CLIENT} -q "ALTER TABLE t_json_empty_parts DELETE WHERE 1 SETTINGS mutations_sync = 1;" +timeout 60 bash -c 'wait_for_delete_empty_parts t_json_empty_parts' +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT 'TTL';" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_empty_parts (id UInt64, d Date, data JSON) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1 SETTINGS old_parts_lifetime=5;" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_empty_parts VALUES (1, '2000-01-01', '{\"k1\": \"aaa\"}') (2, '2000-01-01', '{\"k2\": \"bbb\"}');" +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE t_json_empty_parts FINAL;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;" +${CLICKHOUSE_CLIENT} -q "ALTER TABLE t_json_empty_parts MODIFY TTL d;" +${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE t_json_empty_parts FINAL;" +timeout 60 bash -c 'wait_for_delete_empty_parts t_json_empty_parts' +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM t_json_empty_parts;" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active;" +${CLICKHOUSE_CLIENT} -q "DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1;" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_empty_parts;" diff --git a/tests/queries/0_stateless/02421_type_json_empty_parts.sql b/tests/queries/0_stateless/02421_type_json_empty_parts.sql deleted file mode 100644 index 409a2b18a49..00000000000 --- a/tests/queries/0_stateless/02421_type_json_empty_parts.sql +++ /dev/null @@ -1,61 +0,0 @@ --- Tags: no-fasttest - -SET allow_experimental_object_type = 1; - -DROP TABLE IF EXISTS t_json_empty_parts; - -SELECT 'Collapsing'; -CREATE TABLE t_json_empty_parts (id UInt64, s Int8, data JSON) ENGINE = CollapsingMergeTree(s) ORDER BY id; - -INSERT INTO t_json_empty_parts VALUES (1, 1, '{"k1": "aaa"}') (1, -1, '{"k2": "bbb"}'); - -SELECT count() FROM t_json_empty_parts; -SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active; -DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1; - -DROP TABLE t_json_empty_parts; - -DROP TABLE IF EXISTS t_json_empty_parts; - -SELECT 'DELETE all'; -CREATE TABLE t_json_empty_parts (id UInt64, data JSON) ENGINE = MergeTree ORDER BY id; - -INSERT INTO t_json_empty_parts VALUES (1, '{"k1": "aaa"}') (2, '{"k2": "bbb"}'); - -SELECT count() FROM t_json_empty_parts; -SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active; -DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1; - -SET mutations_sync = 2; -ALTER TABLE t_json_empty_parts DELETE WHERE 1; - -DETACH TABLE t_json_empty_parts; -ATTACH TABLE t_json_empty_parts; - -SELECT count() FROM t_json_empty_parts; -SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active; -DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1; - -DROP TABLE IF EXISTS t_json_empty_parts; - -SELECT 'TTL'; -CREATE TABLE t_json_empty_parts (id UInt64, d Date, data JSON) ENGINE = MergeTree ORDER BY id TTL d WHERE id % 2 = 1; - -INSERT INTO t_json_empty_parts VALUES (1, '2000-01-01', '{"k1": "aaa"}') (2, '2000-01-01', '{"k2": "bbb"}'); -OPTIMIZE TABLE t_json_empty_parts FINAL; - -SELECT count() FROM t_json_empty_parts; -SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active; -DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1; - -ALTER TABLE t_json_empty_parts MODIFY TTL d; -OPTIMIZE TABLE t_json_empty_parts FINAL; - -DETACH TABLE t_json_empty_parts; -ATTACH TABLE t_json_empty_parts; - -SELECT count() FROM t_json_empty_parts; -SELECT count() FROM system.parts WHERE table = 't_json_empty_parts' AND database = currentDatabase() AND active; -DESC TABLE t_json_empty_parts SETTINGS describe_extend_object_types = 1; - -DROP TABLE IF EXISTS t_json_empty_parts; diff --git a/tests/queries/0_stateless/02423_drop_memory_parts.sql b/tests/queries/0_stateless/02423_drop_memory_parts.sql index 0d42847f6e5..fad81d46e68 100644 --- a/tests/queries/0_stateless/02423_drop_memory_parts.sql +++ b/tests/queries/0_stateless/02423_drop_memory_parts.sql @@ -35,4 +35,4 @@ ATTACH TABLE table_in_memory; SELECT count() FROM table_in_memory; SELECT name, part_type, rows, active from system.parts -WHERE table='table_in_memory' AND database=currentDatabase(); +WHERE table='table_in_memory' AND database=currentDatabase() and active; diff --git a/tests/queries/0_stateless/parts.lib b/tests/queries/0_stateless/parts.lib index c35f996ffed..7aec10392f0 100644 --- a/tests/queries/0_stateless/parts.lib +++ b/tests/queries/0_stateless/parts.lib @@ -37,3 +37,6 @@ function wait_for_delete_inactive_parts() echo "Timed out while waiting for delete inactive parts!" >&2 return 2 } + +export -f wait_for_delete_empty_parts +export -f wait_for_delete_inactive_parts