Modify the code for build with new libcxx

(cherry picked from commit c896f0bf677bcc948d988cf83b2108e9f7cd761d)
This commit is contained in:
Nikita Mikhaylov 2024-07-08 16:13:03 +00:00
parent 4add9db848
commit 7ff447de68
26 changed files with 108 additions and 55 deletions

View File

@ -1,5 +1,6 @@
#pragma once
#include <cstdlib>
#include <memory>
#include <string>

View File

@ -108,6 +108,14 @@ struct make_unsigned // NOLINT(readability-identifier-naming)
using type = std::make_unsigned_t<T>;
};
template <> struct make_unsigned<Int8> { using type = UInt8; };
template <> struct make_unsigned<UInt8> { using type = UInt8; };
template <> struct make_unsigned<Int16> { using type = UInt16; };
template <> struct make_unsigned<UInt16> { using type = UInt16; };
template <> struct make_unsigned<Int32> { using type = UInt32; };
template <> struct make_unsigned<UInt32> { using type = UInt32; };
template <> struct make_unsigned<Int64> { using type = UInt64; };
template <> struct make_unsigned<UInt64> { using type = UInt64; };
template <> struct make_unsigned<Int128> { using type = UInt128; };
template <> struct make_unsigned<UInt128> { using type = UInt128; };
template <> struct make_unsigned<Int256> { using type = UInt256; };
@ -121,6 +129,14 @@ struct make_signed // NOLINT(readability-identifier-naming)
using type = std::make_signed_t<T>;
};
template <> struct make_signed<Int8> { using type = Int8; };
template <> struct make_signed<UInt8> { using type = Int8; };
template <> struct make_signed<Int16> { using type = Int16; };
template <> struct make_signed<UInt16> { using type = Int16; };
template <> struct make_signed<Int32> { using type = Int32; };
template <> struct make_signed<UInt32> { using type = Int32; };
template <> struct make_signed<Int64> { using type = Int64; };
template <> struct make_signed<UInt64> { using type = Int64; };
template <> struct make_signed<Int128> { using type = Int128; };
template <> struct make_signed<UInt128> { using type = Int128; };
template <> struct make_signed<Int256> { using type = Int256; };

View File

@ -0,0 +1,9 @@
#pragma once
#include <memory>
template <typename T>
bool isSharedPtrUnique(const std::shared_ptr<T> & ptr)
{
return ptr.use_count() == 1;
}

View File

@ -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<Any> & values);
void Foundation_API formatVector(std::string & result, const std::string & fmt, const std::vector<Any> & values);
/// Supports a variable number of arguments and is used by
/// all other variants of format().

View File

@ -21,6 +21,8 @@
#include "Poco/AtomicCounter.h"
#include "Poco/Foundation.h"
#include <atomic>
namespace Poco
{

View File

@ -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<std::string>(*itVal++);
break;
case 'z':
str << AnyCast<std::size_t>(*itVal++);
str << AnyCast<std::size_t>(*itVal++);
break;
case 'I':
case 'D':
@ -303,7 +303,7 @@ void format(std::string& result, const std::string& fmt, const Any& value)
{
std::vector<Any> 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<Any> 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<Any>& values)
void formatVector(std::string& result, const std::string& fmt, const std::vector<Any>& values)
{
std::string::const_iterator itFmt = fmt.begin();
std::string::const_iterator endFmt = fmt.end();
std::vector<Any>::const_iterator itVal = values.begin();
std::vector<Any>::const_iterator endVal = values.end();
std::vector<Any>::const_iterator endVal = values.end();
while (itFmt != endFmt)
{
switch (*itFmt)

View File

@ -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;
}

View File

@ -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);

View File

@ -17,9 +17,9 @@
#include "Poco/NumberFormatter.h"
#include "Poco/NumberParser.h"
#include "Poco/String.h"
#include <charconv>
#include <format>
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);

View File

