From 7ff447de686d687fab0e08a3094eb173d8d128ff Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Jul 2024 16:13:03 +0000 Subject: [PATCH] Modify the code for build with new libcxx (cherry picked from commit c896f0bf677bcc948d988cf83b2108e9f7cd761d) --- base/base/demangle.h | 1 + base/base/extended_types.h | 16 ++++++++ base/base/isSharedPtrUnique.h | 9 +++++ base/poco/Foundation/include/Poco/Format.h | 2 +- .../include/Poco/RefCountedObject.h | 2 + base/poco/Foundation/src/Format.cpp | 40 +++++++++---------- base/poco/MongoDB/src/ObjectId.cpp | 2 +- base/poco/MongoDB/src/OpMsgCursor.cpp | 8 ++-- base/poco/Net/src/HTTPMessage.cpp | 12 +++--- src/Backups/BackupOperationInfo.h | 2 + src/Common/AtomicLogger.h | 2 + src/Common/ConcurrencyControl.h | 1 + src/Common/RemoteHostFilter.h | 1 + src/Coordination/Changelog.h | 1 + src/Coordination/FourLetterCommand.h | 2 + src/Databases/DatabaseAtomic.cpp | 5 ++- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/MySQL/DatabaseMySQL.cpp | 3 +- src/Functions/FunctionsBitToArray.cpp | 9 ++++- src/Interpreters/ActionsVisitor.cpp | 9 +++-- src/Interpreters/Cache/Metadata.h | 4 +- src/Interpreters/DatabaseCatalog.cpp | 6 ++- src/Interpreters/ExternalLoader.h | 1 + src/Interpreters/JIT/CHJIT.cpp | 6 +-- src/Interpreters/Session.cpp | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 11 ++--- 26 files changed, 108 insertions(+), 55 deletions(-) create mode 100644 base/base/isSharedPtrUnique.h diff --git a/base/base/demangle.h b/base/base/demangle.h index ddca264ecab..af9ccad16c1 100644 --- a/base/base/demangle.h +++ b/base/base/demangle.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include diff --git a/base/base/extended_types.h b/base/base/extended_types.h index 796167ab45d..3bf3f4ed31d 100644 --- a/base/base/extended_types.h +++ b/base/base/extended_types.h @@ -108,6 +108,14 @@ struct make_unsigned // NOLINT(readability-identifier-naming) using type = std::make_unsigned_t; }; +template <> struct make_unsigned { using type = UInt8; }; +template <> struct make_unsigned { using type = UInt8; }; +template <> struct make_unsigned { using type = UInt16; }; +template <> struct make_unsigned { using type = UInt16; }; +template <> struct make_unsigned { using type = UInt32; }; +template <> struct make_unsigned { using type = UInt32; }; +template <> struct make_unsigned { using type = UInt64; }; +template <> struct make_unsigned { using type = UInt64; }; template <> struct make_unsigned { using type = UInt128; }; template <> struct make_unsigned { using type = UInt128; }; template <> struct make_unsigned { using type = UInt256; }; @@ -121,6 +129,14 @@ struct make_signed // NOLINT(readability-identifier-naming) using type = std::make_signed_t; }; +template <> struct make_signed { using type = Int8; }; +template <> struct make_signed { using type = Int8; }; +template <> struct make_signed { using type = Int16; }; +template <> struct make_signed { using type = Int16; }; +template <> struct make_signed { using type = Int32; }; +template <> struct make_signed { using type = Int32; }; +template <> struct make_signed { using type = Int64; }; +template <> struct make_signed { using type = Int64; }; template <> struct make_signed { using type = Int128; }; template <> struct make_signed { using type = Int128; }; template <> struct make_signed { using type = Int256; }; diff --git a/base/base/isSharedPtrUnique.h b/base/base/isSharedPtrUnique.h new file mode 100644 index 00000000000..c153605ecb1 --- /dev/null +++ b/base/base/isSharedPtrUnique.h @@ -0,0 +1,9 @@ +#pragma once + +#include + +template +bool isSharedPtrUnique(const std::shared_ptr & ptr) +{ + return ptr.use_count() == 1; +} diff --git a/base/poco/Foundation/include/Poco/Format.h b/base/poco/Foundation/include/Poco/Format.h index f84be16d3ad..4f91dd44ca5 100644 --- a/base/poco/Foundation/include/Poco/Format.h +++ b/base/poco/Foundation/include/Poco/Format.h @@ -232,7 +232,7 @@ void Foundation_API format( const Any & value10); -void Foundation_API format(std::string & result, const std::string & fmt, const std::vector & values); +void Foundation_API formatVector(std::string & result, const std::string & fmt, const std::vector & values); /// Supports a variable number of arguments and is used by /// all other variants of format(). diff --git a/base/poco/Foundation/include/Poco/RefCountedObject.h b/base/poco/Foundation/include/Poco/RefCountedObject.h index db966089e00..d0d964d8390 100644 --- a/base/poco/Foundation/include/Poco/RefCountedObject.h +++ b/base/poco/Foundation/include/Poco/RefCountedObject.h @@ -21,6 +21,8 @@ #include "Poco/AtomicCounter.h" #include "Poco/Foundation.h" +#include + namespace Poco { diff --git a/base/poco/Foundation/src/Format.cpp b/base/poco/Foundation/src/Format.cpp index 9872ddff042..94ab124510d 100644 --- a/base/poco/Foundation/src/Format.cpp +++ b/base/poco/Foundation/src/Format.cpp @@ -51,8 +51,8 @@ namespace } if (width != 0) str.width(width); } - - + + void parsePrec(std::ostream& str, std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { if (itFmt != endFmt && *itFmt == '.') @@ -67,7 +67,7 @@ namespace if (prec >= 0) str.precision(prec); } } - + char parseMod(std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { char mod = 0; @@ -77,13 +77,13 @@ namespace { case 'l': case 'h': - case 'L': + case 'L': case '?': mod = *itFmt++; break; } } return mod; } - + std::size_t parseIndex(std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { int index = 0; @@ -110,8 +110,8 @@ namespace case 'f': str << std::fixed; break; } } - - + + void writeAnyInt(std::ostream& str, const Any& any) { if (any.type() == typeid(char)) @@ -201,7 +201,7 @@ namespace str << RefAnyCast(*itVal++); break; case 'z': - str << AnyCast(*itVal++); + str << AnyCast(*itVal++); break; case 'I': case 'D': @@ -303,7 +303,7 @@ void format(std::string& result, const std::string& fmt, const Any& value) { std::vector args; args.push_back(value); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -312,7 +312,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons std::vector args; args.push_back(value1); args.push_back(value2); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -322,7 +322,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value1); args.push_back(value2); args.push_back(value3); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -333,7 +333,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value2); args.push_back(value3); args.push_back(value4); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -345,7 +345,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value3); args.push_back(value4); args.push_back(value5); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -358,7 +358,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value4); args.push_back(value5); args.push_back(value6); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -372,7 +372,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value5); args.push_back(value6); args.push_back(value7); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -387,7 +387,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value6); args.push_back(value7); args.push_back(value8); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -403,7 +403,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value7); args.push_back(value8); args.push_back(value9); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -420,16 +420,16 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value8); args.push_back(value9); args.push_back(value10); - format(result, fmt, args); + formatVector(result, fmt, args); } -void format(std::string& result, const std::string& fmt, const std::vector& values) +void formatVector(std::string& result, const std::string& fmt, const std::vector& values) { std::string::const_iterator itFmt = fmt.begin(); std::string::const_iterator endFmt = fmt.end(); std::vector::const_iterator itVal = values.begin(); - std::vector::const_iterator endVal = values.end(); + std::vector::const_iterator endVal = values.end(); while (itFmt != endFmt) { switch (*itFmt) diff --git a/base/poco/MongoDB/src/ObjectId.cpp b/base/poco/MongoDB/src/ObjectId.cpp index 0125c246c2d..e360d129843 100644 --- a/base/poco/MongoDB/src/ObjectId.cpp +++ b/base/poco/MongoDB/src/ObjectId.cpp @@ -57,7 +57,7 @@ std::string ObjectId::toString(const std::string& fmt) const for (int i = 0; i < 12; ++i) { - s += format(fmt, (unsigned int) _id[i]); + s += Poco::format(fmt, (unsigned int) _id[i]); } return s; } diff --git a/base/poco/MongoDB/src/OpMsgCursor.cpp b/base/poco/MongoDB/src/OpMsgCursor.cpp index bc95851ae33..6abd45ecf76 100644 --- a/base/poco/MongoDB/src/OpMsgCursor.cpp +++ b/base/poco/MongoDB/src/OpMsgCursor.cpp @@ -43,9 +43,9 @@ namespace Poco { namespace MongoDB { -static const std::string keyCursor {"cursor"}; -static const std::string keyFirstBatch {"firstBatch"}; -static const std::string keyNextBatch {"nextBatch"}; +[[ maybe_unused ]] static const std::string keyCursor {"cursor"}; +[[ maybe_unused ]] static const std::string keyFirstBatch {"firstBatch"}; +[[ maybe_unused ]] static const std::string keyNextBatch {"nextBatch"}; static Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc); @@ -131,7 +131,7 @@ OpMsgMessage& OpMsgCursor::next(Connection& connection) connection.readResponse(_response); } else -#endif +#endif { _response.clear(); _query.setCursor(_cursorID, _batchSize); diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index c0083ec410c..b7ab5543a85 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -17,9 +17,9 @@ #include "Poco/NumberFormatter.h" #include "Poco/NumberParser.h" #include "Poco/String.h" +#include #include - using Poco::NumberFormatter; using Poco::NumberParser; using Poco::icompare; @@ -75,7 +75,7 @@ void HTTPMessage::setContentLength(std::streamsize length) erase(CONTENT_LENGTH); } - + std::streamsize HTTPMessage::getContentLength() const { const std::string& contentLength = get(CONTENT_LENGTH, EMPTY); @@ -98,7 +98,7 @@ void HTTPMessage::setContentLength64(Poco::Int64 length) erase(CONTENT_LENGTH); } - + Poco::Int64 HTTPMessage::getContentLength64() const { const std::string& contentLength = get(CONTENT_LENGTH, EMPTY); @@ -133,13 +133,13 @@ void HTTPMessage::setChunkedTransferEncoding(bool flag) setTransferEncoding(IDENTITY_TRANSFER_ENCODING); } - + bool HTTPMessage::getChunkedTransferEncoding() const { return icompare(getTransferEncoding(), CHUNKED_TRANSFER_ENCODING) == 0; } - + void HTTPMessage::setContentType(const std::string& mediaType) { if (mediaType.empty()) @@ -154,7 +154,7 @@ void HTTPMessage::setContentType(const MediaType& mediaType) setContentType(mediaType.toString()); } - + const std::string& HTTPMessage::getContentType() const { return get(CONTENT_TYPE, UNKNOWN_CONTENT_TYPE); diff --git a/src/Backups/BackupOperationInfo.h b/src/Backups/BackupOperationInfo.h index 21b5284458c..71589ec3b30 100644 --- a/src/Backups/BackupOperationInfo.h +++ b/src/Backups/BackupOperationInfo.h @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { diff --git a/src/Common/AtomicLogger.h b/src/Common/AtomicLogger.h index 0ece9e8a09a..c1bbdb41866 100644 --- a/src/Common/AtomicLogger.h +++ b/src/Common/AtomicLogger.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include diff --git a/src/Common/ConcurrencyControl.h b/src/Common/ConcurrencyControl.h index ba94502962c..9d35d7cb8b0 100644 --- a/src/Common/ConcurrencyControl.h +++ b/src/Common/ConcurrencyControl.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Common/RemoteHostFilter.h b/src/Common/RemoteHostFilter.h index 2b91306f405..4c8983205fa 100644 --- a/src/Common/RemoteHostFilter.h +++ b/src/Common/RemoteHostFilter.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index c9b45d9a344..0f833c17e1b 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -5,6 +5,7 @@ #include #include +#include #include #include #include diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 2a53bade62f..e3289982b0d 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -2,9 +2,11 @@ #include "config.h" +#include #include #include #include +#include #include namespace DB diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ccab72cfbae..0ed995c6180 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -12,7 +13,7 @@ #include #include #include -#include "Common/logger_useful.h" +#include #include #include #include @@ -397,7 +398,7 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables() LOG_DEBUG(log, "There are {} detached tables. Start searching non used tables.", detached_tables.size()); while (it != detached_tables.end()) { - if (it->second.unique()) + if (isSharedPtrUnique(it->second)) { not_in_use.emplace(it->first, it->second); it = detached_tables.erase(it); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 233db07cd68..da942cebf8f 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -305,7 +306,7 @@ try String table_name = expired_tables.front().table_name; auto it = tables_cache.find(table_name); - if (!it->second.table || it->second.table.unique()) + if (!it->second.table || isSharedPtrUnique(it->second.table)) { LOG_DEBUG(log, "Drop table {} from cache.", backQuote(it->first)); it->second.table.reset(); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 1c82131af0d..bb24373a7e1 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -2,6 +2,7 @@ #if USE_MYSQL # include +# include # include # include # include @@ -354,7 +355,7 @@ void DatabaseMySQL::cleanOutdatedTables() { for (auto iterator = outdated_tables.begin(); iterator != outdated_tables.end();) { - if (!iterator->unique()) + if (!isSharedPtrUnique(*iterator)) ++iterator; else { diff --git a/src/Functions/FunctionsBitToArray.cpp b/src/Functions/FunctionsBitToArray.cpp index adabda1a7f8..81c80ae07bf 100644 --- a/src/Functions/FunctionsBitToArray.cpp +++ b/src/Functions/FunctionsBitToArray.cpp @@ -284,7 +284,13 @@ public: { while (x) { - result_array_values_data.push_back(std::countr_zero(x)); + /// ะก++20 char8_t is not an unsigned type anymore + /// https://stackoverflow.com/questions/57402464/is-c20-char8-t-the-same-as-our-old-char + // and thus you cannot use std::countr_zero on it. + if constexpr (std::is_same_v) + result_array_values_data.push_back(std::countr_zero(static_cast(x))); + else + result_array_values_data.push_back(std::countr_zero(x)); x &= (x - 1); } } @@ -336,4 +342,3 @@ REGISTER_FUNCTION(BitToArray) } } - diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index c3285d73145..9efb1d89a47 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -405,10 +405,6 @@ Block createBlockForSet( } -ScopeStack::Level::Level() = default; -ScopeStack::Level::~Level() = default; -ScopeStack::Level::Level(Level &&) noexcept = default; - FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) { @@ -462,6 +458,7 @@ public: for (const auto * node : index) map.emplace(node->result_name, node); } + ~Index() = default; void addNode(const ActionsDAG::Node * node) { @@ -502,6 +499,10 @@ public: } }; +ScopeStack::Level::Level() = default; +ScopeStack::Level::~Level() = default; +ScopeStack::Level::Level(Level &&) noexcept = default; + ActionsMatcher::Data::Data( ContextPtr context_, SizeLimits set_size_limit_, diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index a5c8f3c0cf4..d2158457a44 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -6,6 +6,8 @@ #include #include #include + +#include #include namespace DB @@ -30,7 +32,7 @@ struct FileSegmentMetadata : private boost::noncopyable explicit FileSegmentMetadata(FileSegmentPtr && file_segment_); - bool releasable() const { return file_segment.unique(); } + bool releasable() const { return file_segment.use_count() == 1; } size_t size() const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 841decf29c5..0cc88ac62f5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -27,6 +28,7 @@ #include #include +#include #include #include "config.h" @@ -1197,7 +1199,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) /// It's unsafe to create another instance while the old one exists /// We cannot wait on shared_ptr's refcount, so it's busy wait - while (!dropped_table.table.unique()) + while (!isSharedPtrUnique(dropped_table.table)) std::this_thread::sleep_for(std::chrono::milliseconds(100)); dropped_table.table.reset(); @@ -1237,7 +1239,7 @@ void DatabaseCatalog::dropTableDataTask() size_t tables_in_use_count = 0; auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem) { - bool not_in_use = !elem.table || elem.table.unique(); + bool not_in_use = !elem.table || isSharedPtrUnique(elem.table); bool old_enough = elem.drop_time <= current_time; min_drop_time = std::min(min_drop_time, elem.drop_time); tables_in_use_count += !not_in_use; diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 49b5e68d821..6356a174a01 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 21c773ee1d7..c35d2442d1f 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -18,12 +18,12 @@ #include #include #include -#include +// #include #include #include -#include +// #include #include -#include +// #include #include #include diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index bb8c415602f..fb80b12ee60 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -130,7 +131,7 @@ public: LOG_TRACE(log, "Reuse session from storage with session_id: {}, user_id: {}", key.second, key.first); - if (!session.unique()) + if (!isSharedPtrUnique(session)) throw Exception(ErrorCodes::SESSION_IS_LOCKED, "Session {} is locked by a concurrent client", session_id); return {session, false}; } @@ -156,7 +157,7 @@ public: return; } - if (!it->second.unique()) + if (!isSharedPtrUnique(it->second)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot close session {} with refcount {}", session_id, it->second.use_count()); sessions.erase(it); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 467a5c82141..e31f6db5409 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -87,6 +87,7 @@ #include #include +#include #include #include @@ -2464,7 +2465,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) } /// Grab only parts that are not used by anyone (SELECTs for example). - if (!part.unique()) + if (!isSharedPtrUnique(part)) { part->removal_state.store(DataPartRemovalState::NON_UNIQUE_OWNERSHIP, std::memory_order_relaxed); skipped_parts.push_back(part->info); @@ -4360,13 +4361,13 @@ bool MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) part.reset(); - if (!((*it)->getState() == DataPartState::Outdated && it->unique())) + if (!((*it)->getState() == DataPartState::Outdated && isSharedPtrUnique(*it))) { if ((*it)->getState() != DataPartState::Outdated) LOG_WARNING(log, "Cannot immediately remove part {} because it's not in Outdated state " "usage counter {}", part_name_with_state, it->use_count()); - if (!it->unique()) + if (!isSharedPtrUnique(*it)) LOG_WARNING(log, "Cannot immediately remove part {} because someone using it right now " "usage counter {}", part_name_with_state, it->use_count()); return false; @@ -4432,7 +4433,7 @@ size_t MergeTreeData::getNumberOfOutdatedPartsWithExpiredRemovalTime() const for (const auto & part : outdated_parts_range) { auto part_remove_time = part->remove_time.load(std::memory_order_relaxed); - if (part_remove_time <= time_now && time_now - part_remove_time >= getSettings()->old_parts_lifetime.totalSeconds() && part.unique()) + if (part_remove_time <= time_now && time_now - part_remove_time >= getSettings()->old_parts_lifetime.totalSeconds() && isSharedPtrUnique(part)) ++res; } @@ -8640,7 +8641,7 @@ size_t MergeTreeData::unloadPrimaryKeysOfOutdatedParts() /// Outdated part may be hold by SELECT query and still needs the index. /// This check requires lock of index_mutex but if outdated part is unique then there is no /// contention on it, so it's relatively cheap and it's ok to check under a global parts lock. - if (part.unique() && part->isIndexLoaded()) + if (isSharedPtrUnique(part) && part->isIndexLoaded()) parts_to_unload_index.push_back(part); } }