mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #47027 from amosbird/clang16
Prepare to build with clang 16
This commit is contained in:
commit
97c0f8a0e0
@ -209,3 +209,5 @@ CheckOptions:
|
||||
# Workaround clang-tidy bug: https://github.com/llvm/llvm-project/issues/46097
|
||||
- key: readability-identifier-naming.TypeTemplateParameterIgnoredRegexp
|
||||
value: expr-type
|
||||
- key: cppcoreguidelines-avoid-do-while.IgnoreMacros
|
||||
value: true
|
||||
|
@ -391,10 +391,12 @@ if (COMPILER_CLANG)
|
||||
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fstrict-vtable-pointers")
|
||||
|
||||
# Set new experimental pass manager, it's a performance, build time and binary size win.
|
||||
# Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang.
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager")
|
||||
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS 16)
|
||||
# Set new experimental pass manager, it's a performance, build time and binary size win.
|
||||
# Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang.
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager")
|
||||
endif ()
|
||||
|
||||
# We cannot afford to use LTO when compiling unit tests, and it's not enough
|
||||
# to only supply -fno-lto at the final linking stage. So we disable it
|
||||
|
@ -8,3 +8,8 @@ int fallocate(int fd, int mode, off_t base, off_t len)
|
||||
{
|
||||
return syscall(SYS_fallocate, fd, mode, base, len);
|
||||
}
|
||||
|
||||
int fallocate64(int fd, int mode, off_t base, off_t len)
|
||||
{
|
||||
return fallocate(fd, mode, base, len);
|
||||
}
|
||||
|
@ -9,3 +9,8 @@ ssize_t pwritev(int fd, const struct iovec *iov, int count, off_t ofs)
|
||||
/// There was cancellable syscall (syscall_cp), but I don't care.
|
||||
return syscall(SYS_pwritev, fd, iov, count, (long)(ofs), (long)(ofs>>32));
|
||||
}
|
||||
|
||||
ssize_t pwritev64(int fd, const struct iovec *iov, int count, off_t ofs)
|
||||
{
|
||||
return pwritev(fd, iov, count, ofs);
|
||||
}
|
||||
|
@ -67,19 +67,7 @@ public:
|
||||
void swap(Timespan & timespan);
|
||||
/// Swaps the Timespan with another one.
|
||||
|
||||
bool operator==(const Timespan & ts) const;
|
||||
bool operator!=(const Timespan & ts) const;
|
||||
bool operator>(const Timespan & ts) const;
|
||||
bool operator>=(const Timespan & ts) const;
|
||||
bool operator<(const Timespan & ts) const;
|
||||
bool operator<=(const Timespan & ts) const;
|
||||
|
||||
bool operator==(TimeDiff microSeconds) const;
|
||||
bool operator!=(TimeDiff microSeconds) const;
|
||||
bool operator>(TimeDiff microSeconds) const;
|
||||
bool operator>=(TimeDiff microSeconds) const;
|
||||
bool operator<(TimeDiff microSeconds) const;
|
||||
bool operator<=(TimeDiff microSeconds) const;
|
||||
auto operator<=>(const Timespan & ts) const = default;
|
||||
|
||||
Timespan operator+(const Timespan & d) const;
|
||||
Timespan operator-(const Timespan & d) const;
|
||||
@ -215,78 +203,6 @@ inline Timespan::TimeDiff Timespan::totalMicroseconds() const
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator==(const Timespan & ts) const
|
||||
{
|
||||
return _span == ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator!=(const Timespan & ts) const
|
||||
{
|
||||
return _span != ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator>(const Timespan & ts) const
|
||||
{
|
||||
return _span > ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator>=(const Timespan & ts) const
|
||||
{
|
||||
return _span >= ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator<(const Timespan & ts) const
|
||||
{
|
||||
return _span < ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator<=(const Timespan & ts) const
|
||||
{
|
||||
return _span <= ts._span;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator==(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span == microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator!=(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span != microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator>(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span > microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator>=(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span >= microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator<(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span < microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline bool Timespan::operator<=(TimeDiff microSeconds) const
|
||||
{
|
||||
return _span <= microSeconds;
|
||||
}
|
||||
|
||||
|
||||
inline void swap(Timespan & s1, Timespan & s2)
|
||||
{
|
||||
s1.swap(s2);
|
||||
|
@ -45,6 +45,7 @@ if (COMPILER_CLANG)
|
||||
no_warning(weak-vtables)
|
||||
no_warning(thread-safety-negative) # experimental flag, too many false positives
|
||||
no_warning(enum-constexpr-conversion) # breaks magic-enum library in clang-16
|
||||
no_warning(unsafe-buffer-usage) # too aggressive
|
||||
# TODO Enable conversion, sign-conversion, double-promotion warnings.
|
||||
elseif (COMPILER_GCC)
|
||||
# Add compiler options only to c++ compiler
|
||||
|
2
contrib/capnproto
vendored
2
contrib/capnproto
vendored
@ -1 +1 @@
|
||||
Subproject commit e19cd661e49dd9022d3f920b69d843333b896451
|
||||
Subproject commit dc8b50b999777bcb23c89bb5907c785c3f654441
|
@ -98,6 +98,16 @@ set(LLVM_ENABLE_BINDINGS 0 CACHE INTERNAL "")
|
||||
|
||||
set (LLVM_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm-project/llvm")
|
||||
set (LLVM_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/llvm-project/llvm")
|
||||
|
||||
# Since we always use toolchain files to generate hermatic builds, cmake will
|
||||
# think it's a cross compilation, and LLVM will try to configure NATIVE LLVM
|
||||
# targets with all tests enabled, which will slow down cmake configuration and
|
||||
# compilation (You'll see Building native llvm-tblgen...). Let's disable the
|
||||
# cross compiling indicator for now.
|
||||
#
|
||||
# TODO We should let cmake know whether it's indeed a cross compilation in the
|
||||
# first place.
|
||||
set (CMAKE_CROSSCOMPILING 0)
|
||||
add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}")
|
||||
|
||||
set_directory_properties (PROPERTIES
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Common/VersionNumber.h>
|
||||
#include <cstdlib>
|
||||
#include <iostream>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -10,29 +10,21 @@ VersionNumber::VersionNumber(std::string version_string)
|
||||
if (version_string.empty())
|
||||
return;
|
||||
|
||||
char * start = &version_string.front();
|
||||
char * end = start;
|
||||
const char * eos = &version_string.back() + 1;
|
||||
|
||||
do
|
||||
ReadBufferFromString rb(version_string);
|
||||
while (!rb.eof())
|
||||
{
|
||||
Int64 value = strtol(start, &end, 10);
|
||||
Int64 value;
|
||||
if (!tryReadIntText(value, rb))
|
||||
break;
|
||||
components.push_back(value);
|
||||
start = end + 1;
|
||||
if (!checkChar('.', rb))
|
||||
break;
|
||||
}
|
||||
while (start < eos && (end < eos && *end == '.'));
|
||||
}
|
||||
|
||||
std::string VersionNumber::toString() const
|
||||
{
|
||||
std::string str;
|
||||
for (Int64 v : components)
|
||||
{
|
||||
if (!str.empty())
|
||||
str += '.';
|
||||
str += std::to_string(v);
|
||||
}
|
||||
return str;
|
||||
return fmt::format("{}", fmt::join(components, "."));
|
||||
}
|
||||
|
||||
int VersionNumber::compare(const VersionNumber & rhs) const
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <tuple>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <iostream>
|
||||
@ -10,33 +9,24 @@ namespace DB
|
||||
{
|
||||
|
||||
/// Simple numeric version representation.
|
||||
///
|
||||
/// Based on QVersionNumber.
|
||||
struct VersionNumber
|
||||
{
|
||||
explicit VersionNumber() = default;
|
||||
|
||||
VersionNumber(const std::initializer_list<Int64> & init)
|
||||
: components(init)
|
||||
{}
|
||||
VersionNumber(Int64 major, Int64 minor = 0, Int64 patch = 0) /// NOLINT
|
||||
: components{major, minor, patch}
|
||||
{}
|
||||
VersionNumber(const std::vector<Int64> & components_) /// NOLINT
|
||||
: components(components_)
|
||||
{}
|
||||
VersionNumber(const std::initializer_list<Int64> & init) : components(init) {}
|
||||
explicit VersionNumber(Int64 major, Int64 minor = 0, Int64 patch = 0) : components{major, minor, patch} {}
|
||||
explicit VersionNumber(const std::vector<Int64> & components_) : components(components_) {}
|
||||
|
||||
/// Parse version number from string.
|
||||
explicit VersionNumber(std::string version);
|
||||
|
||||
/// NOTE: operator<=> can be used once libc++ will be upgraded.
|
||||
bool operator<(const VersionNumber & rhs) const { return compare(rhs.components) < 0; }
|
||||
bool operator<=(const VersionNumber & rhs) const { return compare(rhs.components) <= 0; }
|
||||
bool operator==(const VersionNumber & rhs) const { return compare(rhs.components) == 0; }
|
||||
bool operator>(const VersionNumber & rhs) const { return compare(rhs.components) > 0; }
|
||||
bool operator>=(const VersionNumber & rhs) const { return compare(rhs.components) >= 0; }
|
||||
bool operator==(const VersionNumber & rhs) const = default;
|
||||
|
||||
/// There might be negative version code which differs from default comparison.
|
||||
auto operator<=>(const VersionNumber & rhs) const { return compare(rhs); }
|
||||
|
||||
std::string toString() const;
|
||||
|
||||
friend std::ostream & operator<<(std::ostream & os, const VersionNumber & v)
|
||||
{
|
||||
return os << v.toString();
|
||||
|
@ -322,7 +322,7 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read(std::unique_ptr<ReadCo
|
||||
read_context = std::make_unique<ReadContext>(*connections);
|
||||
}
|
||||
|
||||
do
|
||||
while (true)
|
||||
{
|
||||
if (!read_context->resumeRoutine())
|
||||
return ReadResult(Block());
|
||||
@ -349,7 +349,6 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read(std::unique_ptr<ReadCo
|
||||
return restartQueryWithoutDuplicatedUUIDs(&read_context);
|
||||
}
|
||||
}
|
||||
while (true);
|
||||
#else
|
||||
return read();
|
||||
#endif
|
||||
|
@ -54,6 +54,7 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/QueryPlan/QueryIdHolder.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/Freeze.h>
|
||||
@ -7783,6 +7784,25 @@ void MergeTreeData::removeQueryIdNoLock(const String & query_id) const
|
||||
query_id_set.erase(query_id);
|
||||
}
|
||||
|
||||
std::shared_ptr<QueryIdHolder> MergeTreeData::getQueryIdHolder(const String & query_id, UInt64 max_concurrent_queries) const
|
||||
{
|
||||
auto lock = std::lock_guard<std::mutex>(query_id_set_mutex);
|
||||
if (insertQueryIdOrThrowNoLock(query_id, max_concurrent_queries))
|
||||
{
|
||||
try
|
||||
{
|
||||
return std::make_shared<QueryIdHolder>(query_id, *this);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If we fail to construct the holder, remove query_id explicitly to avoid leak.
|
||||
removeQueryIdNoLock(query_id);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ReservationPtr MergeTreeData::balancedReservation(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
size_t part_size,
|
||||
|
@ -958,7 +958,7 @@ public:
|
||||
/// section from config.xml.
|
||||
CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const;
|
||||
|
||||
std::lock_guard<std::mutex> getQueryIdSetLock() const { return std::lock_guard<std::mutex>(query_id_set_mutex); }
|
||||
std::shared_ptr<QueryIdHolder> getQueryIdHolder(const String & query_id, UInt64 max_concurrent_queries) const;
|
||||
|
||||
/// Record current query id where querying the table. Throw if there are already `max_queries` queries accessing the same table.
|
||||
/// Returns false if the `query_id` already exists in the running set, otherwise return true.
|
||||
|
@ -1198,7 +1198,6 @@ std::shared_ptr<QueryIdHolder> MergeTreeDataSelectExecutor::checkLimits(
|
||||
const MergeTreeData & data,
|
||||
const ReadFromMergeTree::AnalysisResult & result,
|
||||
const ContextPtr & context)
|
||||
TSA_NO_THREAD_SAFETY_ANALYSIS // disabled because TSA is confused by guaranteed copy elision in data.getQueryIdSetLock()
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
const auto data_settings = data.getSettings();
|
||||
@ -1222,22 +1221,7 @@ std::shared_ptr<QueryIdHolder> MergeTreeDataSelectExecutor::checkLimits(
|
||||
{
|
||||
auto query_id = context->getCurrentQueryId();
|
||||
if (!query_id.empty())
|
||||
{
|
||||
auto lock = data.getQueryIdSetLock();
|
||||
if (data.insertQueryIdOrThrowNoLock(query_id, data_settings->max_concurrent_queries))
|
||||
{
|
||||
try
|
||||
{
|
||||
return std::make_shared<QueryIdHolder>(query_id, data);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If we fail to construct the holder, remove query_id explicitly to avoid leak.
|
||||
data.removeQueryIdNoLock(query_id);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
return data.getQueryIdHolder(query_id, data_settings->max_concurrent_queries);
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user