@ -3,6 +3,8 @@
#include <Backups/BackupStatus.h>
#include <Common/ProfileEvents.h>
#include <exception>
namespace DB
{

View File

@ -1,5 +1,7 @@
#pragma once
#include <atomic>
#include <mutex>
#include <memory>
#include <base/types.h>

View File

@ -1,5 +1,6 @@
#pragma once
#include <atomic>
#include <list>
#include <memory>
#include <mutex>

View File

@ -1,5 +1,6 @@
#pragma once
#include <atomic>
#include <string>
#include <vector>
#include <mutex>

View File

@ -5,6 +5,7 @@
#include <Common/ConcurrentBoundedQueue.h>
#include <map>
#include <variant>
#include <unordered_map>
#include <unordered_set>
#include <future>

View File

@ -2,9 +2,11 @@
#include "config.h"
#include <atomic>
#include <memory>
#include <unordered_map>
#include <string>
#include <vector>
#include <boost/noncopyable.hpp>
namespace DB

View File

@ -1,4 +1,5 @@
#include <filesystem>
#include <base/isSharedPtrUnique.h>
#include <Databases/DatabaseAtomic.h>
#include <Databases/DatabaseFactory.h>
#include <Databases/DatabaseOnDisk.h>
@ -12,7 +13,7 @@
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Parsers/formatAST.h>
#include <Storages/StorageMaterializedView.h>
#include "Common/logger_useful.h"
#include <Common/logger_useful.h>
#include <Common/PoolId.h>
#include <Common/atomicRename.h>
#include <Common/filesystemHelpers.h>
@ -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);

View File

@ -1,6 +1,7 @@
#include <Databases/DatabaseLazy.h>
#include <base/sort.h>
#include <base/isSharedPtrUnique.h>
#include <iomanip>
#include <filesystem>
#include <Common/CurrentMetrics.h>
@ -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();

View File

@ -2,6 +2,7 @@
#if USE_MYSQL
# include <string>
# include <base/isSharedPtrUnique.h>
# include <Databases/DatabaseFactory.h>
# include <DataTypes/DataTypeDateTime.h>
# include <DataTypes/DataTypeNullable.h>
@ -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
{

View File

@ -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<UnsignedType, UInt8>)
result_array_values_data.push_back(std::countr_zero(static_cast<unsigned char>(x)));
else
result_array_values_data.push_back(std::countr_zero(x));
x &= (x - 1);
}
}
@ -336,4 +342,3 @@ REGISTER_FUNCTION(BitToArray)
}
}

View File

@ -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_,

View File

@ -6,6 +6,8 @@
#include <Interpreters/Cache/FileSegment.h>
#include <Interpreters/Cache/FileCache_fwd_internal.h>
#include <Common/ThreadPool.h>
#include <memory>
#include <shared_mutex>
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;

View File

@ -1,3 +1,4 @@
#include <algorithm>
#include <string>
#include <mutex>
#include <Interpreters/DatabaseCatalog.h>
@ -27,6 +28,7 @@
#include <Common/noexcept_scope.h>
#include <Common/checkStackSize.h>
#include <base/isSharedPtrUnique.h>
#include <boost/range/adaptor/map.hpp>
#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;

View File

@ -2,6 +2,7 @@
#include <chrono>
#include <functional>
#include <exception>
#include <unordered_map>
#include <base/types.h>
#include <Interpreters/IExternalLoadable.h>

View File

@ -18,12 +18,12 @@
#include <llvm/ExecutionEngine/JITSymbol.h>
#include <llvm/ExecutionEngine/SectionMemoryManager.h>
#include <llvm/ExecutionEngine/JITEventListener.h>
#include <llvm/MC/SubtargetFeature.h>
// #include <llvm/MC/SubtargetFeature.h>
#include <llvm/MC/TargetRegistry.h>
#include <llvm/Support/DynamicLibrary.h>
#include <llvm/Support/Host.h>
// #include <llvm/Support/Host.h>
#include <llvm/Support/TargetSelect.h>
#include <llvm/Transforms/IPO/PassManagerBuilder.h>
// #include <llvm/Transforms/IPO/PassManagerBuilder.h>
#include <llvm/Support/SmallVectorMemoryBuffer.h>
#include <base/getPageSize.h>

View File

@ -1,5 +1,6 @@
#include <Interpreters/Session.h>
#include <base/isSharedPtrUnique.h>
#include <Access/AccessControl.h>
#include <Access/Credentials.h>
#include <Access/ContextAccess.h>
@ -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);

View File

@ -87,6 +87,7 @@
#include <base/insertAtEnd.h>
#include <base/interpolate.h>
#include <base/isSharedPtrUnique.h>
#include <algorithm>
#include <atomic>
@ -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);
}
}