Merge remote-tracking branch 'blessed/master' into disable_adaptative

This commit is contained in:
Raúl Marín 2024-03-19 12:10:15 +01:00
commit 1c4ff83356
328 changed files with 2662 additions and 1622 deletions

View File

@ -5,9 +5,7 @@
# a) the new check is not controversial (this includes many checks in readability-* and google-*) or
# b) too noisy (checks with > 100 new warnings are considered noisy, this includes e.g. cppcoreguidelines-*).
# TODO Let clang-tidy check headers in further directories
# --> HeaderFilterRegex: '^.*/(src|base|programs|utils)/.*(h|hpp)$'
HeaderFilterRegex: '^.*/(base|programs|utils)/.*(h|hpp)$'
HeaderFilterRegex: '^.*/(base|src|programs|utils)/.*(h|hpp)$'
Checks: [
'*',
@ -22,6 +20,7 @@ Checks: [
'-bugprone-branch-clone',
'-bugprone-easily-swappable-parameters',
'-bugprone-exception-escape',
'-bugprone-forward-declaration-namespace',
'-bugprone-implicit-widening-of-multiplication-result',
'-bugprone-narrowing-conversions',
'-bugprone-not-null-terminated-result',
@ -37,6 +36,8 @@ Checks: [
'-cert-oop54-cpp',
'-cert-oop57-cpp',
'-clang-analyzer-optin.performance.Padding',
'-clang-analyzer-unix.Malloc',
'-cppcoreguidelines-*', # impractical in a codebase as large as ClickHouse, also slow

View File

@ -30,7 +30,6 @@ namespace Net
class HTTPServerRequest;
class HTTPServerResponse;
class HTTPRequestHandler;

View File

@ -26,13 +26,13 @@ const uint8_t MetroHash64::test_seed_1[8] = { 0x3B, 0x0D, 0x48, 0x1C, 0xF4, 0x
MetroHash64::MetroHash64(const uint64_t seed)
MetroHash64::MetroHash64(uint64_t seed)
{
Initialize(seed);
}
void MetroHash64::Initialize(const uint64_t seed)
void MetroHash64::Initialize(uint64_t seed)
{
vseed = (static_cast<uint64_t>(seed) + k2) * k0;
@ -47,7 +47,7 @@ void MetroHash64::Initialize(const uint64_t seed)
}
void MetroHash64::Update(const uint8_t * const buffer, const uint64_t length)
void MetroHash64::Update(const uint8_t * const buffer, uint64_t length)
{
const uint8_t * ptr = reinterpret_cast<const uint8_t*>(buffer);
const uint8_t * const end = ptr + length;
@ -62,7 +62,7 @@ void MetroHash64::Update(const uint8_t * const buffer, const uint64_t length)
memcpy(input.b + (bytes % 32), ptr, static_cast<size_t>(fill));
ptr += fill;
bytes += fill;
// input buffer is still partially filled
if ((bytes % 32) != 0) return;
@ -72,7 +72,7 @@ void MetroHash64::Update(const uint8_t * const buffer, const uint64_t length)
state.v[2] += read_u64(&input.b[16]) * k2; state.v[2] = rotate_right(state.v[2],29) + state.v[0];
state.v[3] += read_u64(&input.b[24]) * k3; state.v[3] = rotate_right(state.v[3],29) + state.v[1];
}
// bulk update
bytes += static_cast<uint64_t>(end - ptr);
while (ptr <= (end - 32))
@ -83,14 +83,14 @@ void MetroHash64::Update(const uint8_t * const buffer, const uint64_t length)
state.v[2] += read_u64(ptr) * k2; ptr += 8; state.v[2] = rotate_right(state.v[2],29) + state.v[0];
state.v[3] += read_u64(ptr) * k3; ptr += 8; state.v[3] = rotate_right(state.v[3],29) + state.v[1];
}
// store remaining bytes in input buffer
if (ptr < end)
memcpy(input.b, ptr, static_cast<size_t>(end - ptr));
}
void MetroHash64::Finalize(uint8_t * const hash)
void MetroHash64::Finalize(uint8_t * hash)
{
// finalize bulk loop, if used
if (bytes >= 32)
@ -102,11 +102,11 @@ void MetroHash64::Finalize(uint8_t * const hash)
state.v[0] = vseed + (state.v[0] ^ state.v[1]);
}
// process any bytes remaining in the input buffer
const uint8_t * ptr = reinterpret_cast<const uint8_t*>(input.b);
const uint8_t * const end = ptr + (bytes % 32);
if ((end - ptr) >= 16)
{
state.v[1] = state.v[0] + (read_u64(ptr) * k2); ptr += 8; state.v[1] = rotate_right(state.v[1],29) * k3;
@ -139,7 +139,7 @@ void MetroHash64::Finalize(uint8_t * const hash)
state.v[0] += read_u8 (ptr) * k3;
state.v[0] ^= rotate_right(state.v[0], 37) * k1;
}
state.v[0] ^= rotate_right(state.v[0], 28);
state.v[0] *= k0;
state.v[0] ^= rotate_right(state.v[0], 29);
@ -152,7 +152,7 @@ void MetroHash64::Finalize(uint8_t * const hash)
}
void MetroHash64::Hash(const uint8_t * buffer, const uint64_t length, uint8_t * const hash, const uint64_t seed)
void MetroHash64::Hash(const uint8_t * buffer, uint64_t length, uint8_t * const hash, uint64_t seed)
{
const uint8_t * ptr = reinterpret_cast<const uint8_t*>(buffer);
const uint8_t * const end = ptr + length;
@ -238,7 +238,7 @@ bool MetroHash64::ImplementationVerified()
// verify incremental implementation
MetroHash64 metro;
metro.Initialize(0);
metro.Update(reinterpret_cast<const uint8_t *>(MetroHash64::test_string), strlen(MetroHash64::test_string));
metro.Finalize(hash);
@ -262,9 +262,9 @@ void metrohash64_1(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
const uint8_t * ptr = reinterpret_cast<const uint8_t*>(key);
const uint8_t * const end = ptr + len;
uint64_t hash = ((static_cast<uint64_t>(seed) + k2) * k0) + len;
if (len >= 32)
{
uint64_t v[4];
@ -272,7 +272,7 @@ void metrohash64_1(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v[1] = hash;
v[2] = hash;
v[3] = hash;
do
{
v[0] += read_u64(ptr) * k0; ptr += 8; v[0] = rotate_right(v[0],29) + v[2];
@ -288,7 +288,7 @@ void metrohash64_1(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v[1] ^= rotate_right(((v[1] + v[3]) * k1) + v[2], 33) * k0;
hash += v[0] ^ v[1];
}
if ((end - ptr) >= 16)
{
uint64_t v0 = hash + (read_u64(ptr) * k0); ptr += 8; v0 = rotate_right(v0,33) * k1;
@ -297,32 +297,32 @@ void metrohash64_1(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v1 ^= rotate_right(v1 * k3, 35) + v0;
hash += v1;
}
if ((end - ptr) >= 8)
{
hash += read_u64(ptr) * k3; ptr += 8;
hash ^= rotate_right(hash, 33) * k1;
}
if ((end - ptr) >= 4)
{
hash += read_u32(ptr) * k3; ptr += 4;
hash ^= rotate_right(hash, 15) * k1;
}
if ((end - ptr) >= 2)
{
hash += read_u16(ptr) * k3; ptr += 2;
hash ^= rotate_right(hash, 13) * k1;
}
if ((end - ptr) >= 1)
{
hash += read_u8 (ptr) * k3;
hash ^= rotate_right(hash, 25) * k1;
}
hash ^= rotate_right(hash, 33);
hash *= k0;
hash ^= rotate_right(hash, 33);
@ -336,13 +336,13 @@ void metrohash64_2(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
static const uint64_t k0 = 0xD6D018F5;
static const uint64_t k1 = 0xA2AA033B;
static const uint64_t k2 = 0x62992FC1;
static const uint64_t k3 = 0x30BC5B29;
static const uint64_t k3 = 0x30BC5B29;
const uint8_t * ptr = reinterpret_cast<const uint8_t*>(key);
const uint8_t * const end = ptr + len;
uint64_t hash = ((static_cast<uint64_t>(seed) + k2) * k0) + len;
if (len >= 32)
{
uint64_t v[4];
@ -350,7 +350,7 @@ void metrohash64_2(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v[1] = hash;
v[2] = hash;
v[3] = hash;
do
{
v[0] += read_u64(ptr) * k0; ptr += 8; v[0] = rotate_right(v[0],29) + v[2];
@ -366,7 +366,7 @@ void metrohash64_2(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v[1] ^= rotate_right(((v[1] + v[3]) * k1) + v[2], 30) * k0;
hash += v[0] ^ v[1];
}
if ((end - ptr) >= 16)
{
uint64_t v0 = hash + (read_u64(ptr) * k2); ptr += 8; v0 = rotate_right(v0,29) * k3;
@ -375,31 +375,31 @@ void metrohash64_2(const uint8_t * key, uint64_t len, uint32_t seed, uint8_t * o
v1 ^= rotate_right(v1 * k3, 34) + v0;
hash += v1;
}
if ((end - ptr) >= 8)
{
hash += read_u64(ptr) * k3; ptr += 8;
hash ^= rotate_right(hash, 36) * k1;
}
if ((end - ptr) >= 4)
{
hash += read_u32(ptr) * k3; ptr += 4;
hash ^= rotate_right(hash, 15) * k1;
}
if ((end - ptr) >= 2)
{
hash += read_u16(ptr) * k3; ptr += 2;
hash ^= rotate_right(hash, 15) * k1;
}
if ((end - ptr) >= 1)
{
hash += read_u8 (ptr) * k3;
hash ^= rotate_right(hash, 23) * k1;
}
hash ^= rotate_right(hash, 28);
hash *= k0;
hash ^= rotate_right(hash, 29);

View File

@ -25,24 +25,24 @@ public:
static const uint32_t bits = 64;
// Constructor initializes the same as Initialize()
explicit MetroHash64(const uint64_t seed=0);
explicit MetroHash64(uint64_t seed=0);
// Initializes internal state for new hash with optional seed
void Initialize(const uint64_t seed=0);
void Initialize(uint64_t seed=0);
// Update the hash state with a string of bytes. If the length
// is sufficiently long, the implementation switches to a bulk
// hashing algorithm directly on the argument buffer for speed.
void Update(const uint8_t * buffer, const uint64_t length);
void Update(const uint8_t * buffer, uint64_t length);
// Constructs the final hash and writes it to the argument buffer.
// After a hash is finalized, this instance must be Initialized()-ed
// again or the behavior of Update() and Finalize() is undefined.
void Finalize(uint8_t * const hash);
void Finalize(uint8_t * hash);
// A non-incremental function implementation. This can be significantly
// faster than the incremental implementation for some usage patterns.
static void Hash(const uint8_t * buffer, const uint64_t length, uint8_t * const hash, const uint64_t seed=0);
static void Hash(const uint8_t * buffer, uint64_t length, uint8_t * hash, uint64_t seed=0);
// Does implementation correctly execute test vectors?
static bool ImplementationVerified();

View File

@ -61,6 +61,18 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th
rm /etc/clickhouse-server/users.d/s3_cache_new.xml
rm /etc/clickhouse-server/config.d/zero_copy_destructive_operations.xml
#todo: remove these after 24.3 released.
sudo cat /etc/clickhouse-server/config.d/azure_storage_conf.xml \
| sed "s|<object_storage_type>azure|<object_storage_type>azure_blob_storage|" \
> /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp
sudo mv /etc/clickhouse-server/config.d/azure_storage_conf.xml.tmp /etc/clickhouse-server/config.d/azure_storage_conf.xml
#todo: remove these after 24.3 released.
sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \
| sed "s|<object_storage_type>local|<object_storage_type>local_blob_storage|" \
> /etc/clickhouse-server/config.d/storage_conf.xml.tmp
sudo mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml
function remove_keeper_config()
{
sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \

View File

@ -350,6 +350,7 @@ ALTER TABLE mt DELETE IN PARTITION ID '2' WHERE p = 2;
You can specify the partition expression in `ALTER ... PARTITION` queries in different ways:
- As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`.
- Using the keyword `ALL`. It can be used only with DROP/DETACH/ATTACH. For example, `ALTER TABLE visits ATTACH PARTITION ALL`.
- As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`.
- Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`.
- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`.

View File

@ -2,6 +2,7 @@
#include "Commands.h"
#include <queue>
#include "KeeperClient.h"
#include "Parsers/CommonParsers.h"
namespace DB
@ -106,13 +107,13 @@ bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> &
int mode = zkutil::CreateMode::Persistent;
if (ParserKeyword{"PERSISTENT"}.ignore(pos, expected))
if (ParserKeyword(Keyword::PERSISTENT).ignore(pos, expected))
mode = zkutil::CreateMode::Persistent;
else if (ParserKeyword{"EPHEMERAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::EPHEMERAL).ignore(pos, expected))
mode = zkutil::CreateMode::Ephemeral;
else if (ParserKeyword{"EPHEMERAL SEQUENTIAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::EPHEMERAL_SEQUENTIAL).ignore(pos, expected))
mode = zkutil::CreateMode::EphemeralSequential;
else if (ParserKeyword{"PERSISTENT SEQUENTIAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::PERSISTENT_SEQUENTIAL).ignore(pos, expected))
mode = zkutil::CreateMode::PersistentSequential;
node->args.push_back(std::move(mode));
@ -382,12 +383,16 @@ void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, DB::Expected & expected) const
{
ParserKeyword s_add(Keyword::ADD);
ParserKeyword s_remove(Keyword::REMOVE);
ParserKeyword s_set(Keyword::SET);
ReconfigCommand::Operation operation;
if (ParserKeyword{"ADD"}.ignore(pos, expected))
if (s_add.ignore(pos, expected))
operation = ReconfigCommand::Operation::ADD;
else if (ParserKeyword{"REMOVE"}.ignore(pos, expected))
else if (s_remove.ignore(pos, expected))
operation = ReconfigCommand::Operation::REMOVE;
else if (ParserKeyword{"SET"}.ignore(pos, expected))
else if (s_set.ignore(pos, expected))
operation = ReconfigCommand::Operation::SET;
else
return false;

View File

@ -297,7 +297,7 @@ namespace
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> access_entities,
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
size_t counter,
const AccessControl & access_control)
@ -326,7 +326,7 @@ void AccessRestorerFromBackup::addDataPath(const String & data_path)
return;
fs::path data_path_in_backup_fs = data_path;
Strings filenames = backup->listFiles(data_path);
Strings filenames = backup->listFiles(data_path, /*recursive*/ false);
if (filenames.empty())
return;

View File

@ -21,7 +21,7 @@ struct RestoreSettings;
/// Makes a backup of access entities of a specified type.
std::pair<String, BackupEntryPtr> makeBackupEntryForAccess(
const std::vector<std::pair<UUID, AccessEntityPtr>> access_entities,
const std::vector<std::pair<UUID, AccessEntityPtr>> & access_entities,
const String & data_path_in_backup,
size_t counter,
const AccessControl & access_control);

View File

@ -13,63 +13,63 @@ namespace ErrorCodes
const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType type_)
{
static constexpr auto make_info = [](const char * raw_name_, bool is_password_ = false)
static constexpr auto make_info = [](Keyword keyword_, bool is_password_ = false)
{
String init_name = raw_name_;
String init_name = String(toStringView(keyword_));
boost::to_lower(init_name);
return AuthenticationTypeInfo{raw_name_, std::move(init_name), is_password_};
return AuthenticationTypeInfo{keyword_, std::move(init_name), is_password_};
};
switch (type_)
{
case AuthenticationType::NO_PASSWORD:
{
static const auto info = make_info("NO_PASSWORD");
static const auto info = make_info(Keyword::NO_PASSWORD);
return info;
}
case AuthenticationType::PLAINTEXT_PASSWORD:
{
static const auto info = make_info("PLAINTEXT_PASSWORD", true);
static const auto info = make_info(Keyword::PLAINTEXT_PASSWORD, true);
return info;
}
case AuthenticationType::SHA256_PASSWORD:
{
static const auto info = make_info("SHA256_PASSWORD", true);
static const auto info = make_info(Keyword::SHA256_PASSWORD, true);
return info;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
static const auto info = make_info("DOUBLE_SHA1_PASSWORD", true);
static const auto info = make_info(Keyword::DOUBLE_SHA1_PASSWORD, true);
return info;
}
case AuthenticationType::LDAP:
{
static const auto info = make_info("LDAP");
static const auto info = make_info(Keyword::LDAP);
return info;
}
case AuthenticationType::KERBEROS:
{
static const auto info = make_info("KERBEROS");
static const auto info = make_info(Keyword::KERBEROS);
return info;
}
case AuthenticationType::SSL_CERTIFICATE:
{
static const auto info = make_info("SSL_CERTIFICATE");
static const auto info = make_info(Keyword::SSL_CERTIFICATE);
return info;
}
case AuthenticationType::BCRYPT_PASSWORD:
{
static const auto info = make_info("BCRYPT_PASSWORD", true);
static const auto info = make_info(Keyword::BCRYPT_PASSWORD, true);
return info;
}
case AuthenticationType::SSH_KEY:
{
static const auto info = make_info("SSH_KEY");
static const auto info = make_info(Keyword::SSH_KEY);
return info;
}
case AuthenticationType::HTTP:
{
static const auto info = make_info("HTTP");
static const auto info = make_info(Keyword::HTTP);
return info;
}
case AuthenticationType::MAX:

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/types.h>
#include <Parsers/CommonParsers.h>
namespace DB
{
@ -45,7 +46,7 @@ enum class AuthenticationType
struct AuthenticationTypeInfo
{
const char * const raw_name;
Keyword keyword; // Keyword used in parser
const String name; /// Lowercased with underscores, e.g. "sha256_password".
bool is_password;
static const AuthenticationTypeInfo & get(AuthenticationType type_);
@ -53,7 +54,7 @@ struct AuthenticationTypeInfo
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
return String(toStringView(AuthenticationTypeInfo::get(type_).keyword));
}
}

View File

@ -7,7 +7,7 @@ namespace DB
{
ContextAccessParams::ContextAccessParams(
const std::optional<UUID> user_id_,
std::optional<UUID> user_id_,
bool full_access_,
bool use_default_roles_,
const std::shared_ptr<const std::vector<UUID>> & current_roles_,

View File

@ -15,7 +15,7 @@ class ContextAccessParams
{
public:
ContextAccessParams(
const std::optional<UUID> user_id_,
std::optional<UUID> user_id_,
bool full_access_,
bool use_default_roles_,
const std::shared_ptr<const std::vector<UUID>> & current_roles_,

View File

@ -34,7 +34,7 @@ public:
};
explicit GSSAcceptorContext(const Params & params_);
virtual ~GSSAcceptorContext() override;
~GSSAcceptorContext() override;
GSSAcceptorContext(const GSSAcceptorContext &) = delete;
GSSAcceptorContext(GSSAcceptorContext &&) = delete;

View File

@ -204,7 +204,7 @@ void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchR
}
void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchResultsList & external_roles, const std::size_t external_roles_hash) const
void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchResultsList & external_roles, std::size_t external_roles_hash) const
{
const auto & user_name = user.getName();
auto & granted_roles = user.granted_roles;

View File

@ -33,29 +33,29 @@ public:
static constexpr char STORAGE_TYPE[] = "ldap";
explicit LDAPAccessStorage(const String & storage_name_, AccessControl & access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
virtual ~LDAPAccessStorage() override = default;
~LDAPAccessStorage() override = default;
String getLDAPServerName() const;
// IAccessStorage implementations.
virtual const char * getStorageType() const override;
virtual String getStorageParamsJSON() const override;
virtual bool isReadOnly() const override { return true; }
virtual bool exists(const UUID & id) const override;
const char * getStorageType() const override;
String getStorageParamsJSON() const override;
bool isReadOnly() const override { return true; }
bool exists(const UUID & id) const override;
private: // IAccessStorage implementations.
virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
virtual std::vector<UUID> findAllImpl(AccessEntityType type) const override;
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
virtual std::optional<AuthResult> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
std::optional<AuthResult> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
void setConfiguration(const Poco::Util::AbstractConfiguration & config, const String & prefix);
void processRoleChange(const UUID & id, const AccessEntityPtr & entity);
void applyRoleChangeNoLock(bool grant, const UUID & role_id, const String & role_name);
void assignRolesNoLock(User & user, const LDAPClient::SearchResultsList & external_roles) const;
void assignRolesNoLock(User & user, const LDAPClient::SearchResultsList & external_roles, const std::size_t external_roles_hash) const;
void assignRolesNoLock(User & user, const LDAPClient::SearchResultsList & external_roles, std::size_t external_roles_hash) const;
void updateAssignedRolesNoLock(const UUID & id, const String & user_name, const LDAPClient::SearchResultsList & external_roles) const;
std::set<String> mapExternalRolesNoLock(const LDAPClient::SearchResultsList & external_roles) const;
bool areLDAPCredentialsValidNoLock(const User & user, const Credentials & credentials,

View File

@ -252,7 +252,6 @@ void dumpFlameGraph(
fillColumn(chars, offsets, out.str());
}
// NOLINTBEGIN(clang-analyzer-optin.performance.Padding)
struct AggregateFunctionFlameGraphData
{
struct Entry
@ -469,7 +468,6 @@ struct AggregateFunctionFlameGraphData
DB::dumpFlameGraph(tree.dump(max_depth, min_bytes), chars, offsets);
}
};
// NOLINTEND(clang-analyzer-optin.performance.Padding)
/// Aggregate function which builds a flamegraph using the list of stacktraces.
/// The output is an array of strings which can be used by flamegraph.pl util.

View File

@ -157,7 +157,7 @@ public:
void update(UInt64 batch_size, std::vector<Float64> & weights, Float64 & bias, Float64 learning_rate, const std::vector<Float64> & batch_gradient) override;
virtual void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void write(WriteBuffer & buf) const override;
@ -189,7 +189,7 @@ public:
void update(UInt64 batch_size, std::vector<Float64> & weights, Float64 & bias, Float64 learning_rate, const std::vector<Float64> & batch_gradient) override;
virtual void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void write(WriteBuffer & buf) const override;
@ -226,7 +226,7 @@ public:
void update(UInt64 batch_size, std::vector<Float64> & weights, Float64 & bias, Float64 learning_rate, const std::vector<Float64> & batch_gradient) override;
virtual void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void merge(const IWeightsUpdater & rhs, Float64 frac, Float64 rhs_frac) override;
void write(WriteBuffer & buf) const override;

View File

@ -234,9 +234,6 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
template <template <typename> typename FunctionTemplate, StatisticsFunctionKind kind>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
@ -273,5 +270,3 @@ AggregateFunctionPtr createAggregateFunctionStatisticsBinary(
}
}
}

View File

@ -42,9 +42,6 @@ struct UniqCombinedHashTableGrower : public HashTableGrowerWithPrecalculation<>
void increaseSize() { increaseSizeDegree(1); }
};
namespace
{
template <typename T, UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData
{
@ -268,8 +265,6 @@ AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_typ
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, false>>(argument_types, params);
}
}
template <UInt8 K>
AggregateFunctionPtr createAggregateFunctionWithHashType(bool use_64_bit_hash, const DataTypes & argument_types, const Array & params)
{

View File

@ -521,7 +521,7 @@ public:
}
}
chassert(nullable_filters.size() > 0);
chassert(!nullable_filters.empty());
bool found_one = false;
if (nullable_filters.size() == 1)
{

View File

@ -38,7 +38,7 @@ public:
offset = other->offset;
}
int length()
int length() const
{
return static_cast<int>(bins.size());
}

View File

@ -150,7 +150,7 @@ struct QuantileExactExclusive : public QuantileExact<Value>
return static_cast<Float64>(*std::min_element(array.begin(), array.end()));
::nth_element(array.begin(), array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
auto * nth_elem = std::min_element(array.begin() + n, array.end());
return static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
}
@ -179,7 +179,7 @@ struct QuantileExactExclusive : public QuantileExact<Value>
else
{
::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
auto * nth_elem = std::min_element(array.begin() + n, array.end());
result[indices[i]] = static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
prev_n = n - 1;
@ -214,7 +214,7 @@ struct QuantileExactInclusive : public QuantileExact<Value>
else if (n < 1)
return static_cast<Float64>(*std::min_element(array.begin(), array.end()));
::nth_element(array.begin(), array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
auto * nth_elem = std::min_element(array.begin() + n, array.end());
return static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
}
@ -241,7 +241,7 @@ struct QuantileExactInclusive : public QuantileExact<Value>
else
{
::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
auto * nth_elem = std::min_element(array.begin() + n, array.end());
result[indices[i]] = static_cast<Float64>(array[n - 1]) + (h - n) * (static_cast<Float64>(*nth_elem) - array[n - 1]);
prev_n = n - 1;

View File

@ -191,7 +191,7 @@ public:
/// TODO: After implementation of "versioning aggregate function state",
/// change the serialization format.
Element elem;
memset(&elem, 0, sizeof(elem));
memset(&elem, 0, sizeof(elem)); /// NOLINT(bugprone-undefined-memory-manipulation)
elem = samples[i];
DB::transformEndianness<std::endian::little>(elem);

View File

@ -23,7 +23,7 @@ struct SingleValueDataBase
/// For example argMin holds 1 of these (for the result), while keeping a template for the value
static constexpr UInt32 MAX_STORAGE_SIZE = 64;
virtual ~SingleValueDataBase() { }
virtual ~SingleValueDataBase() = default;
virtual bool has() const = 0;
virtual void insertResultInto(IColumn &) const = 0;
virtual void write(WriteBuffer &, const ISerialization &) const = 0;

View File

@ -39,8 +39,8 @@ public:
/// This method will convert all the SingleLevelSet to TwoLevelSet in parallel if the hashsets are not all singlelevel or not all twolevel.
static void parallelizeMergePrepare(const std::vector<UniqExactSet *> & data_vec, ThreadPool & thread_pool)
{
unsigned long single_level_set_num = 0;
unsigned long all_single_hash_size = 0;
UInt64 single_level_set_num = 0;
UInt64 all_single_hash_size = 0;
for (auto ele : data_vec)
{

View File

@ -77,7 +77,7 @@ public:
* Available expression columns are extracted from table expression.
* Table expression node must have query, union, table, table function type.
*/
QueryAnalysisPass(QueryTreeNodePtr table_expression_, bool only_analyze_ = false);
explicit QueryAnalysisPass(QueryTreeNodePtr table_expression_, bool only_analyze_ = false);
String getName() override
{

View File

@ -26,7 +26,7 @@ struct BackupSettings
String password;
/// S3 storage class.
String s3_storage_class = "";
String s3_storage_class;
/// If this is set to true then only create queries will be written to backup,
/// without the data of tables.

View File

@ -327,7 +327,7 @@ public:
metric_active_threads = CurrentMetrics::RestoreThreadsActive;
metric_active_threads = CurrentMetrics::RestoreThreadsScheduled;
max_threads = num_restore_threads;
use_queue = (thread_pool_id != ThreadPoolId::RESTORE);
use_queue = true;
break;
}
}

View File

@ -80,7 +80,7 @@ public:
/// Returns names of entries stored in a specified directory in the backup.
/// If `directory` is empty or '/' the functions returns entries in the backup's root.
virtual Strings listFiles(const String & directory, bool recursive = false) const = 0;
virtual Strings listFiles(const String & directory, bool recursive) const = 0;
/// Checks if a specified directory contains any files.
/// The function returns the same as `!listFiles(directory).empty()`.
@ -108,11 +108,9 @@ public:
virtual std::unique_ptr<SeekableReadBuffer> readFile(const SizeAndChecksum & size_and_checksum) const = 0;
/// Copies a file from the backup to a specified destination disk. Returns the number of bytes written.
virtual size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path,
WriteMode write_mode = WriteMode::Rewrite) const = 0;
virtual size_t copyFileToDisk(const String & file_name, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const = 0;
virtual size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path,
WriteMode write_mode = WriteMode::Rewrite) const = 0;
virtual size_t copyFileToDisk(const SizeAndChecksum & size_and_checksum, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) const = 0;
/// Puts a new entry to the backup.
virtual void writeFile(const BackupFileInfo & file_info, BackupEntryPtr entry) = 0;

View File

@ -273,7 +273,7 @@ void RestorerFromBackup::findRootPathsInBackup()
root_paths_in_backup.push_back(root_path);
/// Add shard-related part to the root path.
Strings shards_in_backup = backup->listFiles(root_path / "shards");
Strings shards_in_backup = backup->listFiles(root_path / "shards", /*recursive*/ false);
if (shards_in_backup.empty())
{
if (restore_settings.shard_num_in_backup > 1)
@ -295,7 +295,7 @@ void RestorerFromBackup::findRootPathsInBackup()
}
/// Add replica-related part to the root path.
Strings replicas_in_backup = backup->listFiles(root_path / "replicas");
Strings replicas_in_backup = backup->listFiles(root_path / "replicas", /*recursive*/ false);
if (replicas_in_backup.empty())
{
if (restore_settings.replica_num_in_backup > 1)
@ -514,7 +514,7 @@ void RestorerFromBackup::findDatabaseInBackupImpl(const String & database_name_i
if (!metadata_path && !try_metadata_path.empty() && backup->fileExists(try_metadata_path))
metadata_path = try_metadata_path;
Strings file_names = backup->listFiles(try_tables_metadata_path);
Strings file_names = backup->listFiles(try_tables_metadata_path, /*recursive*/ false);
for (const String & file_name : file_names)
{
if (!file_name.ends_with(".sql"))
@ -575,7 +575,7 @@ void RestorerFromBackup::findEverythingInBackup(const std::set<String> & except_
for (const auto & root_path_in_backup : root_paths_in_backup)
{
Strings file_names = backup->listFiles(root_path_in_backup / "metadata");
Strings file_names = backup->listFiles(root_path_in_backup / "metadata", /*recursive*/ false);
for (String & file_name : file_names)
{
if (file_name.ends_with(".sql"))

View File

@ -159,7 +159,7 @@ private:
size_t requested_connections_count = 0;
const size_t max_parallel_replicas = 0;
const bool skip_unavailable_shards = 0;
const bool skip_unavailable_shards = false;
};
}

View File

@ -77,12 +77,12 @@ private:
{
if (actual_server_error && std::find(server_errors.begin(), server_errors.end(), actual_server_error) == server_errors.end())
return false;
if (!actual_server_error && server_errors.size())
if (!actual_server_error && !server_errors.empty())
return false;
if (actual_client_error && std::find(client_errors.begin(), client_errors.end(), actual_client_error) == client_errors.end())
return false;
if (!actual_client_error && client_errors.size())
if (!actual_client_error && !client_errors.empty())
return false;
return true;
@ -90,7 +90,7 @@ private:
bool lostExpectedError(int actual_server_error, int actual_client_error) const
{
return (server_errors.size() && !actual_server_error) || (client_errors.size() && !actual_client_error);
return (!server_errors.empty() && !actual_server_error) || (!client_errors.empty() && !actual_client_error);
}
};

View File

@ -671,7 +671,7 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
size_t max_dictionary_size)
{
auto overflowed_keys = column_holder->cloneEmpty();
auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
auto * overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
if (!overflowed_keys_ptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid keys type for ColumnUnique.");

View File

@ -86,7 +86,7 @@ struct ReverseIndexHashTableCell
{
/// Careful: apparently this uses SFINAE to redefine isZero for all types
/// except the IndexType, for which the default ZeroTraits::isZero is used.
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
static_assert(!std::is_same_v<typename std::decay_t<T>, typename std::decay_t<IndexType>>);
return false;
}

View File

@ -26,10 +26,10 @@ struct AllocatorWithMemoryTracking
[[nodiscard]] T * allocate(size_t n)
{
if (n > std::numeric_limits<size_t>::max() / sizeof(T))
if (n > std::numeric_limits<size_t>::max() / sizeof(T)) /// NOLINT(bugprone-sizeof-expression)
throw std::bad_alloc();
size_t bytes = n * sizeof(T);
size_t bytes = n * sizeof(T); /// NOLINT(bugprone-sizeof-expression)
auto trace = CurrentMemoryTracker::alloc(bytes);
T * p = static_cast<T *>(malloc(bytes));
@ -43,7 +43,7 @@ struct AllocatorWithMemoryTracking
void deallocate(T * p, size_t n) noexcept
{
size_t bytes = n * sizeof(T);
size_t bytes = n * sizeof(T); /// NOLINT(bugprone-sizeof-expression)
free(p);
auto trace = CurrentMemoryTracker::free(bytes);

View File

@ -49,7 +49,7 @@ private:
MemoryChunk() = default;
void swap(MemoryChunk & other)
void swap(MemoryChunk & other) noexcept
{
std::swap(begin, other.begin);
std::swap(pos, other.pos);
@ -57,12 +57,12 @@ private:
prev.swap(other.prev);
}
MemoryChunk(MemoryChunk && other)
MemoryChunk(MemoryChunk && other) noexcept
{
*this = std::move(other);
}
MemoryChunk & operator=(MemoryChunk && other)
MemoryChunk & operator=(MemoryChunk && other) noexcept
{
swap(other);
return *this;

View File

@ -47,7 +47,6 @@ public:
HashMap<UInt16, Float64> map;
};
public:
using Map = HashMap<StringRef, Float64>;
using Container = std::vector<Language>;

View File

@ -44,11 +44,12 @@ public:
virtual const Metrics & getMetrics() const = 0;
virtual ~IHTTPConnectionPoolForEndpoint() = default;
IHTTPConnectionPoolForEndpoint(const IHTTPConnectionPoolForEndpoint &) = delete;
IHTTPConnectionPoolForEndpoint & operator=(const IHTTPConnectionPoolForEndpoint &) = delete;
protected:
IHTTPConnectionPoolForEndpoint() = default;
IHTTPConnectionPoolForEndpoint(const IHTTPConnectionPoolForEndpoint &) = delete;
IHTTPConnectionPoolForEndpoint & operator=(const IHTTPConnectionPoolForEndpoint &) = delete;
};
enum class HTTPConnectionGroupType
@ -70,11 +71,12 @@ public:
static constexpr size_t warning_step = 100;
};
private:
HTTPConnectionPools();
HTTPConnectionPools(const HTTPConnectionPools &) = delete;
HTTPConnectionPools & operator=(const HTTPConnectionPools &) = delete;
private:
HTTPConnectionPools();
public:
static HTTPConnectionPools & instance();

View File

@ -92,7 +92,8 @@ inline bool bitEquals(T && a, T && b)
using RealT = std::decay_t<T>;
if constexpr (std::is_floating_point_v<RealT>)
return 0 == memcmp(&a, &b, sizeof(RealT)); /// Note that memcmp with constant size is compiler builtin.
/// Note that memcmp with constant size is compiler builtin.
return 0 == memcmp(&a, &b, sizeof(RealT)); /// NOLINT
else
return a == b;
}
@ -644,7 +645,7 @@ protected:
/// Copy to a new location and zero the old one.
x.setHash(hash_value);
memcpy(static_cast<void*>(&buf[place_value]), &x, sizeof(x));
memcpy(static_cast<void*>(&buf[place_value]), &x, sizeof(x)); /// NOLINT(bugprone-undefined-memory-manipulation)
x.setZero();
/// Then the elements that previously were in collision with this can move to the old place.

View File

@ -12,7 +12,7 @@ struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash,
using Base::Base;
static constexpr bool need_zero_value_storage = false;
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const Key & getKey(const value_type & value_) { return value_.first; }
};
@ -32,7 +32,7 @@ struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16,
void setZero() { this->value.first.items[1] = 0; }
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const StringKey16 & getKey(const value_type & value_) { return value_.first; }
};
@ -53,7 +53,7 @@ struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24,
void setZero() { this->value.first.c = 0; }
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const StringKey24 & getKey(const value_type & value_) { return value_.first; }
};

View File

@ -19,7 +19,7 @@ struct StringKey24
bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
};
inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey8 & n)
{
assert(n != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
@ -28,7 +28,7 @@ inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
return {reinterpret_cast<const char *>(&n), 8ul - (std::countl_zero(n) >> 3)};
#endif
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey16 & n)
{
assert(n.items[1] != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
@ -37,7 +37,7 @@ inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
return {reinterpret_cast<const char *>(&n), 16ul - (std::countl_zero(n.items[1]) >> 3)};
#endif
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey24 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey24 & n)
{
assert(n.c != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__

View File

@ -67,8 +67,8 @@ public:
class Entry
{
public:
explicit Entry(Entry && entry) = default;
explicit Entry(Entry & entry) = delete;
Entry(Entry && entry) = default;
Entry(Entry & entry) = delete;
// no access as r-value
const String * operator->() && = delete;
@ -89,7 +89,7 @@ public:
Entry(HostResolver & pool_, Poco::Net::IPAddress address_)
: pool(pool_.getWeakFromThis())
, address(std::move(address_))
, address(address_)
, resolved_host(address.toString())
{ }
@ -126,14 +126,14 @@ protected:
struct Record
{
Record(Poco::Net::IPAddress address_, Poco::Timestamp resolve_time_)
: address(std::move(address_))
: address(address_)
, resolve_time(resolve_time_)
{}
explicit Record(Record && rec) = default;
Record(Record && rec) = default;
Record& operator=(Record && s) = default;
explicit Record(const Record & rec) = default;
Record(const Record & rec) = default;
Record& operator=(const Record & s) = default;
Poco::Net::IPAddress address;
@ -198,10 +198,11 @@ class HostResolversPool
{
private:
HostResolversPool() = default;
public:
HostResolversPool(const HostResolversPool &) = delete;
HostResolversPool & operator=(const HostResolversPool &) = delete;
public:
static HostResolversPool & instance();
void dropCache();

View File

@ -240,7 +240,7 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
return "toDayOfMonth";
case IntervalKind::Kind::Week:
// TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15')))
// else if (ParserKeyword("WEEK").ignore(pos, expected))
// else if (ParserKeyword(Keyword::WEEK).ignore(pos, expected))
// function_name = "toRelativeWeekNum";
throw Exception(ErrorCodes::SYNTAX_ERROR, "The syntax 'EXTRACT(WEEK FROM date)' is not supported, cannot extract the number of a week");
case IntervalKind::Kind::Month:

View File

@ -27,7 +27,7 @@
*/
namespace
namespace impl
{
/// After the most significant bit 1, set all subsequent less significant bits to 1 as well.
inline UInt64 toMask(UInt64 n)
@ -85,7 +85,7 @@ void intervalBinaryPartition(UInt64 first, UInt64 last, F && callback)
/// split = 15: 00001111
UInt64 diff = first ^ last;
UInt64 mask = toMask(diff) >> 1;
UInt64 mask = impl::toMask(diff) >> 1;
/// The current interval represents a whole range with fixed prefix.
if ((first & mask) == 0 && (last & mask) == mask)

View File

@ -29,7 +29,7 @@ public:
}
NetException * clone() const override { return new NetException(*this); }
void rethrow() const override { throw *this; }
void rethrow() const override { throw *this; } /// NOLINT(cert-err60-cpp)
private:
const char * name() const noexcept override { return "DB::NetException"; }

View File

@ -301,6 +301,8 @@ public:
}
};
/// NOLINTBEGIN(bugprone-sizeof-expression)
template <typename T, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
class PODArray : public PODArrayBase<sizeof(T), initial_bytes, TAllocator, pad_right_, pad_left_>
{
@ -755,6 +757,8 @@ public:
}
};
/// NOLINTEND(bugprone-sizeof-expression)
template <typename T, size_t initial_bytes, typename TAllocator, size_t pad_right_, size_t pad_left_>
void swap(PODArray<T, initial_bytes, TAllocator, pad_right_, pad_left_> & lhs, PODArray<T, initial_bytes, TAllocator, pad_right_, pad_left_> & rhs) /// NOLINT
{

View File

@ -50,7 +50,7 @@ struct ProxyConfiguration
bool tunneling = false;
Protocol original_request_protocol = Protocol::HTTP;
bool isEmpty() const { return host.size() == 0; }
bool isEmpty() const { return host.empty(); }
};
}

View File

@ -85,7 +85,6 @@ private:
StatePtr state; // hold state to avoid ResourceLink invalidation due to resource deregistration from SchedulerRoot
};
private:
SchedulerRoot scheduler;
std::mutex mutex;
StatePtr state;

View File

@ -248,7 +248,6 @@ private:
parent->activateChild(this);
}
private:
/// Beginning of `items` vector is heap of active children: [0; `heap_size`).
/// Next go inactive children in unsorted order.
/// NOTE: we have to track vruntime of inactive children for max-min fairness.

View File

@ -231,7 +231,6 @@ private:
value->next = nullptr;
}
private:
void schedulerThread()
{
while (!stop_flag.load())
@ -253,7 +252,6 @@ private:
request->execute();
}
private:
TResource * current = nullptr; // round-robin pointer
std::unordered_map<ISchedulerNode *, TResource> children; // resources by pointer
std::atomic<bool> stop_flag = false;

View File

@ -291,7 +291,7 @@ private:
class ThreadReference
{
public:
const ThreadReference & operator = (ThreadFromGlobalPool && thread_)
ThreadReference & operator = (ThreadFromGlobalPool && thread_)
{
std::lock_guard<std::mutex> l(lock);
thread = std::move(thread_);

View File

@ -120,7 +120,7 @@ public:
iteration_succeeded = false;
user_error.code = code;
user_error.message = std::move(message);
user_error.message = message;
user_error.exception = exception;
keeper_error = KeeperError{};
}

View File

@ -215,7 +215,7 @@ inline ALWAYS_INLINE size_t untrackMemory(void * ptr [[maybe_unused]], Allocatio
#endif
trace = CurrentMemoryTracker::free(actual_size);
}
catch (...)
catch (...) /// NOLINT(bugprone-empty-catch)
{
}

View File

@ -27,7 +27,7 @@ public:
if (!finished)
rollback();
}
catch (...)
catch (...) /// NOLINT(bugprone-empty-catch)
{
}
}

View File

@ -36,7 +36,7 @@ using Int64 = int64_t;
using UInt32 = uint32_t;
using Int32 = int32_t;
using MYSQL_LENGTH = unsigned long;
using MYSQL_LENGTH = unsigned long; /// NOLINT
using MYSQL_LENGTHS = MYSQL_LENGTH *;
using MYSQL_FIELDS = MYSQL_FIELD *;

View File

@ -57,7 +57,7 @@ template <typename Result, typename T>
std::future<Result> scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, Priority priority = {})
{
auto schedule = threadPoolCallbackRunner<Result, T>(pool, thread_name);
return schedule(std::move(task), priority);
return schedule(std::move(task), priority); /// NOLINT
}
}

View File

@ -26,7 +26,7 @@ public:
qpl_job * acquireJob(UInt32 & job_id);
void releaseJob(UInt32 job_id);
const bool & isJobPoolReady() { return job_pool_ready; }
const bool & isJobPoolReady() const { return job_pool_ready; }
private:
bool tryLockJob(UInt32 index);

View File

@ -16,7 +16,7 @@ public:
~WriteBufferFromNuraftBuffer() override;
private:
void finalizeImpl() override final;
void finalizeImpl() final;
void nextImpl() override;

View File

@ -40,6 +40,7 @@ using FieldVector = std::vector<Field, AllocatorWithMemoryTracking<Field>>;
/// construct a Field of Array or a Tuple type. An alternative approach would be
/// to construct both of these types from FieldVector, and have the caller
/// specify the desired Field type explicitly.
/// NOLINTBEGIN(modernize-type-traits)
#define DEFINE_FIELD_VECTOR(X) \
struct X : public FieldVector \
{ \
@ -48,6 +49,7 @@ struct X : public FieldVector \
DEFINE_FIELD_VECTOR(Array);
DEFINE_FIELD_VECTOR(Tuple);
/// NOLINTEND(modernize-type-traits)
/// An array with the following structure: [(key1, value1), (key2, value2), ...]
DEFINE_FIELD_VECTOR(Map); /// TODO: use map instead of vector.

View File

@ -546,7 +546,7 @@ namespace MySQLReplication
virtual void setGTIDSets(GTIDSets sets) = 0;
virtual void setChecksumSignatureLength(size_t checksum_signature_length_) = 0;
virtual ~IFlavor() override = default;
~IFlavor() override = default;
};
class MySQLFlavor : public IFlavor

View File

@ -841,6 +841,9 @@ class IColumn;
M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \
M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \
M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \
M(UInt64, keeper_max_retries, 10, "Max retries for general keeper operations", 0) \
M(UInt64, keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for general keeper operations", 0) \
M(UInt64, keeper_retry_max_backoff_ms, 5000, "Max backoff timeout for general keeper operations", 0) \
M(UInt64, insert_keeper_max_retries, 20, "Max retries for keeper operations during insert", 0) \
M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \
M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \
@ -1195,7 +1198,6 @@ class IColumn;
FORMAT_FACTORY_SETTINGS(M, ALIAS) \
OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \
/// NOLINTNEXTLINE(clang-analyzer-optin.performance.Padding)
DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS)
@ -1240,7 +1242,6 @@ private:
/*
* User-specified file format settings for File and URL engines.
*/
/// NOLINTNEXTLINE(clang-analyzer-optin.performance.Padding)
DECLARE_SETTINGS_TRAITS(FormatFactorySettingsTraits, LIST_OF_ALL_FORMAT_SETTINGS)
struct FormatFactorySettings : public BaseSettings<FormatFactorySettingsTraits>

View File

@ -97,6 +97,9 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."},
{"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"},
{"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"},
{"keeper_max_retries", 10, 10, "Max retries for general keeper operations"},
{"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"},
{"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"},
}},
{"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"},
{"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"},

View File

@ -5,14 +5,12 @@
namespace DB
{
// NOLINTBEGIN(clang-analyzer-optin.performance.Padding)
struct CheckResult
{
Int32 id;
String name;
bool need_convert;
};
// NOLINTEND(clang-analyzer-optin.performance.Padding)
TEST(CharsetTest, CharsetTest)
{

View File

@ -103,7 +103,7 @@ public:
GraphiteWriter * getGraphiteWriter(const std::string & config_name = DEFAULT_GRAPHITE_CONFIG_NAME)
{
if (graphite_writers.count(config_name))
if (graphite_writers.contains(config_name))
return graphite_writers[config_name].get();
return nullptr;
}
@ -183,7 +183,7 @@ std::optional<std::reference_wrapper<Daemon>> BaseDaemon::tryGetInstance()
{
ptr = dynamic_cast<Daemon *>(&Poco::Util::Application::instance());
}
catch (const Poco::NullPointerException &)
catch (const Poco::NullPointerException &) /// NOLINT(bugprone-empty-catch)
{
/// if daemon doesn't exist than instance() throw NullPointerException
}

View File

@ -217,11 +217,13 @@ template <typename A> struct ToInteger
// CLICKHOUSE-29. The same depth, different signs
// NOTE: This case is applied for 64-bit integers only (for backward compatibility), but could be used for any-bit integers
/// NOLINTBEGIN(misc-redundant-expression)
template <typename A, typename B>
constexpr bool LeastGreatestSpecialCase =
std::is_integral_v<A> && std::is_integral_v<B>
&& (8 == sizeof(A) && sizeof(A) == sizeof(B))
&& (is_signed_v<A> ^ is_signed_v<B>);
/// NOLINTEND(misc-redundant-expression)
template <typename A, typename B>
using ResultOfLeast = std::conditional_t<LeastGreatestSpecialCase<A, B>,

View File

@ -18,7 +18,7 @@ public:
explicit SerializationCustomSimpleText(const SerializationPtr & nested_);
// Methods that subclasses must override in order to get full serialization/deserialization support.
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0;
/// whole = true means that buffer contains only one value, so we should read until EOF.
/// It's needed to check if there is garbage after parsed field.
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const = 0;

View File

@ -27,7 +27,7 @@ public:
Kind getKind() const override { return Kind::SPARSE; }
virtual void enumerateStreams(
void enumerateStreams(
EnumerateStreamsSettings & settings,
const StreamCallback & callback,
const SubstreamData & data) const override;

View File

@ -205,8 +205,8 @@ void SerializationVariantElement::removeVariantFromPath(DB::ISerialization::Subs
SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
const ColumnVariant::Discriminator global_variant_discriminator_,
const ColumnVariant::Discriminator local_variant_discriminator_)
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_)
: local_discriminators(local_discriminators_)
, variant_element_name(variant_element_name_)
, global_variant_discriminator(global_variant_discriminator_)

View File

@ -75,8 +75,8 @@ private:
VariantSubcolumnCreator(
const ColumnPtr & local_discriminators_,
const String & variant_element_name_,
const ColumnVariant::Discriminator global_variant_discriminator_,
const ColumnVariant::Discriminator local_variant_discriminator_);
ColumnVariant::Discriminator global_variant_discriminator_,
ColumnVariant::Discriminator local_variant_discriminator_);
DataTypePtr create(const DataTypePtr & prev) const override;
ColumnPtr create(const ColumnPtr & prev) const override;

View File

@ -23,7 +23,6 @@ public:
String getEngineName() const override { return "Overlay"; }
public:
bool isTableExist(const String & table_name, ContextPtr context) const override;
StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override;

View File

@ -12,23 +12,23 @@ StorageID tryParseTableIDFromDDL(const String & query, const String & default_da
Tokens tokens(query.data(), query.data() + query.size());
IParser::Pos pos(tokens, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
Expected expected;
if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected))
if (ParserKeyword(Keyword::CREATE_TEMPORARY_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::CREATE_TABLE).ignore(pos, expected))
{
ParserKeyword("IF NOT EXISTS").ignore(pos, expected);
ParserKeyword(Keyword::IF_NOT_EXISTS).ignore(pos, expected);
is_ddl = true;
}
else if (ParserKeyword("ALTER TABLE").ignore(pos, expected) || ParserKeyword("RENAME TABLE").ignore(pos, expected))
else if (ParserKeyword(Keyword::ALTER_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::RENAME_TABLE).ignore(pos, expected))
{
is_ddl = true;
}
else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected))
else if (ParserKeyword(Keyword::DROP_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::DROP_TEMPORARY_TABLE).ignore(pos, expected))
{
ParserKeyword("IF EXISTS").ignore(pos, expected);
ParserKeyword(Keyword::IF_EXISTS).ignore(pos, expected);
is_ddl = true;
}
else if (ParserKeyword("TRUNCATE").ignore(pos, expected))
else if (ParserKeyword(Keyword::TRUNCATE).ignore(pos, expected))
{
ParserKeyword("TABLE").ignore(pos, expected);
ParserKeyword(Keyword::TABLE).ignore(pos, expected);
is_ddl = true;
}

View File

@ -308,7 +308,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
/// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
result_of_fetch_from_storage = cache_storage_ptr->fetchColumnsForKeys(keys, request);
result_of_fetch_from_storage = cache_storage_ptr->fetchColumnsForKeys(keys, request, /*default_mask*/ nullptr);
}
size_t found_keys_size = result_of_fetch_from_storage.found_keys_size;

View File

@ -171,7 +171,7 @@ private:
const PaddedPODArray<KeyState> & key_index_to_fetched_columns_from_storage_result,
const MutableColumns & fetched_columns_during_update,
const HashMap<KeyType, size_t> & found_keys_to_fetched_columns_during_update_index,
IColumn::Filter * const default_mask = nullptr) const;
IColumn::Filter * default_mask = nullptr) const;
void update(CacheDictionaryUpdateUnitPtr<dictionary_key_type> update_unit_ptr);

View File

@ -73,7 +73,7 @@ public:
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request,
IColumn::Filter * const default_mask = nullptr) override
IColumn::Filter * const default_mask) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request, default_mask);
@ -110,7 +110,7 @@ public:
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests,
IColumn::Filter * const default_mask = nullptr) override
IColumn::Filter * const default_mask) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests, default_mask);

View File

@ -82,7 +82,7 @@ public:
virtual SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request,
IColumn::Filter * default_mask = nullptr) = 0;
IColumn::Filter * default_mask) = 0;
/// Fetch columns for keys, this method is not write thread safe
virtual void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) = 0;
@ -100,7 +100,7 @@ public:
virtual ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests,
IColumn::Filter * default_mask = nullptr) = 0;
IColumn::Filter * default_mask) = 0;
/// Fetch columns for keys, this method is not write thread safe
virtual void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) = 0;

