diff --git a/contrib/libssh b/contrib/libssh index acea3e2d5ad..2c76332ef56 160000 --- a/contrib/libssh +++ b/contrib/libssh @@ -1 +1 @@ -Subproject commit acea3e2d5ad6e22f52961b43411b4ed2d9224b9e +Subproject commit 2c76332ef56d90f55965ab24da6b6dbcbef29c4c diff --git a/contrib/libssh-cmake/CMakeLists.txt b/contrib/libssh-cmake/CMakeLists.txt index 9a7c8eb2355..58db81cf352 100644 --- a/contrib/libssh-cmake/CMakeLists.txt +++ b/contrib/libssh-cmake/CMakeLists.txt @@ -55,9 +55,6 @@ if (OS_LINUX) elseif (ARCH_PPC64LE) target_include_directories(_ssh PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/libssh-cmake/linux/ppc64le") elseif (ARCH_S390X) - # Like ppc64le but with HAVE_OPENSSL_FIPS_MODE undefined. This is because the OpenSSL used by s390x doesn't support - # FIPS_mode(). Besides that, the custom s390x/config.h only exists to make things compile without additional ifdefs. - # With high probability, libssl with OpenSSL on s390x is broken. target_include_directories(_ssh PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/libssh-cmake/linux/s390x") elseif (ARCH_RISCV64) target_include_directories(_ssh PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/libssh-cmake/linux/riscv64") diff --git a/contrib/libssh-cmake/IncludeSources.cmake b/contrib/libssh-cmake/IncludeSources.cmake index b60d8ed6c3c..ca5d0218c24 100644 --- a/contrib/libssh-cmake/IncludeSources.cmake +++ b/contrib/libssh-cmake/IncludeSources.cmake @@ -101,8 +101,8 @@ set(libssh_SRCS ${LIB_SOURCE_DIR}/src/dh_crypto.c ) -# see the comment on s390x in libssh-cmake/CMakeLists.txt -if(OPENSSL_VERSION VERSION_LESS "1.1.0" AND NOT ARCH_S390X) +if (NOT (ENABLE_OPENSSL OR ENABLE_OPENSSL_DYNAMIC)) + add_compile_definitions(USE_BORINGSSL=1) set(libssh_SRCS ${libssh_SRCS} ${LIB_SOURCE_DIR}/src/libcrypto-compat.c) endif() diff --git a/contrib/libssh-cmake/darwin/config.h b/contrib/libssh-cmake/darwin/config.h index 8d54617775e..12378a64cea 100644 --- a/contrib/libssh-cmake/darwin/config.h +++ b/contrib/libssh-cmake/darwin/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/freebsd/config.h b/contrib/libssh-cmake/freebsd/config.h index 264748025f2..8a70acb473c 100644 --- a/contrib/libssh-cmake/freebsd/config.h +++ b/contrib/libssh-cmake/freebsd/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/aarch64-musl/config.h b/contrib/libssh-cmake/linux/aarch64-musl/config.h index 6be07939bd0..15236527fdf 100644 --- a/contrib/libssh-cmake/linux/aarch64-musl/config.h +++ b/contrib/libssh-cmake/linux/aarch64-musl/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/aarch64/config.h b/contrib/libssh-cmake/linux/aarch64/config.h index a3af7d16d5a..e65ccb8ba3e 100644 --- a/contrib/libssh-cmake/linux/aarch64/config.h +++ b/contrib/libssh-cmake/linux/aarch64/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/ppc64le/config.h b/contrib/libssh-cmake/linux/ppc64le/config.h index 3528bf28485..c56b1ad0334 100644 --- a/contrib/libssh-cmake/linux/ppc64le/config.h +++ b/contrib/libssh-cmake/linux/ppc64le/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/riscv64/config.h b/contrib/libssh-cmake/linux/riscv64/config.h index 3cb4336e615..33c91bf542b 100644 --- a/contrib/libssh-cmake/linux/riscv64/config.h +++ b/contrib/libssh-cmake/linux/riscv64/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ -/* #undef HAVE_OPENSSL_FIPS_MODE */ +#if USE_BORINGSSL +#define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/s390x/config.h b/contrib/libssh-cmake/linux/s390x/config.h index dcc3ee0bbea..289a8cabc8e 100644 --- a/contrib/libssh-cmake/linux/s390x/config.h +++ b/contrib/libssh-cmake/linux/s390x/config.h @@ -5,7 +5,7 @@ #define VERSION "0.9.7" #define SYSCONFDIR "etc" -#define BINARYDIR "/home/ubuntu/workdir/ClickHouse/build/ppc64le" +#define BINARYDIR "/home/ubuntu/workdir/ClickHouse/build/s390x" #define SOURCEDIR "/home/ubuntu/workdir/ClickHouse" /* Global bind configuration file path */ @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ -/* #undef HAVE_OPENSSL_FIPS_MODE */ +#if USE_BORINGSSL +#define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 @@ -282,4 +284,4 @@ /* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most significant byte first (like Motorola and SPARC, unlike Intel). */ -/* #undef WORDS_BIGENDIAN */ +#define WORDS_BIGENDIAN 1 diff --git a/contrib/libssh-cmake/linux/x86-64-musl/config.h b/contrib/libssh-cmake/linux/x86-64-musl/config.h index b189d71d1cd..fd7c2e2b0c1 100644 --- a/contrib/libssh-cmake/linux/x86-64-musl/config.h +++ b/contrib/libssh-cmake/linux/x86-64-musl/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/contrib/libssh-cmake/linux/x86-64/config.h b/contrib/libssh-cmake/linux/x86-64/config.h index b3395b2326c..4090c5a45ad 100644 --- a/contrib/libssh-cmake/linux/x86-64/config.h +++ b/contrib/libssh-cmake/linux/x86-64/config.h @@ -127,7 +127,9 @@ /* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ /* Define to 1 if you have the `FIPS_mode' function. */ +#if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 +#endif /* Define to 1 if you have the `EVP_DigestSign' function. */ #define HAVE_OPENSSL_EVP_DIGESTSIGN 1 diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4b82b6f8746..0aa475d23e6 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1267,6 +1267,10 @@ bool KeyCondition::tryPrepareSetIndex( size_t set_types_size = set_types.size(); size_t indexes_mapping_size = indexes_mapping.size(); + /// When doing strict matches, we have to check all elements in set. + if (strict && indexes_mapping_size < set_types_size) + return false; + for (auto & index_mapping : indexes_mapping) if (index_mapping.tuple_index >= set_types_size) return false; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cf286e3d227..6c014307c3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2572,97 +2572,6 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t "({} != {} + {}), it's a bug", parts_to_remove.size(), sum_of_ranges, excluded_parts.size()); } -size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory() -{ - /** - * Remove old (configured by setting) broken detached parts. - * Only parts with certain prefixes are removed. These prefixes - * are such that it is guaranteed that they will never be needed - * and need to be cleared. ctime is used to check when file was - * moved to detached/ directory (see https://unix.stackexchange.com/a/211134) - */ - - DetachedPartsInfo detached_parts = getDetachedParts(); - if (detached_parts.empty()) - return 0; - - auto get_last_touched_time = [&](const DetachedPartInfo & part_info) -> time_t - { - auto path = fs::path(relative_data_path) / "detached" / part_info.dir_name; - time_t last_change_time = part_info.disk->getLastChanged(path); - time_t last_modification_time = part_info.disk->getLastModified(path).epochTime(); - return std::max(last_change_time, last_modification_time); - }; - - time_t ttl_seconds = getSettings()->merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds; - - size_t unfinished_deleting_parts = 0; - time_t current_time = time(nullptr); - for (const auto & part_info : detached_parts) - { - if (!part_info.dir_name.starts_with("deleting_")) - continue; - - time_t startup_time = current_time - static_cast(Context::getGlobalContextInstance()->getUptimeSeconds()); - time_t last_touch_time = get_last_touched_time(part_info); - - /// Maybe it's being deleted right now (for example, in ALTER DROP DETACHED) - bool had_restart = last_touch_time < startup_time; - bool ttl_expired = last_touch_time + ttl_seconds <= current_time; - if (!had_restart && !ttl_expired) - continue; - - /// We were trying to delete this detached part but did not finish deleting, probably because the server crashed - LOG_INFO(log, "Removing detached part {} that we failed to remove previously", part_info.dir_name); - try - { - removeDetachedPart(part_info.disk, fs::path(relative_data_path) / "detached" / part_info.dir_name / "", part_info.dir_name); - ++unfinished_deleting_parts; - } - catch (...) - { - tryLogCurrentException(log); - } - } - - if (!getSettings()->merge_tree_enable_clear_old_broken_detached) - return unfinished_deleting_parts; - - const auto full_path = fs::path(relative_data_path) / "detached"; - size_t removed_count = 0; - for (const auto & part_info : detached_parts) - { - if (!part_info.valid_name || part_info.prefix.empty()) - continue; - - const auto & removable_detached_parts_prefixes = DetachedPartInfo::DETACHED_REASONS_REMOVABLE_BY_TIMEOUT; - bool can_be_removed_by_timeout = std::find( - removable_detached_parts_prefixes.begin(), - removable_detached_parts_prefixes.end(), - part_info.prefix) != removable_detached_parts_prefixes.end(); - - if (!can_be_removed_by_timeout) - continue; - - ssize_t threshold = current_time - ttl_seconds; - time_t last_touch_time = get_last_touched_time(part_info); - - if (last_touch_time == 0 || last_touch_time >= threshold) - continue; - - const String & old_name = part_info.dir_name; - String new_name = "deleting_" + part_info.dir_name; - part_info.disk->moveFile(fs::path(full_path) / old_name, fs::path(full_path) / new_name); - - removeDetachedPart(part_info.disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name); - LOG_WARNING(log, "Removed broken detached part {} due to a timeout for broken detached parts", old_name); - ++removed_count; - } - - LOG_INFO(log, "Cleaned up {} detached parts", removed_count); - - return removed_count + unfinished_deleting_parts; -} size_t MergeTreeData::clearEmptyParts() { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 3e2fc0548c7..55b4b23e351 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -689,8 +689,6 @@ public: /// Try to clear parts from filesystem. Throw exception in case of errors. void clearPartsFromFilesystem(const DataPartsVector & parts, bool throw_on_error = true, NameSet * parts_failed_to_delete = nullptr); - size_t clearOldBrokenPartsFromDetachedDirectory(); - /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. size_t clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes = {"tmp_", "tmp-fetch_"}); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 7f463795d86..2d8e095ea29 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { - extern const int DIRECTORY_ALREADY_EXISTS; extern const int NOT_IMPLEMENTED; } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index cee910dc185..dbae87b0c5e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -63,11 +63,9 @@ struct Settings; M(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ - M(UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30, "Remove old broken detached parts in the background if they remained intouched for a specified by this setting period of time.", 0) \ M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ M(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ M(UInt64, number_of_free_entries_in_pool_to_execute_optimize_entire_partition, 25, "When there is less than specified number of free entries in pool, do not try to execute optimize entire partition with a merge (this merge is created when set min_age_to_force_merge_seconds > 0 and min_age_to_force_merge_on_partition_only = true). This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(UInt64, merge_tree_enable_clear_old_broken_detached, false, "Enable clearing old broken detached parts operation in background.", 0) \ M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ M(CleanDeletedRows, clean_deleted_rows, CleanDeletedRows::Never, "Is the Replicated Merge cleanup has to be done automatically at each merge or manually (possible values are 'Always'/'Never' (default))", 0) \ M(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 0e6f14e8146..8daee661c75 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -154,7 +154,6 @@ Float32 ReplicatedMergeTreeCleanupThread::iterate() /// Both use relative_data_path which changes during rename, so we /// do it under share lock cleaned_part_like += storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); - cleaned_part_like += storage.clearOldBrokenPartsFromDetachedDirectory(); } /// This is loose condition: no problem if we actually had lost leadership at this moment diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 46f250ade11..470e30b7947 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1383,7 +1383,6 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign cleared_count += clearOldPartsFromFilesystem(); cleared_count += clearOldMutations(); cleared_count += clearEmptyParts(); - cleared_count += clearOldBrokenPartsFromDetachedDirectory(); return cleared_count; /// TODO maybe take into account number of cleared objects when calculating backoff }, common_assignee_trigger, getStorageID()), /* need_trigger */ false); diff --git a/tests/integration/test_broken_detached_part_clean_up/__init__.py b/tests/integration/test_broken_detached_part_clean_up/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_broken_detached_part_clean_up/configs/store_cleanup.xml b/tests/integration/test_broken_detached_part_clean_up/configs/store_cleanup.xml deleted file mode 100644 index 5fbe87cce00..00000000000 --- a/tests/integration/test_broken_detached_part_clean_up/configs/store_cleanup.xml +++ /dev/null @@ -1,11 +0,0 @@ - - 0 - 60 - 1 - - - - testkeeper - - \ No newline at end of file diff --git a/tests/integration/test_broken_detached_part_clean_up/test.py b/tests/integration/test_broken_detached_part_clean_up/test.py deleted file mode 100644 index bdf993ddedf..00000000000 --- a/tests/integration/test_broken_detached_part_clean_up/test.py +++ /dev/null @@ -1,360 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk -from helpers.corrupt_part_data_on_disk import break_part -import time - -cluster = ClickHouseCluster(__file__) - -node1 = cluster.add_instance( - "node1", stay_alive=True, main_configs=["configs/store_cleanup.xml"] -) - -path_to_data = "/var/lib/clickhouse/" - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def remove_broken_detached_part_impl(table, node, expect_broken_prefix): - assert ( - node.query( - f"SELECT COUNT() FROM system.parts WHERE table='{table}' AND active=1" - ) - == "4\n" - ) - - path_to_detached = path_to_data + f"data/default/{table}/detached/" - - result = node.exec_in_container(["ls", path_to_detached]) - assert result.strip() == "" - - corrupt_part_data_on_disk(node, table, "all_3_3_0") - break_part(node, table, "all_3_3_0") - node.query(f"ALTER TABLE {table} DETACH PART 'all_1_1_0'") - result = node.exec_in_container(["touch", f"{path_to_detached}trash"]) - - node.exec_in_container(["mkdir", f"{path_to_detached}../broken_all_fake"]) - node.exec_in_container( - ["touch", "-t", "1312031429.30", f"{path_to_detached}../broken_all_fake"] - ) - result = node.exec_in_container(["stat", f"{path_to_detached}../broken_all_fake"]) - print(result) - assert "Modify: 2013-12-03" in result - node.exec_in_container( - [ - "mv", - f"{path_to_detached}../broken_all_fake", - f"{path_to_detached}broken_all_fake", - ] - ) - - for name in [ - "unexpected_all_42_1337_5", - "deleting_all_123_456_7", - "covered-by-broken_all_12_34_5", - ]: - node.exec_in_container(["mkdir", f"{path_to_detached}../{name}"]) - node.exec_in_container( - [ - "touch", - "-t", - "1312031429.30", - f"{path_to_detached}../{name}", - ] - ) - result = node.exec_in_container(["stat", f"{path_to_detached}../{name}"]) - print(result) - assert "Modify: 2013-12-03" in result - node.exec_in_container( - [ - "mv", - f"{path_to_detached}../{name}", - f"{path_to_detached}{name}", - ] - ) - - result = node.query( - f"CHECK TABLE {table}", settings={"check_query_single_value_result": 0} - ) - assert "all_3_3_0\t0" in result - - node.query(f"DETACH TABLE {table}") - node.query(f"ATTACH TABLE {table}") - - node.wait_for_log_line( - "Removing detached part deleting_all_123_456_7", - timeout=90, - look_behind_lines=1000000, - ) - node.wait_for_log_line( - f"Removed broken detached part {expect_broken_prefix}_all_3_3_0 due to a timeout", - timeout=10, - look_behind_lines=1000000, - ) - node.wait_for_log_line( - "Removed broken detached part unexpected_all_42_1337_5 due to a timeout", - timeout=10, - look_behind_lines=1000000, - ) - - result = node.exec_in_container(["ls", path_to_detached]) - print(result) - assert f"{expect_broken_prefix}_all_3_3_0" not in result - assert "all_1_1_0" in result - assert "trash" in result - assert "broken_all_fake" in result - assert "covered-by-broken_all_12_34_5" in result - assert "unexpected_all_42_1337_5" not in result - assert "deleting_all_123_456_7" not in result - - node.query( - f"ALTER TABLE {table} DROP DETACHED PART 'covered-by-broken_all_12_34_5'", - settings={"allow_drop_detached": 1}, - ) - result = node.exec_in_container(["ls", path_to_detached]) - assert "covered-by-broken_all_12_34_5" not in result - - node.query(f"DROP TABLE {table} SYNC") - - -def test_remove_broken_detached_part_merge_tree(started_cluster): - node1.query( - """ - CREATE TABLE - mt(id UInt32, value Int32) - ENGINE = MergeTree() ORDER BY id - SETTINGS - merge_tree_enable_clear_old_broken_detached=1, - merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5; - """ - ) - - for i in range(4): - node1.query( - f"INSERT INTO mt SELECT number, number * number FROM numbers ({i * 100000}, 100000)" - ) - - remove_broken_detached_part_impl("mt", node1, "broken-on-start") - - -def test_remove_broken_detached_part_replicated_merge_tree(started_cluster): - node1.query( - f""" - CREATE TABLE - replicated_mt(date Date, id UInt32, value Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{node1.name}') ORDER BY id - SETTINGS - merge_tree_enable_clear_old_broken_detached=1, - merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds=5, - cleanup_delay_period=1, - cleanup_delay_period_random_add=0, - cleanup_thread_preferred_points_per_iteration=0; - """ - ) - - for i in range(4): - node1.query( - f"INSERT INTO replicated_mt SELECT toDate('2019-10-01'), number, number * number FROM numbers ({i * 100000}, 100000)" - ) - - remove_broken_detached_part_impl("replicated_mt", node1, "broken") - - -def test_store_cleanup(started_cluster): - node1.query("CREATE DATABASE db UUID '10000000-1000-4000-8000-000000000001'") - node1.query( - "CREATE TABLE db.log UUID '10000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" - ) - node1.query( - "CREATE TABLE db.mt UUID '10000000-1000-4000-8000-000000000003' ENGINE=MergeTree ORDER BY tuple() AS SELECT 1" - ) - node1.query( - "CREATE TABLE db.mem UUID '10000000-1000-4000-8000-000000000004' ENGINE=Memory AS SELECT 1" - ) - - node1.query("CREATE DATABASE db2 UUID '20000000-1000-4000-8000-000000000001'") - node1.query( - "CREATE TABLE db2.log UUID '20000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" - ) - node1.query("DETACH DATABASE db2") - - node1.query("CREATE DATABASE db3 UUID '30000000-1000-4000-8000-000000000001'") - node1.query( - "CREATE TABLE db3.log UUID '30000000-1000-4000-8000-000000000002' ENGINE=Log AS SELECT 1" - ) - node1.query( - "CREATE TABLE db3.log2 UUID '30000000-1000-4000-8000-000000000003' ENGINE=Log AS SELECT 1" - ) - node1.query("DETACH TABLE db3.log") - node1.query("DETACH TABLE db3.log2 PERMANENTLY") - - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store"] - ) - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/100"] - ) - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/200"] - ) - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/300"] - ) - - node1.stop_clickhouse(kill=True) - # All dirs related to `db` will be removed - node1.exec_in_container(["rm", f"{path_to_data}/metadata/db.sql"]) - - node1.exec_in_container(["mkdir", f"{path_to_data}/store/kek"]) - node1.exec_in_container(["touch", f"{path_to_data}/store/12"]) - try: - node1.exec_in_container(["mkdir", f"{path_to_data}/store/456"]) - except Exception as e: - print("Failed to create 456/:", str(e)) - node1.exec_in_container(["mkdir", f"{path_to_data}/store/456/testgarbage"]) - node1.exec_in_container( - ["mkdir", f"{path_to_data}/store/456/30000000-1000-4000-8000-000000000003"] - ) - node1.exec_in_container( - ["touch", f"{path_to_data}/store/456/45600000-1000-4000-8000-000000000003"] - ) - node1.exec_in_container( - ["mkdir", f"{path_to_data}/store/456/45600000-1000-4000-8000-000000000004"] - ) - - node1.start_clickhouse() - node1.query("DETACH DATABASE db2") - node1.query("DETACH TABLE db3.log") - - node1.wait_for_log_line( - "Removing access rights for unused directory", - timeout=60, - look_behind_lines=1000000, - ) - node1.wait_for_log_line( - "directories from store", timeout=60, look_behind_lines=1000000 - ) - - store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) - assert "100" in store - assert "200" in store - assert "300" in store - assert "456" in store - assert "kek" in store - assert "12" in store - assert "d---------" in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store"] - ) - assert "d---------" in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/456"] - ) - - # Metadata is removed, so store/100 contains garbage - store100 = node1.exec_in_container(["ls", f"{path_to_data}/store/100"]) - assert "10000000-1000-4000-8000-000000000001" in store100 - assert "10000000-1000-4000-8000-000000000002" in store100 - assert "10000000-1000-4000-8000-000000000003" in store100 - assert "d---------" in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/100"] - ) - - # Database is detached, nothing to clean up - store200 = node1.exec_in_container(["ls", f"{path_to_data}/store/200"]) - assert "20000000-1000-4000-8000-000000000001" in store200 - assert "20000000-1000-4000-8000-000000000002" in store200 - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/200"] - ) - - # Tables are detached, nothing to clean up - store300 = node1.exec_in_container(["ls", f"{path_to_data}/store/300"]) - assert "30000000-1000-4000-8000-000000000001" in store300 - assert "30000000-1000-4000-8000-000000000002" in store300 - assert "30000000-1000-4000-8000-000000000003" in store300 - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/300"] - ) - - # Manually created garbage - store456 = node1.exec_in_container(["ls", f"{path_to_data}/store/456"]) - assert "30000000-1000-4000-8000-000000000003" in store456 - assert "45600000-1000-4000-8000-000000000003" in store456 - assert "45600000-1000-4000-8000-000000000004" in store456 - assert "testgarbage" in store456 - assert "----------" in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/456"] - ) - - node1.wait_for_log_line( - "Removing unused directory", timeout=90, look_behind_lines=1000000 - ) - node1.wait_for_log_line( - "directories from store", timeout=90, look_behind_lines=1000000 - ) - node1.wait_for_log_line( - "Nothing to clean up from store/", timeout=90, look_behind_lines=1000000 - ) - - store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) - assert "100" in store - assert "200" in store - assert "300" in store - assert "456" in store - assert "kek" not in store # changed - assert "\n12\n" not in store # changed - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store"] - ) # changed - - # Metadata is removed, so store/100 contains garbage - store100 = node1.exec_in_container(["ls", f"{path_to_data}/store/100"]) # changed - assert "10000000-1000-4000-8000-000000000001" not in store100 # changed - assert "10000000-1000-4000-8000-000000000002" not in store100 # changed - assert "10000000-1000-4000-8000-000000000003" not in store100 # changed - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/100"] - ) # changed - - # Database is detached, nothing to clean up - store200 = node1.exec_in_container(["ls", f"{path_to_data}/store/200"]) - assert "20000000-1000-4000-8000-000000000001" in store200 - assert "20000000-1000-4000-8000-000000000002" in store200 - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/200"] - ) - - # Tables are detached, nothing to clean up - store300 = node1.exec_in_container(["ls", f"{path_to_data}/store/300"]) - assert "30000000-1000-4000-8000-000000000001" in store300 - assert "30000000-1000-4000-8000-000000000002" in store300 - assert "30000000-1000-4000-8000-000000000003" in store300 - assert "d---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/300"] - ) - - # Manually created garbage - store456 = node1.exec_in_container(["ls", f"{path_to_data}/store/456"]) - assert "30000000-1000-4000-8000-000000000003" not in store456 # changed - assert "45600000-1000-4000-8000-000000000003" not in store456 # changed - assert "45600000-1000-4000-8000-000000000004" not in store456 # changed - assert "testgarbage" not in store456 # changed - assert "---------" not in node1.exec_in_container( - ["ls", "-l", f"{path_to_data}/store/456"] - ) # changed - - node1.query("ATTACH TABLE db3.log2") - node1.query("ATTACH DATABASE db2") - node1.query("ATTACH TABLE db3.log") - - assert "1\n" == node1.query("SELECT * FROM db3.log") - assert "1\n" == node1.query("SELECT * FROM db3.log2") - assert "1\n" == node1.query("SELECT * FROM db2.log") diff --git a/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.reference b/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.sql b/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.sql new file mode 100644 index 00000000000..29fd313b12d --- /dev/null +++ b/tests/queries/0_stateless/02890_partition_prune_in_extra_columns.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS e; + +CREATE TABLE e (dt DateTime, t Int32) ENGINE = MergeTree() PARTITION BY (t, toYYYYMM(dt)) ORDER BY tuple(); + +INSERT INTO e SELECT toDateTime('2022-12-12 11:00:00') + number, 86 FROM numbers(10); + +SELECT COUNT(*) FROM e WHERE (t, dt) IN (86, '2022-12-12 11:00:00'); + +DROP TABLE e;