View File

@ -288,7 +288,7 @@ private:
extern template class RangeHashedDictionary<DictionaryKeyType::Simple>;
extern template class RangeHashedDictionary<DictionaryKeyType::Complex>;
namespace
namespace impl
{
template <typename F>
void callOnRangeType(const DataTypePtr & range_type, F && func)
@ -465,7 +465,7 @@ ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Colum
auto & out = result->getData();
size_t keys_found = 0;
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;
@ -523,7 +523,7 @@ void RangeHashedDictionary<dictionary_key_type>::createAttributes()
getDictionaryID().getNameForLogs());
}
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;
@ -553,7 +553,7 @@ void RangeHashedDictionary<dictionary_key_type>::loadData()
updateData();
}
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;
@ -573,7 +573,7 @@ void RangeHashedDictionary<dictionary_key_type>::loadData()
template <DictionaryKeyType dictionary_key_type>
void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
{
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;
@ -783,7 +783,7 @@ void RangeHashedDictionary<dictionary_key_type>::blockToAttributes(const Block &
max_range_null_map = &max_range_column_nullable->getNullMapColumn().getData();
}
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;
@ -930,7 +930,7 @@ Pipe RangeHashedDictionary<dictionary_key_type>::read(const Names & column_names
PaddedPODArray<KeyType> keys;
callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
impl::callOnRangeType(dict_struct.range_min->type, [&](const auto & types)
{
using Types = std::decay_t<decltype(types)>;
using RangeColumnType = typename Types::LeftType;

View File

@ -37,7 +37,7 @@ void RangeHashedDictionary<dictionary_key_type>::getItemsImpl(
DictionaryKeysExtractor<dictionary_key_type> keys_extractor(key_columns_copy, arena_holder.getComplexKeyArena());
const size_t keys_size = keys_extractor.getKeysSize();
callOnRangeType(
impl::callOnRangeType(
dict_struct.range_min->type,
[&](const auto & types)
{

View File

@ -37,7 +37,7 @@ size_t RangeHashedDictionary<dictionary_key_type>::getItemsShortCircuitImpl(
const size_t keys_size = keys_extractor.getKeysSize();
default_mask.resize(keys_size);
callOnRangeType(
impl::callOnRangeType(
dict_struct.range_min->type,
[&](const auto & types)
{

View File

@ -870,7 +870,7 @@ public:
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request,
IColumn::Filter * const default_mask = nullptr) override
IColumn::Filter * const default_mask) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request, default_mask);
@ -907,7 +907,7 @@ public:
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & fetch_request,
IColumn::Filter * const default_mask = nullptr) override
IColumn::Filter * const default_mask) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request, default_mask);

View File

@ -62,8 +62,8 @@ public:
virtual void copyFile(
const std::string & from_file_path,
const std::string & to_file_path,
const ReadSettings & read_settings = {},
const WriteSettings & write_settings = {}) = 0;
const ReadSettings & read_settings,
const WriteSettings & write_settings) = 0;
/// Open the file for write and return WriteBufferFromFileBase object.
virtual std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT

View File

@ -78,7 +78,7 @@ private:
public:
explicit IOUringReader(uint32_t entries_);
inline bool isSupported() { return is_supported; }
bool isSupported() const { return is_supported; }
std::future<Result> submit(Request request) override;
Result execute(Request /* request */) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `execute` not implemented for IOUringReader"); }

View File

@ -64,7 +64,7 @@ public:
DiskSelectorPtr disk_selector
);
virtual ReservationPtr reserve(UInt64 bytes) override = 0;
ReservationPtr reserve(UInt64 bytes) override = 0;
/// This is a volume.
bool isVolume() const override { return true; }

View File

@ -204,7 +204,7 @@ void DiskObjectStorage::copyFile( /// NOLINT
/// It may use s3-server-side copy
auto & to_disk_object_storage = dynamic_cast<DiskObjectStorage &>(to_disk);
auto transaction = createObjectStorageTransactionToAnotherDisk(to_disk_object_storage);
transaction->copyFile(from_file_path, to_file_path);
transaction->copyFile(from_file_path, to_file_path, /*read_settings*/ {}, /*write_settings*/ {});
transaction->commit();
}
else

View File

@ -54,7 +54,7 @@ public:
return batch;
}
virtual std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override
std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override
{
return std::nullopt;
}

View File

@ -9,14 +9,6 @@
#include <IO/S3/Client.h>
namespace Aws
{
namespace S3
{
class Client;
}
}
namespace DB
{

View File

@ -47,7 +47,7 @@ struct CountSubstringsImpl
const UInt8 * pos = begin;
/// FIXME: suboptimal
memset(&res[0], 0, res.size() * sizeof(res[0]));
memset(&res[0], 0, res.size() * sizeof(res[0])); /// NOLINT(readability-container-data-pointer)
if (needle.empty())
return; // Return all zeros

View File

@ -58,7 +58,7 @@ struct CustomWeekTransformImpl
template <typename Transform>
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
{
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform, is_extended_result>{std::move(transform)};
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform, is_extended_result>{transform};
static constexpr UInt8 default_week_mode = 0;
UInt8 week_mode = default_week_mode;

View File

@ -2094,7 +2094,6 @@ struct Transformer
|| std::is_same_v<Additions, DateTimeAccurateOrNullConvertStrategyAdditions>)
{
bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL;
if (!is_valid_input)
{
if constexpr (std::is_same_v<Additions, DateTimeAccurateOrNullConvertStrategyAdditions>)

View File

@ -13,8 +13,9 @@ class FunctionConstantBase : public IFunction
{
public:
template <typename U>
explicit FunctionConstantBase(U && constant_value_, bool is_distributed_ = false)
: constant_value(static_cast<T>(std::forward<U>(constant_value_))), is_distributed(is_distributed_)
explicit FunctionConstantBase(const U & constant_value_, bool is_distributed_ = false)
: constant_value(static_cast<T>(constant_value_))
, is_distributed(is_distributed_)
{
}

View File

@ -96,7 +96,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher)
{
if constexpr (compatibility_mode == CompatibilityMode::MySQL)
{
switch (EVP_CIPHER_mode(evp_cipher))
switch (EVP_CIPHER_mode(evp_cipher)) /// NOLINT(bugprone-switch-missing-default-case)
{
case EVP_CIPH_ECB_MODE: [[fallthrough]];
case EVP_CIPH_CBC_MODE: [[fallthrough]];
@ -107,7 +107,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher)
}
else if constexpr (compatibility_mode == CompatibilityMode::OpenSSL)
{
switch (EVP_CIPHER_mode(evp_cipher))
switch (EVP_CIPHER_mode(evp_cipher)) /// NOLINT(bugprone-switch-missing-default-case)
{
case EVP_CIPH_ECB_MODE: [[fallthrough]];
case EVP_CIPH_CBC_MODE: [[fallthrough]];

View File

@ -519,12 +519,12 @@ public:
"but one of them has type {}.", getName(), arguments[i + 1]->getName());
if (!array_type)
throw exception;
throw exception; /// NOLINT
auto nested_type = array_type->getNestedType();
WhichDataType which(nested_type);
if (!(which.isUInt8() || which.isUInt16() || which.isUInt32() || which.isUInt64()))
throw exception;
throw exception; /// NOLINT
}
return arguments[0];
}

View File

@ -283,7 +283,7 @@ struct StringComparisonImpl
size_t size = a_data.size();
for (size_t i = 0, j = 0; i < size; i += 16, ++j)
c[j] = Op::apply(memcmp16(&a_data[i], &b_data[0]), 0);
c[j] = Op::apply(memcmp16(&a_data[i], &b_data[0]), 0); /// NOLINT(readability-container-data-pointer)
}
static void NO_INLINE fixed_string_vector_fixed_string_vector( /// NOLINT

View File

@ -393,8 +393,8 @@ public:
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function {} does not support nullable keys", getName());
// Wrap all the attribute types in Array()
for (auto it = attribute_types.begin(); it != attribute_types.end(); ++it)
*it = std::make_shared<DataTypeArray>(*it);
for (auto & attr_type : attribute_types)
attr_type = std::make_shared<DataTypeArray>(attr_type);
}
if (attribute_types.size() > 1)
{

View File

@ -190,14 +190,26 @@ public:
{
if (col_haystack_const && col_needle_const)
{
const auto is_col_start_pos_const = !column_start_pos || isColumnConst(*column_start_pos);
auto column_start_position_arg = column_start_pos;
bool is_col_start_pos_const = false;
if (column_start_pos)
{
if (const ColumnConst * const_column_start_pos = typeid_cast<const ColumnConst *>(&*column_start_pos))
{
is_col_start_pos_const = true;
column_start_position_arg = const_column_start_pos->getDataColumnPtr();
}
}
else
is_col_start_pos_const = true;
vec_res.resize(is_col_start_pos_const ? 1 : column_start_pos->size());
const auto null_map = create_null_map();
Impl::constantConstant(
col_haystack_const->getValue<String>(),
col_needle_const->getValue<String>(),
column_start_pos,
column_start_position_arg,
vec_res,
null_map.get());

View File

@ -73,6 +73,8 @@ struct ToStartOfTransform;
TRANSFORM_TIME(Second)
#undef TRANSFORM_TIME
/// NOLINTBEGIN(bugprone-macro-parentheses)
#define TRANSFORM_SUBSECONDS(INTERVAL_KIND, DEF_SCALE) \
template<> \
struct ToStartOfTransform<IntervalKind::Kind::INTERVAL_KIND> \
@ -156,6 +158,8 @@ template <> \
ADD_SUBSECONDS(Nanosecond, 9)
#undef ADD_SUBSECONDS
/// NOLINTEND(bugprone-macro-parentheses)
template <TimeWindowFunctionName type>
struct TimeWindowImpl
{

View File

@ -126,19 +126,19 @@ struct ArrayAndValueSourceSelectorBySink : public ArraySinkSelector<ArrayAndValu
template <typename Sink, typename ... Args>
static void selectImpl(Sink && sink, IArraySource & array_source, IValueSource & value_source, Args && ... args)
{
using SynkType = typename std::decay<Sink>::type;
using ArraySource = typename SynkType::CompatibleArraySource;
using ValueSource = typename SynkType::CompatibleValueSource;
using SinkType = typename std::decay_t<Sink>;
using ArraySource = typename SinkType::CompatibleArraySource;
using ValueSource = typename SinkType::CompatibleValueSource;
auto check_type = [] (auto source_ptr)
{
if (source_ptr == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "{} expected {} or {} or {} or {} but got {}",
demangle(typeid(Base).name()),
demangle(typeid(typename SynkType::CompatibleArraySource).name()),
demangle(typeid(ConstSource<typename SynkType::CompatibleArraySource>).name()),
demangle(typeid(typename SynkType::CompatibleValueSource).name()),
demangle(typeid(ConstSource<typename SynkType::CompatibleValueSource>).name()),
demangle(typeid(typename SinkType::CompatibleArraySource).name()),
demangle(typeid(ConstSource<typename SinkType::CompatibleArraySource>).name()),
demangle(typeid(typename SinkType::CompatibleValueSource).name()),
demangle(typeid(ConstSource<typename SinkType::CompatibleValueSource>).name()),
demangle(typeid(*source_ptr).name()));
};
auto check_type_and_call_concat = [& sink, & check_type, & args ...] (auto array_source_ptr, auto value_source_ptr)

View File

@ -144,6 +144,8 @@ struct NumericArraySource : public ArraySourceImpl<NumericArraySource<T>>
#pragma clang diagnostic ignored "-Wsuggest-override"
#pragma clang diagnostic ignored "-Wsuggest-destructor-override"
/// NOLINTBEGIN(hicpp-use-override)
template <typename Base>
struct ConstSource : public Base
{
@ -231,6 +233,8 @@ struct ConstSource : public Base
}
};
/// NOLINTEND(hicpp-use-override)
#pragma clang diagnostic pop
struct StringSource

View File

@ -13,7 +13,7 @@ class WriteBuffer;
class GregorianDate
{
public:
GregorianDate() {}
GregorianDate() = default;
void init(ReadBuffer & in);
bool tryInit(ReadBuffer & in);
@ -84,7 +84,7 @@ private:
class OrdinalDate
{
public:
OrdinalDate() {}
OrdinalDate() = default;
void init(int32_t year, uint16_t day_of_year);
bool tryInit(int32_t year, uint16_t day_of_year);

View File

@ -14,8 +14,6 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
namespace
{
using namespace GatherUtils;
@ -154,5 +152,3 @@ private:
};
}
}

Some files were not shown because too many files have changed in this diff Show More