Merge pull request #35248 from kitaisreal/enable-clang-tidy-for-header-files

Enable clang tidy for header files
This commit is contained in:
Maksim Kita 2022-03-14 19:20:58 +01:00 committed by GitHub
commit fe1bed0b40
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
531 changed files with 1857 additions and 1880 deletions

View File

@ -46,9 +46,9 @@ struct StringRef
constexpr StringRef(const char * data_, size_t size_) : data(data_), size(size_) {}
StringRef(const std::string & s) : data(s.data()), size(s.size()) {}
StringRef(const std::string & s) : data(s.data()), size(s.size()) {} /// NOLINT
constexpr explicit StringRef(std::string_view s) : data(s.data()), size(s.size()) {}
constexpr StringRef(const char * data_) : StringRef(std::string_view{data_}) {}
constexpr StringRef(const char * data_) : StringRef(std::string_view{data_}) {} /// NOLINT
constexpr StringRef() = default;
std::string toString() const { return std::string(data, size); }

View File

@ -15,7 +15,7 @@ class AccessRights
{
public:
AccessRights();
AccessRights(const AccessFlags & access);
explicit AccessRights(const AccessFlags & access);
~AccessRights();
AccessRights(const AccessRights & src);
AccessRights & operator =(const AccessRights & src);

View File

@ -15,17 +15,17 @@ using Strings = std::vector<String>;
class AccessFlags
{
public:
AccessFlags(AccessType type);
AccessFlags(AccessType type); /// NOLINT
/// The same as AccessFlags(AccessType::NONE).
AccessFlags() = default;
/// Constructs from a string like "SELECT".
AccessFlags(const std::string_view & keyword);
AccessFlags(const std::string_view & keyword); /// NOLINT
/// Constructs from a list of strings like "SELECT, UPDATE, INSERT".
AccessFlags(const std::vector<std::string_view> & keywords);
AccessFlags(const Strings & keywords);
AccessFlags(const std::vector<std::string_view> & keywords); /// NOLINT
AccessFlags(const Strings & keywords); /// NOLINT
AccessFlags(const AccessFlags & src) = default;
AccessFlags(AccessFlags && src) = default;
@ -109,7 +109,7 @@ private:
using Flags = std::bitset<SIZE>;
Flags flags;
AccessFlags(const Flags & flags_) : flags(flags_) {}
AccessFlags(const Flags & flags_) : flags(flags_) {} /// NOLINT
};
AccessFlags operator |(AccessType left, AccessType right);

View File

@ -26,7 +26,7 @@ struct AccessRightsElement
AccessRightsElement(AccessRightsElement &&) = default;
AccessRightsElement & operator=(AccessRightsElement &&) = default;
AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {}
explicit AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {}
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_);
AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_);

View File

@ -18,7 +18,7 @@ public:
class IPSubnet
{
public:
IPSubnet() {}
IPSubnet() = default;
IPSubnet(const IPAddress & prefix_, const IPAddress & mask_) { set(prefix_, mask_); }
IPSubnet(const IPAddress & prefix_, size_t num_prefix_bits) { set(prefix_, num_prefix_bits); }
explicit IPSubnet(const IPAddress & address) { set(address); }
@ -43,9 +43,9 @@ public:
struct AnyHostTag {};
AllowedClientHosts() {}
AllowedClientHosts(AnyHostTag) { addAnyHost(); }
~AllowedClientHosts() {}
AllowedClientHosts() = default;
AllowedClientHosts(AnyHostTag) { addAnyHost(); } /// NOLINT
~AllowedClientHosts() = default;
AllowedClientHosts(const AllowedClientHosts & src) = default;
AllowedClientHosts & operator =(const AllowedClientHosts & src) = default;

View File

@ -54,7 +54,7 @@ class AuthenticationData
public:
using Digest = std::vector<uint8_t>;
AuthenticationData(AuthenticationType type_ = AuthenticationType::NO_PASSWORD) : type(type_) {}
explicit AuthenticationData(AuthenticationType type_ = AuthenticationType::NO_PASSWORD) : type(type_) {}
AuthenticationData(const AuthenticationData & src) = default;
AuthenticationData & operator =(const AuthenticationData & src) = default;
AuthenticationData(AuthenticationData && src) = default;

View File

@ -158,7 +158,7 @@ public:
private:
friend class AccessControl;
ContextAccess() {}
ContextAccess() {} /// NOLINT
ContextAccess(const AccessControl & access_control_, const Params & params_);
void initialize();

View File

@ -21,7 +21,6 @@ public:
protected:
[[noreturn]] static void throwNotReady();
protected:
bool is_ready = false;
String user_name;
};

View File

@ -60,8 +60,8 @@ public:
private:
friend class QuotaCache;
EnabledQuota(const Params & params_);
EnabledQuota() {}
explicit EnabledQuota(const Params & params_);
EnabledQuota() {} /// NOLINT
const String & getUserName() const { return params.user_name; }

View File

@ -41,7 +41,7 @@ public:
private:
friend class RoleCache;
EnabledRoles(const Params & params_);
explicit EnabledRoles(const Params & params_);
void setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> & info_, scope_guard & notifications);

View File

@ -44,7 +44,7 @@ public:
private:
friend class RowPolicyCache;
EnabledRowPolicies(const Params & params_);
explicit EnabledRowPolicies(const Params & params_);
struct MixedFiltersKey
{

View File

@ -38,7 +38,7 @@ public:
private:
friend class SettingsProfilesCache;
EnabledSettings(const Params & params_);
explicit EnabledSettings(const Params & params_);
void setInfo(const std::shared_ptr<const SettingsProfilesInfo> & info_);
const Params params;

View File

@ -51,7 +51,6 @@ private:
using LDAPCaches = std::map<String, LDAPCache>; // server name -> cache
using LDAPParams = std::map<String, LDAPClient::Params>; // server name -> params
private:
mutable std::recursive_mutex mutex;
LDAPParams ldap_client_params_blueprint;
mutable LDAPCaches ldap_caches;

View File

@ -47,7 +47,6 @@ private:
void resetHandles() noexcept;
void initHandles();
private:
const Params params;
bool is_failed = false;

View File

@ -24,8 +24,8 @@ class ExternalAuthenticators;
class IAccessStorage
{
public:
IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {}
virtual ~IAccessStorage() {}
explicit IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {}
virtual ~IAccessStorage() = default;
/// Returns the name of this storage.
const String & getStorageName() const { return storage_name; }

View File

@ -37,7 +37,7 @@ public:
String getLDAPServerName() const;
public: // IAccessStorage implementations.
// IAccessStorage implementations.
virtual const char * getStorageType() const override;
virtual String getStorageParamsJSON() const override;
virtual bool isReadOnly() const override { return true; }
@ -54,7 +54,6 @@ private: // IAccessStorage implementations.
virtual scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override;
virtual std::optional<UUID> 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;
private:
void setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix);
void processRoleChange(const UUID & id, const AccessEntityPtr & entity);

View File

@ -153,7 +153,7 @@ namespace
}
void LDAPClient::diag(const int rc, String text)
void LDAPClient::diag(int rc, String text)
{
std::scoped_lock lock(ldap_global_mutex);

View File

@ -133,12 +133,11 @@ public:
LDAPClient & operator= (LDAPClient &&) = delete;
protected:
MAYBE_NORETURN void diag(const int rc, String text = "");
MAYBE_NORETURN void diag(int rc, String text = "");
MAYBE_NORETURN bool openConnection();
void closeConnection() noexcept;
SearchResults search(const SearchParams & search_params);
protected:
const Params params;
#if USE_LDAP
LDAP * handle = nullptr;

View File

@ -15,7 +15,7 @@ class MemoryAccessStorage : public IAccessStorage
public:
static constexpr char STORAGE_TYPE[] = "memory";
MemoryAccessStorage(const String & storage_name_ = STORAGE_TYPE);
explicit MemoryAccessStorage(const String & storage_name_ = STORAGE_TYPE);
const char * getStorageType() const override { return STORAGE_TYPE; }

View File

@ -17,7 +17,7 @@ public:
using StoragePtr = std::shared_ptr<Storage>;
using ConstStoragePtr = std::shared_ptr<const Storage>;
MultipleAccessStorage(const String & storage_name_ = STORAGE_TYPE);
explicit MultipleAccessStorage(const String & storage_name_ = STORAGE_TYPE);
~MultipleAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }

View File

@ -19,7 +19,7 @@ struct RolesOrUsersSet;
class QuotaCache
{
public:
QuotaCache(const AccessControl & access_control_);
explicit QuotaCache(const AccessControl & access_control_);
~QuotaCache();
std::shared_ptr<const EnabledQuota> getEnabledQuota(

View File

@ -16,7 +16,7 @@ using RolePtr = std::shared_ptr<const Role>;
class RoleCache
{
public:
RoleCache(const AccessControl & access_control_);
explicit RoleCache(const AccessControl & access_control_);
~RoleCache();
std::shared_ptr<const EnabledRoles> getEnabledRoles(

View File

@ -26,13 +26,13 @@ struct RolesOrUsersSet
RolesOrUsersSet & operator =(RolesOrUsersSet && src) noexcept;
struct AllTag {};
RolesOrUsersSet(AllTag);
RolesOrUsersSet(AllTag); /// NOLINT
RolesOrUsersSet(const UUID & id);
RolesOrUsersSet(const std::vector<UUID> & ids_);
RolesOrUsersSet(const UUID & id); /// NOLINT
RolesOrUsersSet(const std::vector<UUID> & ids_); /// NOLINT
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
RolesOrUsersSet(const ASTRolesOrUsersSet & ast);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast); /// NOLINT
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const std::optional<UUID> & current_user_id);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control);
RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control, const std::optional<UUID> & current_user_id);

View File

@ -18,7 +18,7 @@ using RowPolicyPtr = std::shared_ptr<const RowPolicy>;
class RowPolicyCache
{
public:
RowPolicyCache(const AccessControl & access_control_);
explicit RowPolicyCache(const AccessControl & access_control_);
~RowPolicyCache();
std::shared_ptr<const EnabledRowPolicies> getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set<UUID> & enabled_roles);
@ -26,7 +26,7 @@ public:
private:
struct PolicyInfo
{
PolicyInfo(const RowPolicyPtr & policy_) { setPolicy(policy_); }
explicit PolicyInfo(const RowPolicyPtr & policy_) { setPolicy(policy_); }
void setPolicy(const RowPolicyPtr & policy_);
RowPolicyPtr policy;

View File

@ -51,7 +51,7 @@ class AccessControl;
class SettingsConstraints
{
public:
SettingsConstraints(const AccessControl & access_control_);
explicit SettingsConstraints(const AccessControl & access_control_);
SettingsConstraints(const SettingsConstraints & src);
SettingsConstraints & operator=(const SettingsConstraints & src);
SettingsConstraints(SettingsConstraints && src) noexcept;

View File

@ -15,7 +15,7 @@ struct SettingsConstraintsAndProfileIDs
std::vector<UUID> current_profiles;
std::vector<UUID> enabled_profiles;
SettingsConstraintsAndProfileIDs(const AccessControl & access_control_) : constraints(access_control_) {}
explicit SettingsConstraintsAndProfileIDs(const AccessControl & access_control_) : constraints(access_control_) {}
};
}

View File

@ -33,10 +33,10 @@ struct SettingsProfileElement
friend bool operator <=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(rhs < lhs); }
friend bool operator >=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs < rhs); }
SettingsProfileElement() {}
SettingsProfileElement() = default;
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
SettingsProfileElement(const ASTSettingsProfileElement & ast);
SettingsProfileElement(const ASTSettingsProfileElement & ast); /// NOLINT
SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControl & access_control);
std::shared_ptr<ASTSettingsProfileElement> toAST() const;
std::shared_ptr<ASTSettingsProfileElement> toASTWithNames(const AccessControl & access_control) const;
@ -49,10 +49,10 @@ private:
class SettingsProfileElements : public std::vector<SettingsProfileElement>
{
public:
SettingsProfileElements() {}
SettingsProfileElements() = default;
/// The constructor from AST requires the AccessControl if `ast.id_mode == false`.
SettingsProfileElements(const ASTSettingsProfileElements & ast);
SettingsProfileElements(const ASTSettingsProfileElements & ast); /// NOLINT
SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControl & access_control);
std::shared_ptr<ASTSettingsProfileElements> toAST() const;
std::shared_ptr<ASTSettingsProfileElements> toASTWithNames(const AccessControl & access_control) const;

View File

@ -18,7 +18,7 @@ struct SettingsProfilesInfo;
class SettingsProfilesCache
{
public:
SettingsProfilesCache(const AccessControl & access_control_);
explicit SettingsProfilesCache(const AccessControl & access_control_);
~SettingsProfilesCache();
void setDefaultProfileName(const String & default_profile_name);

View File

@ -29,7 +29,7 @@ struct SettingsProfilesInfo
/// Names of all the profiles in `profiles`.
std::unordered_map<UUID, String> names_of_profiles;
SettingsProfilesInfo(const AccessControl & access_control_) : constraints(access_control_), access_control(access_control_) {}
explicit SettingsProfilesInfo(const AccessControl & access_control_) : constraints(access_control_), access_control(access_control_) {}
std::shared_ptr<const SettingsConstraintsAndProfileIDs> getConstraintsAndProfileIDs(
const std::shared_ptr<const SettingsConstraintsAndProfileIDs> & previous = nullptr) const;

View File

@ -24,8 +24,9 @@ public:
using IsNoPasswordFunction = std::function<bool()>;
using IsPlaintextPasswordFunction = std::function<bool()>;
UsersConfigAccessStorage(const String & storage_name_ = STORAGE_TYPE, const CheckSettingNameFunction & check_setting_name_function_ = {}, const IsNoPasswordFunction & is_no_password_allowed_function_ ={}, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_ = {});
UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_);
UsersConfigAccessStorage(const String & storage_name_ = STORAGE_TYPE, const CheckSettingNameFunction & check_setting_name_function_ = {}, const IsNoPasswordFunction & is_no_password_allowed_function_ ={}, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_ = {}); /// NOLINT
UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_); /// NOLINT
~UsersConfigAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }

View File

@ -97,7 +97,7 @@ private:
/** Calculates the slope of a line between leftmost and rightmost data points.
* (y2 - y1) / (x2 - x1)
*/
Float64 NO_SANITIZE_UNDEFINED getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const
static Float64 NO_SANITIZE_UNDEFINED getBoundingRatio(const AggregateFunctionBoundingRatioData & data)
{
if (data.empty)
return std::numeric_limits<Float64>::quiet_NaN();
@ -111,11 +111,11 @@ public:
return "boundingRatio";
}
AggregateFunctionBoundingRatio(const DataTypes & arguments)
explicit AggregateFunctionBoundingRatio(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>(arguments, {})
{
const auto x_arg = arguments.at(0).get();
const auto y_arg = arguments.at(1).get();
const auto * x_arg = arguments.at(0).get();
const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.",

View File

@ -63,12 +63,12 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto y_col = static_cast<const ColumnUInt8 *>(columns[category_count]);
const auto * y_col = static_cast<const ColumnUInt8 *>(columns[category_count]);
bool y = y_col->getData()[row_num];
for (size_t i : collections::range(0, category_count))
{
auto x_col = static_cast<const ColumnUInt8 *>(columns[i]);
const auto * x_col = static_cast<const ColumnUInt8 *>(columns[i]);
bool x = x_col->getData()[row_num];
if (x)
@ -104,7 +104,7 @@ public:
);
}
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override /// NOLINT
{
auto & col = static_cast<ColumnArray &>(to);
auto & data_col = static_cast<ColumnFloat64 &>(col.getData());

View File

@ -37,7 +37,7 @@ namespace ErrorCodes
class AggregateFunctionCount final : public IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCount>
{
public:
AggregateFunctionCount(const DataTypes & argument_types_) : IAggregateFunctionDataHelper(argument_types_, {}) {}
explicit AggregateFunctionCount(const DataTypes & argument_types_) : IAggregateFunctionDataHelper(argument_types_, {}) {}
String getName() const override { return "count"; }
@ -107,7 +107,7 @@ public:
}
/// Reset the state to specified value. This function is not the part of common interface.
void set(AggregateDataPtr __restrict place, UInt64 new_count) const
static void set(AggregateDataPtr __restrict place, UInt64 new_count)
{
data(place).count = new_count;
}
@ -206,7 +206,7 @@ public:
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) const override
{
auto & nc = assert_cast<const ColumnNullable &>(*columns[0]);
const auto & nc = assert_cast<const ColumnNullable &>(*columns[0]);
if (if_argument_pos >= 0)
{
const auto & flags = assert_cast<const ColumnUInt8 &>(*columns[if_argument_pos]).getData();

View File

@ -91,7 +91,7 @@ private:
size_t num_args;
public:
AggregateFunctionEntropy(const DataTypes & argument_types_)
explicit AggregateFunctionEntropy(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<EntropyData<Value>, AggregateFunctionEntropy<Value>>(argument_types_, {})
, num_args(argument_types_.size())
{

View File

@ -39,7 +39,7 @@ struct AggregateFunctionWithProperties
AggregateFunctionWithProperties & operator = (const AggregateFunctionWithProperties &) = default;
template <typename Creator, std::enable_if_t<!std::is_same_v<Creator, AggregateFunctionWithProperties>> * = nullptr>
AggregateFunctionWithProperties(Creator creator_, AggregateFunctionProperties properties_ = {})
AggregateFunctionWithProperties(Creator creator_, AggregateFunctionProperties properties_ = {}) /// NOLINT
: creator(std::forward<Creator>(creator_)), properties(std::move(properties_))
{
}

View File

@ -144,7 +144,7 @@ public:
}
}
void create(AggregateDataPtr __restrict place) const override
void create(AggregateDataPtr __restrict place) const override /// NOLINT
{
[[maybe_unused]] auto a = new (place) Data;
if constexpr (Trait::sampler == Sampler::RNG)
@ -447,7 +447,7 @@ public:
}
}
void create(AggregateDataPtr __restrict place) const override
void create(AggregateDataPtr __restrict place) const override /// NOLINT
{
[[maybe_unused]] auto a = new (place) Data;
if constexpr (Trait::sampler == Sampler::RNG)

View File

@ -77,11 +77,11 @@ struct MovingAvgData : public MovingData<T>
};
template <typename T, typename Tlimit_num_elems, typename Data>
template <typename T, typename LimitNumElements, typename Data>
class MovingImpl final
: public IAggregateFunctionDataHelper<Data, MovingImpl<T, Tlimit_num_elems, Data>>
: public IAggregateFunctionDataHelper<Data, MovingImpl<T, LimitNumElements, Data>>
{
static constexpr bool limit_num_elems = Tlimit_num_elems::value;
static constexpr bool limit_num_elems = LimitNumElements::value;
UInt64 window_size;
public:
@ -93,7 +93,7 @@ public:
using ColumnResult = ColumnVectorOrDecimal<ResultT>;
explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<Data, MovingImpl<T, Tlimit_num_elems, Data>>({data_type_}, {})
: IAggregateFunctionDataHelper<Data, MovingImpl<T, LimitNumElements, Data>>({data_type_}, {})
, window_size(window_size_) {}
String getName() const override { return Data::name; }

View File

@ -18,7 +18,7 @@ template <typename T, typename Data>
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>
{
public:
AggregateFunctionBitmap(const DataTypePtr & type)
explicit AggregateFunctionBitmap(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>({type}, {})
{
}
@ -55,9 +55,9 @@ template <typename T, typename Data, typename Policy>
class AggregateFunctionBitmapL2 final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmapL2<T, Data, Policy>>
{
private:
static constexpr auto STATE_VERSION_1_MIN_REVISION = 54455;
static constexpr size_t STATE_VERSION_1_MIN_REVISION = 54455;
public:
AggregateFunctionBitmapL2(const DataTypePtr & type)
explicit AggregateFunctionBitmapL2(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitmapL2<T, Data, Policy>>({type}, {})
{
}

View File

@ -151,7 +151,7 @@ public:
/**
* Computes the intersection between two bitmaps
*/
void rb_and(const RoaringBitmapWithSmallSet & r1)
void rb_and(const RoaringBitmapWithSmallSet & r1) /// NOLINT
{
ValueBuffer buffer;
if (isSmall() && r1.isSmall())
@ -195,12 +195,12 @@ public:
/**
* Computes the union between two bitmaps.
*/
void rb_or(const RoaringBitmapWithSmallSet & r1) { merge(r1); }
void rb_or(const RoaringBitmapWithSmallSet & r1) { merge(r1); } /// NOLINT
/**
* Computes the symmetric difference (xor) between two bitmaps.
*/
void rb_xor(const RoaringBitmapWithSmallSet & r1)
void rb_xor(const RoaringBitmapWithSmallSet & r1) /// NOLINT
{
if (isSmall())
toLarge();
@ -212,7 +212,7 @@ public:
/**
* Computes the difference (andnot) between two bitmaps
*/
void rb_andnot(const RoaringBitmapWithSmallSet & r1)
void rb_andnot(const RoaringBitmapWithSmallSet & r1) /// NOLINT
{
ValueBuffer buffer;
if (isSmall() && r1.isSmall())
@ -256,7 +256,7 @@ public:
/**
* Computes the cardinality of the intersection between two bitmaps.
*/
UInt64 rb_and_cardinality(const RoaringBitmapWithSmallSet & r1) const
UInt64 rb_and_cardinality(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
UInt64 ret = 0;
if (isSmall() && r1.isSmall())
@ -283,8 +283,8 @@ public:
/**
* Computes the cardinality of the union between two bitmaps.
*/
UInt64 rb_or_cardinality(const RoaringBitmapWithSmallSet & r1) const
*/
UInt64 rb_or_cardinality(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
UInt64 c1 = size();
UInt64 c2 = r1.size();
@ -294,8 +294,8 @@ public:
/**
* Computes the cardinality of the symmetric difference (andnot) between two bitmaps.
*/
UInt64 rb_xor_cardinality(const RoaringBitmapWithSmallSet & r1) const
*/
UInt64 rb_xor_cardinality(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
UInt64 c1 = size();
UInt64 c2 = r1.size();
@ -306,7 +306,7 @@ public:
/**
* Computes the cardinality of the difference (andnot) between two bitmaps.
*/
UInt64 rb_andnot_cardinality(const RoaringBitmapWithSmallSet & r1) const
UInt64 rb_andnot_cardinality(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
UInt64 c1 = size();
UInt64 inter = rb_and_cardinality(r1);
@ -316,7 +316,7 @@ public:
/**
* Return 1 if the two bitmaps contain the same elements.
*/
UInt8 rb_equals(const RoaringBitmapWithSmallSet & r1)
UInt8 rb_equals(const RoaringBitmapWithSmallSet & r1) /// NOLINT
{
if (isSmall())
toLarge();
@ -329,7 +329,7 @@ public:
* Check whether two bitmaps intersect.
* Intersection with an empty set is always 0 (consistent with hasAny).
*/
UInt8 rb_intersect(const RoaringBitmapWithSmallSet & r1) const
UInt8 rb_intersect(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
if (isSmall())
{
@ -370,7 +370,7 @@ public:
* Empty set is a subset of any other set (consistent with hasAll).
* It's used in subset and currently only support comparing same type
*/
UInt8 rb_is_subset(const RoaringBitmapWithSmallSet & r1) const
UInt8 rb_is_subset(const RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
if (isSmall())
{
@ -420,7 +420,7 @@ public:
/**
* Check whether this bitmap contains the argument.
*/
UInt8 rb_contains(UInt64 x) const
UInt8 rb_contains(UInt64 x) const /// NOLINT
{
if (!std::is_same_v<T, UInt64> && x > rb_max())
return 0;
@ -434,7 +434,7 @@ public:
/**
* Remove value
*/
void rb_remove(UInt64 x)
void rb_remove(UInt64 x) /// NOLINT
{
if (!std::is_same_v<T, UInt64> && x > rb_max())
return;
@ -451,7 +451,7 @@ public:
* range_end - range_start.
* Areas outside the range are passed through unchanged.
*/
void rb_flip(UInt64 begin, UInt64 end)
void rb_flip(UInt64 begin, UInt64 end) /// NOLINT
{
if (isSmall())
toLarge();
@ -462,7 +462,7 @@ public:
/**
* returns the number of integers that are smaller or equal to offsetid.
*/
UInt64 rb_rank(UInt64 x)
UInt64 rb_rank(UInt64 x) /// NOLINT
{
if (isSmall())
toLarge();
@ -474,7 +474,7 @@ public:
* Convert elements to integer array, return number of elements
*/
template <typename Element>
UInt64 rb_to_array(PaddedPODArray<Element> & res) const
UInt64 rb_to_array(PaddedPODArray<Element> & res) const /// NOLINT
{
UInt64 count = 0;
if (isSmall())
@ -500,7 +500,7 @@ public:
* Return new set with specified range (not include the range_end)
* It's used in subset and currently only support UInt32
*/
UInt64 rb_range(UInt64 range_start, UInt64 range_end, RoaringBitmapWithSmallSet & r1) const
UInt64 rb_range(UInt64 range_start, UInt64 range_end, RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
UInt64 count = 0;
if (range_start >= range_end)
@ -540,7 +540,7 @@ public:
* Return new set of the smallest `limit` values in set which is no less than `range_start`.
* It's used in subset and currently only support UInt32
*/
UInt64 rb_limit(UInt64 range_start, UInt64 limit, RoaringBitmapWithSmallSet & r1) const
UInt64 rb_limit(UInt64 range_start, UInt64 limit, RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
if (limit == 0)
return 0;
@ -586,7 +586,7 @@ public:
}
}
UInt64 rb_offset_limit(UInt64 offset, UInt64 limit, RoaringBitmapWithSmallSet & r1) const
UInt64 rb_offset_limit(UInt64 offset, UInt64 limit, RoaringBitmapWithSmallSet & r1) const /// NOLINT
{
if (limit == 0 || offset >= size())
return 0;
@ -617,7 +617,7 @@ public:
}
}
UInt64 rb_min() const
UInt64 rb_min() const /// NOLINT
{
if (isSmall())
{
@ -636,7 +636,7 @@ public:
return rb->minimum();
}
UInt64 rb_max() const
UInt64 rb_max() const /// NOLINT
{
if (isSmall())
{
@ -659,7 +659,7 @@ public:
* Replace value.
* It's used in transform and currently can only support UInt32
*/
void rb_replace(const UInt64 * from_vals, const UInt64 * to_vals, size_t num)
void rb_replace(const UInt64 * from_vals, const UInt64 * to_vals, size_t num) /// NOLINT
{
if (isSmall())
toLarge();

View File

@ -37,11 +37,11 @@ struct AggregateFunctionGroupUniqArrayData
/// Puts all values to the hash set. Returns an array of unique values. Implemented for numeric types.
template <typename T, typename Tlimit_num_elem>
template <typename T, typename LimitNumElems>
class AggregateFunctionGroupUniqArray
: public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayData<T>, AggregateFunctionGroupUniqArray<T, Tlimit_num_elem>>
: public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayData<T>, AggregateFunctionGroupUniqArray<T, LimitNumElems>>
{
static constexpr bool limit_num_elems = Tlimit_num_elem::value;
static constexpr bool limit_num_elems = LimitNumElems::value;
UInt64 max_elems;
private:
@ -50,7 +50,7 @@ private:
public:
AggregateFunctionGroupUniqArray(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayData<T>,
AggregateFunctionGroupUniqArray<T, Tlimit_num_elem>>({argument_type}, parameters_),
AggregateFunctionGroupUniqArray<T, LimitNumElems>>({argument_type}, parameters_),
max_elems(max_elems_) {}
String getName() const override { return "groupUniqArray"; }
@ -139,21 +139,21 @@ static void deserializeAndInsertImpl(StringRef str, IColumn & data_to);
/** Template parameter with true value should be used for columns that store their elements in memory continuously.
* For such columns groupUniqArray() can be implemented more efficiently (especially for small numeric arrays).
*/
template <bool is_plain_column = false, typename Tlimit_num_elem = std::false_type>
template <bool is_plain_column = false, typename LimitNumElems = std::false_type>
class AggregateFunctionGroupUniqArrayGeneric
: public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData,
AggregateFunctionGroupUniqArrayGeneric<is_plain_column, Tlimit_num_elem>>
AggregateFunctionGroupUniqArrayGeneric<is_plain_column, LimitNumElems>>
{
DataTypePtr & input_data_type;
static constexpr bool limit_num_elems = Tlimit_num_elem::value;
static constexpr bool limit_num_elems = LimitNumElems::value;
UInt64 max_elems;
using State = AggregateFunctionGroupUniqArrayGenericData;
public:
AggregateFunctionGroupUniqArrayGeneric(const DataTypePtr & input_data_type_, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData, AggregateFunctionGroupUniqArrayGeneric<is_plain_column, Tlimit_num_elem>>({input_data_type_}, parameters_)
: IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData, AggregateFunctionGroupUniqArrayGeneric<is_plain_column, LimitNumElems>>({input_data_type_}, parameters_)
, input_data_type(this->argument_types[0])
, max_elems(max_elems_) {}

View File

@ -54,13 +54,12 @@ private:
Mean mean;
Weight weight;
WeightedValue operator+ (const WeightedValue & other)
WeightedValue operator+(const WeightedValue & other) const
{
return {mean + other.weight * (other.mean - mean) / (other.weight + weight), other.weight + weight};
}
};
private:
// quantity of stored weighted-values
UInt32 size;
@ -71,7 +70,6 @@ private:
// Weighted values representation of histogram.
WeightedValue points[0];
private:
void sort()
{
::sort(points, points + size,
@ -87,18 +85,18 @@ private:
size_t size = 0;
T * data_ptr;
PriorityQueueStorage(T * value)
explicit PriorityQueueStorage(T * value)
: data_ptr(value)
{
}
void push_back(T val)
void push_back(T val) /// NOLINT
{
data_ptr[size] = std::move(val);
++size;
}
void pop_back() { --size; }
void pop_back() { --size; } /// NOLINT
T * begin() { return data_ptr; }
T * end() const { return data_ptr + size; }
bool empty() const { return size == 0; }

View File

@ -333,7 +333,7 @@ public:
return std::make_shared<DataTypeNumber<Float64>>();
}
void create(AggregateDataPtr __restrict place) const override
void create(AggregateDataPtr __restrict place) const override /// NOLINT
{
std::shared_ptr<IWeightsUpdater> new_weights_updater;
if (weights_updater_name == "SGD")

View File

@ -149,7 +149,7 @@ public:
if (params[0].getType() != Field::Types::String)
throw Exception("Aggregate function " + getName() + " require first parameter to be a String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto param = params[0].get<String>();
const auto & param = params[0].get<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")

View File

@ -1130,7 +1130,7 @@ private:
SerializationPtr serialization;
public:
AggregateFunctionsSingleValue(const DataTypePtr & type)
explicit AggregateFunctionsSingleValue(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionsSingleValue<Data>>({type}, {})
, serialization(type->getDefaultSerialization())
{
@ -1188,7 +1188,7 @@ public:
}
}
void addBatchSinglePlaceNotNull(
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
AggregateDataPtr place,
const IColumn ** columns,

View File

@ -77,7 +77,7 @@ protected:
static bool getFlag(ConstAggregateDataPtr __restrict place) noexcept
{
return result_is_nullable ? place[0] : 1;
return result_is_nullable ? place[0] : true;
}
public:
@ -148,7 +148,7 @@ public:
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> version, Arena * arena) const override
{
bool flag = 1;
bool flag = true;
if constexpr (serialize_flag)
readBinary(flag, buf);
if (flag)
@ -306,7 +306,7 @@ public:
}
}
void addBatchSinglePlace(
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
{
const ColumnNullable * column = assert_cast<const ColumnNullable *>(columns[0]);

View File

@ -108,7 +108,7 @@ public:
place[size_of_data] = 1;
}
void addBatch(
void addBatch( /// NOLINT
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
@ -134,7 +134,7 @@ public:
}
}
void addBatchSinglePlace(
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
{
if (if_argument_pos >= 0)
@ -160,7 +160,7 @@ public:
}
}
void addBatchSinglePlaceNotNull(
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
AggregateDataPtr place,
const IColumn ** columns,

View File

@ -77,7 +77,7 @@ public:
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
auto & b = this->data(rhs);
const auto & b = this->data(rhs);
a.merge(b, arena);
}

View File

@ -21,7 +21,7 @@ class AggregateFunctionResample final : public IAggregateFunctionHelper<Aggregat
{
private:
/// Sanity threshold to avoid creation of too large arrays. The choice of this number is arbitrary.
const size_t MAX_ELEMENTS = 1048576;
static constexpr size_t max_elements = 1048576;
AggregateFunctionPtr nested_function;
@ -75,7 +75,7 @@ public:
total = (sum - 1) / step; // total = (end - begin + step - 1) / step
}
if (total > MAX_ELEMENTS)
if (total > max_elements)
throw Exception("The range given in function "
+ getName() + " contains too many elements",
ErrorCodes::ARGUMENT_OUT_OF_BOUND);

View File

@ -75,12 +75,12 @@ public:
return "retention";
}
AggregateFunctionRetention(const DataTypes & arguments)
explicit AggregateFunctionRetention(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionRetentionData, AggregateFunctionRetention>(arguments, {})
{
for (const auto i : collections::range(0, arguments.size()))
{
auto cond_arg = arguments[i].get();
const auto * cond_arg = arguments[i].get();
if (!isUInt8(cond_arg))
throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i) + " of aggregate function "
+ getName() + ", must be UInt8",

View File

@ -37,7 +37,7 @@ struct ComparePairFirst final
}
};
static constexpr auto max_events = 32;
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionSequenceMatchData final
@ -187,7 +187,7 @@ private:
std::uint64_t extra;
PatternAction() = default;
PatternAction(const PatternActionType type_, const std::uint64_t extra_ = 0) : type{type_}, extra{extra_} {}
explicit PatternAction(const PatternActionType type_, const std::uint64_t extra_ = 0) : type{type_}, extra{extra_} {}
};
using PatternActions = PODArrayWithStackMemory<PatternAction, 64>;
@ -246,7 +246,7 @@ private:
throw_exception("Unknown time condition");
UInt64 duration = 0;
auto prev_pos = pos;
const auto * prev_pos = pos;
pos = tryReadIntText(duration, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
@ -262,7 +262,7 @@ private:
else
{
UInt64 event_number = 0;
auto prev_pos = pos;
const auto * prev_pos = pos;
pos = tryReadIntText(event_number, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
@ -580,7 +580,7 @@ private:
struct DFAState
{
DFAState(bool has_kleene_ = false)
explicit DFAState(bool has_kleene_ = false)
: has_kleene{has_kleene_}, event{0}, transition{DFATransition::None}
{}

View File

@ -216,7 +216,7 @@ public:
a.value.push_back(v->clone(arena), arena);
}
void create(AggregateDataPtr place) const override
void create(AggregateDataPtr place) const override /// NOLINT
{
new (place) Data;
}

View File

@ -185,18 +185,18 @@ private:
std::optional<Float64> max_y;
std::optional<Float64> new_y;
std::vector<std::optional<Float64>> newPoints;
newPoints.reserve(width);
std::vector<std::optional<Float64>> new_points;
new_points.reserve(width);
std::pair<size_t, Float64> bound{0, 0.0};
size_t cur_bucket_num = 0;
// upper bound for bucket
auto upperBound = [&](size_t bucket_num)
auto upper_bound = [&](size_t bucket_num)
{
bound.second = (bucket_num + 1) * multiple_d;
bound.first = std::floor(bound.second);
};
upperBound(cur_bucket_num);
upper_bound(cur_bucket_num);
for (size_t i = 0; i <= (diff_x + 1); ++i)
{
if (i == bound.first) // is bound
@ -211,7 +211,7 @@ private:
{
Float64 avg_y = new_y.value() / multiple_d;
newPoints.emplace_back(avg_y);
new_points.emplace_back(avg_y);
// If min_y has no value, or if the avg_y of the current bucket is less than min_y, update it.
if (!min_y || avg_y < min_y)
min_y = avg_y;
@ -220,12 +220,12 @@ private:
}
else
{
newPoints.emplace_back();
new_points.emplace_back();
}
// next bucket
new_y = found ? ((1 - proportion) * it->getMapped()) : std::optional<Float64>();
upperBound(++cur_bucket_num);
upper_bound(++cur_bucket_num);
}
else
{
@ -240,19 +240,19 @@ private:
Float64 diff_y = max_y.value() - min_y.value();
auto getBars = [&] (const std::optional<Float64> & point_y)
auto get_bars = [&] (const std::optional<Float64> & point_y)
{
value += getBar(point_y ? std::round(((point_y.value() - min_y.value()) / diff_y) * 7) + 1 : 0);
};
auto getBarsForConstant = [&] (const std::optional<Float64> & point_y)
auto get_bars_for_constant = [&] (const std::optional<Float64> & point_y)
{
value += getBar(point_y ? 1 : 0);
};
if (diff_y)
std::for_each(newPoints.begin(), newPoints.end(), getBars);
std::for_each(new_points.begin(), new_points.end(), get_bars);
else
std::for_each(newPoints.begin(), newPoints.end(), getBarsForConstant);
std::for_each(new_points.begin(), new_points.end(), get_bars_for_constant);
}
return value;
}

View File

@ -114,7 +114,7 @@ class AggregateFunctionVariance final
: public IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>
{
public:
AggregateFunctionVariance(const DataTypePtr & arg)
explicit AggregateFunctionVariance(const DataTypePtr & arg)
: IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>({arg}, {}) {}
String getName() const override { return Op::name; }
@ -249,7 +249,6 @@ protected:
readBinary(right_m2, buf);
}
protected:
Float64 left_m2 = 0.0;
Float64 right_m2 = 0.0;
};
@ -367,7 +366,7 @@ class AggregateFunctionCovariance final
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>
{
public:
AggregateFunctionCovariance(const DataTypes & args) : IAggregateFunctionDataHelper<
explicit AggregateFunctionCovariance(const DataTypes & args) : IAggregateFunctionDataHelper<
CovarianceData<T, U, Op, compute_marginal_moments>,
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>(args, {}) {}

View File

@ -80,7 +80,7 @@ public:
using ResultType = typename StatFunc::ResultType;
using ColVecResult = ColumnVector<ResultType>;
AggregateFunctionVarianceSimple(const DataTypes & argument_types_)
explicit AggregateFunctionVarianceSimple(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<typename StatFunc::Data, AggregateFunctionVarianceSimple<StatFunc>>(argument_types_, {})
, src_scale(0)
{}

View File

@ -97,7 +97,7 @@ struct AggregateFunctionSumData
template <typename Value, bool add_if_zero>
void NO_SANITIZE_UNDEFINED NO_INLINE
addManyConditional_internal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
{
const auto * end = ptr + count;
@ -124,7 +124,8 @@ struct AggregateFunctionSumData
/// For floating point we use a similar trick as above, except that now we reinterpret the floating point number as an unsigned
/// integer of the same size and use a mask instead (0 to discard, 0xFF..FF to keep)
static_assert(sizeof(Value) == 4 || sizeof(Value) == 8);
typedef typename std::conditional_t<sizeof(Value) == 4, UInt32, UInt64> equivalent_integer;
using equivalent_integer = typename std::conditional_t<sizeof(Value) == 4, UInt32, UInt64>;
constexpr size_t unroll_count = 128 / sizeof(T);
T partial_sums[unroll_count]{};
@ -163,13 +164,13 @@ struct AggregateFunctionSumData
template <typename Value>
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t count)
{
return addManyConditional_internal<Value, true>(ptr, null_map, count);
return addManyConditionalInternal<Value, true>(ptr, null_map, count);
}
template <typename Value>
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t count)
{
return addManyConditional_internal<Value, false>(ptr, cond_map, count);
return addManyConditionalInternal<Value, false>(ptr, cond_map, count);
}
void NO_SANITIZE_UNDEFINED merge(const AggregateFunctionSumData & rhs)
@ -248,7 +249,7 @@ struct AggregateFunctionSumKahanData
}
template <typename Value, bool add_if_zero>
void NO_INLINE addManyConditional_internal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
void NO_INLINE addManyConditionalInternal(const Value * __restrict ptr, const UInt8 * __restrict condition_map, size_t count)
{
constexpr size_t unroll_count = 4;
T partial_sums[unroll_count]{};
@ -281,13 +282,13 @@ struct AggregateFunctionSumKahanData
template <typename Value>
void ALWAYS_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t count)
{
return addManyConditional_internal<Value, true>(ptr, null_map, count);
return addManyConditionalInternal<Value, true>(ptr, null_map, count);
}
template <typename Value>
void ALWAYS_INLINE addManyConditional(const Value * __restrict ptr, const UInt8 * __restrict cond_map, size_t count)
{
return addManyConditional_internal<Value, false>(ptr, cond_map, count);
return addManyConditionalInternal<Value, false>(ptr, cond_map, count);
}
void ALWAYS_INLINE mergeImpl(T & to_sum, T & to_compensation, T from_sum, T from_compensation)
@ -351,7 +352,7 @@ public:
__builtin_unreachable();
}
AggregateFunctionSum(const DataTypes & argument_types_)
explicit AggregateFunctionSum(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionSum<T, TResult, Data, Type>>(argument_types_, {})
, scale(0)
{}

View File

@ -489,15 +489,15 @@ public:
"Aggregate function '{}' requires exactly one parameter "
"of Array type", getName());
Array keys_to_keep_;
if (!params_.front().tryGet<Array>(keys_to_keep_))
Array keys_to_keep_values;
if (!params_.front().tryGet<Array>(keys_to_keep_values))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Aggregate function {} requires an Array as a parameter",
getName());
keys_to_keep.reserve(keys_to_keep_.size());
keys_to_keep.reserve(keys_to_keep_values.size());
for (const Field & f : keys_to_keep_)
for (const Field & f : keys_to_keep_values)
keys_to_keep.emplace(f.safeGet<T>());
}

View File

@ -64,7 +64,7 @@ namespace ErrorCodes
* Both WelchTTest and StudentTTest have t-statistric with Student distribution but with different degrees of freedom.
* So the procedure of computing p-value is the same.
*/
static inline Float64 getPValue(Float64 degrees_of_freedom, Float64 t_stat2)
static inline Float64 getPValue(Float64 degrees_of_freedom, Float64 t_stat2) /// NOLINT
{
Float64 numerator = integrateSimpson(0, degrees_of_freedom / (t_stat2 + degrees_of_freedom),
[degrees_of_freedom](double x) { return std::pow(x, degrees_of_freedom / 2 - 1) / std::sqrt(1 - x); });
@ -92,7 +92,7 @@ public:
AggregateFunctionTTest(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>({arguments}, params)
{
if (params.size() > 0)
if (!params.empty())
{
need_confidence_interval = true;
confidence_level = params.at(0).safeGet<Float64>();

View File

@ -20,7 +20,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
static constexpr auto max_events = 32;
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionWindowFunnelData

View File

@ -117,7 +117,7 @@ template <typename Data>
class AggregateFunctionCrossTab : public IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>>
{
public:
AggregateFunctionCrossTab(const DataTypes & arguments)
explicit AggregateFunctionCrossTab(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>>({arguments}, {})
{
}

View File

@ -55,7 +55,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
{
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<Data<TYPE>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<Data<TYPE>>(std::forward<TArgs>(args)...); /// NOLINT
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate<Data<Int8>>(std::forward<TArgs>(args)...);
@ -94,7 +94,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
{
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<TYPE, Data<TYPE>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<TYPE, Data<TYPE>>(std::forward<TArgs>(args)...); /// NOLINT
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate<Int8, Data<Int8>>(std::forward<TArgs>(args)...);
@ -121,7 +121,7 @@ static IAggregateFunction * createWithBasicNumberOrDateOrDateTime(const IDataTyp
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return new AggregateFunctionTemplate<TYPE, Data<TYPE>>(std::forward<TArgs>(args)...);
return new AggregateFunctionTemplate<TYPE, Data<TYPE>>(std::forward<TArgs>(args)...); /// NOLINT
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH

View File

@ -25,7 +25,7 @@ static IAggregateFunction * createAggregateFunctionSingleValue(const String & na
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<TYPE>>>(argument_type);
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<TYPE>>>(argument_type); /// NOLINT
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
@ -56,7 +56,7 @@ static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTyp
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type);
return new AggregateFunctionArgMinMax<AggregateFunctionArgMinMaxData<ResData, MinMaxData<SingleValueDataFixed<TYPE>>>>(res_type, val_type); /// NOLINT
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
@ -90,7 +90,7 @@ static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name
WhichDataType which(res_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<TYPE>>(res_type, val_type);
return createAggregateFunctionArgMinMaxSecond<MinMaxData, SingleValueDataFixed<TYPE>>(res_type, val_type); /// NOLINT
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH

View File

@ -127,10 +127,10 @@ public:
virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0;
/// Serializes state (to transmit it over the network, for example).
virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version = std::nullopt) const = 0;
virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version = std::nullopt) const = 0; /// NOLINT
/// Deserializes state. This function is called only for empty (just created) states.
virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> version = std::nullopt, Arena * arena = nullptr) const = 0;
virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> version = std::nullopt, Arena * arena = nullptr) const = 0; /// NOLINT
/// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()).
virtual bool allocatesMemoryInArena() const = 0;
@ -174,7 +174,7 @@ public:
/** Contains a loop with calls to "add" function. You can collect arguments into array "places"
* and do a single call to "addBatch" for devirtualization and inlining.
*/
virtual void addBatch(
virtual void addBatch( /// NOLINT
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
@ -198,7 +198,7 @@ public:
/** The same for single place.
*/
virtual void addBatchSinglePlace(
virtual void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const = 0;
/// The version of "addBatchSinglePlace", that handle sparse columns as arguments.
@ -208,7 +208,7 @@ public:
/** The same for single place when need to aggregate only filtered data.
* Instead of using an if-column, the condition is combined inside the null_map
*/
virtual void addBatchSinglePlaceNotNull(
virtual void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
AggregateDataPtr place,
const IColumn ** columns,
@ -216,7 +216,7 @@ public:
Arena * arena,
ssize_t if_argument_pos = -1) const = 0;
virtual void addBatchSinglePlaceFromInterval(
virtual void addBatchSinglePlaceFromInterval( /// NOLINT
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
const = 0;
@ -354,7 +354,7 @@ public:
AddFunc getAddressOfAddFunction() const override { return &addFree; }
void addBatch(
void addBatch( /// NOLINT
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
@ -407,7 +407,7 @@ public:
static_cast<const Derived *>(this)->merge(places[i] + place_offset, rhs[i], arena);
}
void addBatchSinglePlace(
void addBatchSinglePlace( /// NOLINT
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
{
if (if_argument_pos >= 0)
@ -439,7 +439,7 @@ public:
static_cast<const Derived *>(this)->add(place, &values, offset_it.getValueIndex(), arena);
}
void addBatchSinglePlaceNotNull(
void addBatchSinglePlaceNotNull( /// NOLINT
size_t batch_size,
AggregateDataPtr place,
const IColumn ** columns,
@ -462,7 +462,7 @@ public:
}
}
void addBatchSinglePlaceFromInterval(
void addBatchSinglePlaceFromInterval( /// NOLINT
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
const override
{
@ -586,7 +586,7 @@ public:
IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_)
: IAggregateFunctionHelper<Derived>(argument_types_, parameters_) {}
void create(AggregateDataPtr place) const override
void create(AggregateDataPtr place) const override /// NOLINT
{
new (place) Data;
}

View File

@ -68,7 +68,7 @@ public:
const DataTypes & arguments,
const Array & params) const = 0;
virtual ~IAggregateFunctionCombinator() {}
virtual ~IAggregateFunctionCombinator() = default;
};
using AggregateFunctionCombinatorPtr = std::shared_ptr<const IAggregateFunctionCombinator>;

View File

@ -65,7 +65,7 @@ template <typename T, ReservoirSamplerOnEmpty::Enum OnEmpty = ReservoirSamplerOn
class ReservoirSampler
{
public:
ReservoirSampler(size_t sample_count_ = DEFAULT_SAMPLE_COUNT)
explicit ReservoirSampler(size_t sample_count_ = DEFAULT_SAMPLE_COUNT)
: sample_count(sample_count_)
{
rng.seed(123456);
@ -111,7 +111,7 @@ public:
sortIfNeeded();
double index = level * (samples.size() - 1);
size_t int_index = static_cast<size_t>(index + 0.5);
size_t int_index = static_cast<size_t>(index + 0.5); /// NOLINT
int_index = std::max(0LU, std::min(samples.size() - 1, int_index));
return samples[int_index];
}
@ -190,7 +190,7 @@ public:
}
else
{
for (double i = 0; i < sample_count; i += frequency)
for (double i = 0; i < sample_count; i += frequency) /// NOLINT
samples[i] = b.samples[i];
}
}

View File

@ -67,7 +67,7 @@ private:
}
public:
ReservoirSamplerDeterministic(const size_t max_sample_size_ = detail::DEFAULT_MAX_SAMPLE_SIZE)
explicit ReservoirSamplerDeterministic(const size_t max_sample_size_ = detail::DEFAULT_MAX_SAMPLE_SIZE)
: max_sample_size{max_sample_size_}
{
}
@ -103,7 +103,7 @@ public:
sortIfNeeded();
double index = level * (samples.size() - 1);
size_t int_index = static_cast<size_t>(index + 0.5);
size_t int_index = static_cast<size_t>(index + 0.5); /// NOLINT
int_index = std::max(0LU, std::min(samples.size() - 1, int_index));
return samples[int_index].first;
}

View File

@ -6,6 +6,7 @@
#include <boost/noncopyable.hpp>
#include <memory>
#include <base/StringRef.h>
#include <theta_sketch.hpp>
#include <theta_union.hpp>

View File

@ -105,8 +105,8 @@ private:
}
}
inline size_t buf_size() const { return 1ULL << size_degree; }
inline size_t max_fill() const { return 1ULL << (size_degree - 1); }
inline size_t buf_size() const { return 1ULL << size_degree; } /// NOLINT
inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT
inline size_t mask() const { return buf_size() - 1; }
inline size_t place(HashValue x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); }
@ -304,8 +304,11 @@ public:
memcpy(buf, rhs.buf, buf_size() * sizeof(buf[0]));
}
UniquesHashSet & operator= (const UniquesHashSet & rhs)
UniquesHashSet & operator=(const UniquesHashSet & rhs)
{
if (&rhs == this)
return *this;
if (size_degree != rhs.size_degree)
{
free();

View File

@ -12,7 +12,7 @@ class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
BackupEntryFromAppendOnlyFile(
explicit BackupEntryFromAppendOnlyFile(
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},

View File

@ -16,7 +16,7 @@ class BackupEntryFromImmutableFile : public IBackupEntry
{
public:
/// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data.
BackupEntryFromImmutableFile(
explicit BackupEntryFromImmutableFile(
const String & file_path_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {},

View File

@ -13,7 +13,7 @@ class BackupEntryFromMemory : public IBackupEntry
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromMemory(const void * data_, size_t size_, const std::optional<UInt128> & checksum_ = {});
BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_ = {});
explicit BackupEntryFromMemory(String data_, const std::optional<UInt128> & checksum_ = {});
UInt64 getSize() const override { return data.size(); }
std::optional<UInt128> getChecksum() const override { return checksum; }

View File

@ -14,7 +14,7 @@ class BackupEntryFromSmallFile : public BackupEntryFromMemory
{
public:
/// The constructor is allowed to not set `checksum_`, in that case it will be calculated from the data.
BackupEntryFromSmallFile(
explicit BackupEntryFromSmallFile(
const String & file_path_,
const std::optional<UInt128> & checksum_ = {});

View File

@ -16,7 +16,7 @@ using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
class IBackup : public std::enable_shared_from_this<IBackup>, public TypePromotion<IBackup>
{
public:
IBackup() {}
IBackup() = default;
virtual ~IBackup() = default;
/// Name of the backup.
@ -44,7 +44,7 @@ public:
/// before the terminator. For example, list("", "") returns names of all the entries
/// in the backup; and list("data/", "/") return kind of a list of folders and
/// files stored in the "data/" directory inside the backup.
virtual Strings listFiles(const String & prefix = "", const String & terminator = "/") const = 0;
virtual Strings listFiles(const String & prefix = "", const String & terminator = "/") const = 0; /// NOLINT
/// Checks if an entry with a specified name exists.
virtual bool fileExists(const String & file_name) const = 0;

View File

@ -103,7 +103,6 @@ protected:
const std::vector<Arguments> & hosts_and_ports_arguments) = 0;
virtual void processConfig() = 0;
protected:
bool processQueryText(const String & text);
private:

View File

@ -22,8 +22,8 @@ struct ConnectionParameters
Protocol::Compression compression = Protocol::Compression::Enable;
ConnectionTimeouts timeouts;
ConnectionParameters() {}
ConnectionParameters(const Poco::Util::AbstractConfiguration & config);
ConnectionParameters() = default;
explicit ConnectionParameters(const Poco::Util::AbstractConfiguration & config);
ConnectionParameters(const Poco::Util::AbstractConfiguration & config, std::string host, int port);
static int getPortFromConfig(const Poco::Util::AbstractConfiguration & config);

View File

@ -25,12 +25,11 @@ class IConnectionPool : private boost::noncopyable
public:
using Entry = PoolBase<Connection>::Entry;
public:
virtual ~IConnectionPool() = default;
/// Selects the connection to work.
/// If force_connected is false, the client must manually ensure that returned connection is good.
virtual Entry get(const ConnectionTimeouts & timeouts,
virtual Entry get(const ConnectionTimeouts & timeouts, /// NOLINT
const Settings * settings = nullptr,
bool force_connected = true) = 0;
@ -76,7 +75,7 @@ public:
{
}
Entry get(const ConnectionTimeouts & timeouts,
Entry get(const ConnectionTimeouts & timeouts, /// NOLINT
const Settings * settings = nullptr,
bool force_connected = true) override
{

View File

@ -109,7 +109,6 @@ private:
GetPriorityFunc makeGetPriorityFunc(const Settings * settings);
private:
std::vector<size_t> hostname_differences; /// Distances from name of this host to the names of hosts of pools.
size_t last_used = 0; /// Last used for round_robin policy.
LoadBalancing default_load_balancing;

View File

@ -22,7 +22,7 @@ struct PocoSocketWrapper : public Poco::Net::SocketImpl
~PocoSocketWrapper() override { reset(-1); }
};
void IConnections::DrainCallback::operator()(int fd, Poco::Timespan, const std::string fd_description) const
void IConnections::DrainCallback::operator()(int fd, Poco::Timespan, const std::string & fd_description) const
{
if (!PocoSocketWrapper(fd).poll(drain_timeout, Poco::Net::Socket::SELECT_READ))
{

View File

@ -16,7 +16,7 @@ public:
struct DrainCallback
{
Poco::Timespan drain_timeout;
void operator()(int fd, Poco::Timespan, const std::string fd_description = "") const;
void operator()(int fd, Poco::Timespan, const std::string & fd_description = "") const;
};
/// Send all scalars to replicas.

View File

@ -14,7 +14,6 @@
#include <Storages/MergeTree/RequestResponse.h>
#include <boost/noncopyable.hpp>

View File

@ -546,7 +546,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
* small probability. Do this after we add this fuzzer to CI and fix all the
* problems it can routinely find even in this boring version.
*/
void QueryFuzzer::collectFuzzInfoMain(const ASTPtr ast)
void QueryFuzzer::collectFuzzInfoMain(ASTPtr ast)
{
collectFuzzInfoRecurse(ast);
@ -569,7 +569,7 @@ void QueryFuzzer::collectFuzzInfoMain(const ASTPtr ast)
}
}
void QueryFuzzer::addTableLike(const ASTPtr ast)
void QueryFuzzer::addTableLike(ASTPtr ast)
{
if (table_like_map.size() > 1000)
{
@ -583,7 +583,7 @@ void QueryFuzzer::addTableLike(const ASTPtr ast)
}
}
void QueryFuzzer::addColumnLike(const ASTPtr ast)
void QueryFuzzer::addColumnLike(ASTPtr ast)
{
if (column_like_map.size() > 1000)
{
@ -606,7 +606,7 @@ void QueryFuzzer::addColumnLike(const ASTPtr ast)
}
}
void QueryFuzzer::collectFuzzInfoRecurse(const ASTPtr ast)
void QueryFuzzer::collectFuzzInfoRecurse(ASTPtr ast)
{
if (auto * impl = dynamic_cast<ASTWithAlias *>(ast.get()))
{

View File

@ -71,10 +71,10 @@ struct QueryFuzzer
void fuzzWindowFrame(ASTWindowDefinition & def);
void fuzz(ASTs & asts);
void fuzz(ASTPtr & ast);
void collectFuzzInfoMain(const ASTPtr ast);
void addTableLike(const ASTPtr ast);
void addColumnLike(const ASTPtr ast);
void collectFuzzInfoRecurse(const ASTPtr ast);
void collectFuzzInfoMain(ASTPtr ast);
void addTableLike(ASTPtr ast);
void addColumnLike(ASTPtr ast);
void collectFuzzInfoRecurse(ASTPtr ast);
};
}

View File

@ -33,7 +33,7 @@ public:
private:
AvailableCollationLocales();
private:
AvailableLocalesMap locales_map;
};

View File

@ -92,7 +92,7 @@ private:
/// Create a new column that has another column as a source.
MutablePtr createView() const;
ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional<size_t> version_ = std::nullopt);
explicit ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional<size_t> version_ = std::nullopt);
ColumnAggregateFunction(const AggregateFunctionPtr & func_, const ConstArenas & arenas_);

View File

@ -158,7 +158,7 @@ public:
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnArray *>(&rhs))
if (const auto * rhs_concrete = typeid_cast<const ColumnArray *>(&rhs))
return data->structureEquals(*rhs_concrete->data);
return false;
}

View File

@ -126,7 +126,7 @@ protected:
Lazy lazy;
private:
[[noreturn]] void throwMustBeDecompressed() const
[[noreturn]] static void throwMustBeDecompressed()
{
throw Exception("ColumnCompressed must be decompressed before use", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -163,7 +163,7 @@ public:
const char * deserializeAndInsertFromArena(const char * pos) override
{
auto res = data->deserializeAndInsertFromArena(pos);
const auto * res = data->deserializeAndInsertFromArena(pos);
data->popBack(1);
++s;
return res;
@ -242,7 +242,7 @@ public:
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnConst *>(&rhs))
if (const auto * rhs_concrete = typeid_cast<const ColumnConst *>(&rhs))
return data->structureEquals(*rhs_concrete->data);
return false;
}

View File

@ -41,9 +41,9 @@ private:
using ComparatorEqual = ComparatorEqualImpl<ComparatorBase>;
/** Create an empty column of strings of fixed-length `n` */
ColumnFixedString(size_t n_) : n(n_) {}
explicit ColumnFixedString(size_t n_) : n(n_) {}
ColumnFixedString(const ColumnFixedString & src) : chars(src.chars.begin(), src.chars.end()), n(src.n) {}
ColumnFixedString(const ColumnFixedString & src) : chars(src.chars.begin(), src.chars.end()), n(src.n) {} /// NOLINT
public:
std::string getName() const override { return "FixedString(" + std::to_string(n) + ")"; }
@ -190,7 +190,7 @@ public:
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_concrete = typeid_cast<const ColumnFixedString *>(&rhs))
if (const auto * rhs_concrete = typeid_cast<const ColumnFixedString *>(&rhs))
return n == rhs_concrete->n;
return false;
}

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
}
ColumnFunction::ColumnFunction(size_t size, FunctionBasePtr function_, const ColumnsWithTypeAndName & columns_to_capture, bool is_short_circuit_argument_, bool is_function_compiled_)
: size_(size), function(function_), is_short_circuit_argument(is_short_circuit_argument_), is_function_compiled(is_function_compiled_)
: elements_size(size), function(function_), is_short_circuit_argument(is_short_circuit_argument_), is_function_compiled(is_function_compiled_)
{
appendArguments(columns_to_capture);
}
@ -40,15 +40,15 @@ MutableColumnPtr ColumnFunction::cloneResized(size_t size) const
ColumnPtr ColumnFunction::replicate(const Offsets & offsets) const
{
if (size_ != offsets.size())
if (elements_size != offsets.size())
throw Exception("Size of offsets (" + toString(offsets.size()) + ") doesn't match size of column ("
+ toString(size_) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
+ toString(elements_size) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnsWithTypeAndName capture = captured_columns;
for (auto & column : capture)
column.column = column.column->replicate(offsets);
size_t replicated_size = 0 == size_ ? 0 : offsets.back();
size_t replicated_size = 0 == elements_size ? 0 : offsets.back();
return ColumnFunction::create(replicated_size, function, capture, is_short_circuit_argument, is_function_compiled);
}
@ -75,7 +75,7 @@ void ColumnFunction::insertFrom(const IColumn & src, size_t n)
captured_columns[i].column = std::move(mut_column);
}
++size_;
++elements_size;
}
void ColumnFunction::insertRangeFrom(const IColumn & src, size_t start, size_t length)
@ -92,14 +92,14 @@ void ColumnFunction::insertRangeFrom(const IColumn & src, size_t start, size_t l
captured_columns[i].column = std::move(mut_column);
}
size_ += length;
elements_size += length;
}
ColumnPtr ColumnFunction::filter(const Filter & filt, ssize_t result_size_hint) const
{
if (size_ != filt.size())
if (elements_size != filt.size())
throw Exception("Size of filter (" + toString(filt.size()) + ") doesn't match size of column ("
+ toString(size_) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
+ toString(elements_size) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
ColumnsWithTypeAndName capture = captured_columns;
for (auto & column : capture)
@ -124,7 +124,7 @@ void ColumnFunction::expand(const Filter & mask, bool inverted)
column.column->assumeMutable()->expand(mask, inverted);
}
size_ = mask.size();
elements_size = mask.size();
}
ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const
@ -150,9 +150,9 @@ ColumnPtr ColumnFunction::index(const IColumn & indexes, size_t limit) const
std::vector<MutableColumnPtr> ColumnFunction::scatter(IColumn::ColumnIndex num_columns,
const IColumn::Selector & selector) const
{
if (size_ != selector.size())
if (elements_size != selector.size())
throw Exception("Size of selector (" + toString(selector.size()) + ") doesn't match size of column ("
+ toString(size_) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
+ toString(elements_size) + ")", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
std::vector<size_t> counts;
if (captured_columns.empty())
@ -266,7 +266,7 @@ ColumnWithTypeAndName ColumnFunction::reduce() const
if (is_function_compiled)
ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute);
res.column = function->execute(columns, res.type, size_);
res.column = function->execute(columns, res.type, elements_size);
return res;
}

View File

@ -37,7 +37,7 @@ public:
MutableColumnPtr cloneResized(size_t size) const override;
size_t size() const override { return size_; }
size_t size() const override { return elements_size; }
ColumnPtr cut(size_t start, size_t length) const override;
ColumnPtr replicate(const Offsets & offsets) const override;
@ -178,7 +178,7 @@ public:
DataTypePtr getResultType() const;
private:
size_t size_;
size_t elements_size;
FunctionBasePtr function;
ColumnsWithTypeAndName captured_columns;

View File

@ -179,7 +179,7 @@ public:
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_low_cardinality = typeid_cast<const ColumnLowCardinality *>(&rhs))
if (const auto * rhs_low_cardinality = typeid_cast<const ColumnLowCardinality *>(&rhs))
return idx.getPositions()->structureEquals(*rhs_low_cardinality->idx.getPositions())
&& dictionary.getColumnUnique().structureEquals(rhs_low_cardinality->dictionary.getColumnUnique());
return false;

View File

@ -11,7 +11,7 @@ class ColumnNothing final : public COWHelper<IColumnDummy, ColumnNothing>
private:
friend class COWHelper<IColumnDummy, ColumnNothing>;
ColumnNothing(size_t s_)
explicit ColumnNothing(size_t s_)
{
s = s_;
}

View File

@ -137,7 +137,7 @@ public:
bool structureEquals(const IColumn & rhs) const override
{
if (auto rhs_nullable = typeid_cast<const ColumnNullable *>(&rhs))
if (const auto * rhs_nullable = typeid_cast<const ColumnNullable *>(&rhs))
return nested_column->structureEquals(*rhs_nullable->nested_column);
return false;
}

View File

@ -40,7 +40,7 @@ public:
template <typename TColumnPtr, typename = typename std::enable_if<IsMutableColumns<TColumnPtr>::value>::type>
static MutablePtr create(TColumnPtr && values_, TColumnPtr && offsets_, size_t size_)
{
return Base::create(std::move(values_), std::move(offsets_), size_);
return Base::create(std::forward<TColumnPtr>(values_), std::forward<TColumnPtr>(offsets_), size_);
}
static Ptr create(const ColumnPtr & values_)
@ -230,7 +230,7 @@ private:
/// Sorted offsets of non-default values in the full column.
/// 'offsets[i]' corresponds to 'values[i + 1]'.
WrappedPtr offsets;
size_t _size;
size_t _size; /// NOLINT
};
ColumnPtr recursiveRemoveSparse(const ColumnPtr & column);

View File

@ -317,7 +317,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
if (valuesHaveFixedSize())
return uniqueInsertData(&x.reinterpret<char>(), size_of_value_if_fixed);
auto & val = x.get<String>();
const auto & val = x.get<String>();
return uniqueInsertData(val.data(), val.size());
}
@ -327,7 +327,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
if (is_nullable && src.isNullAt(n))
return getNullValueIndex();
if (auto * nullable = checkAndGetColumn<ColumnNullable>(src))
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(src))
return uniqueInsertFrom(nullable->getNestedColumn(), n);
auto ref = src.getDataAt(n);
@ -354,7 +354,7 @@ StringRef ColumnUnique<ColumnType>::serializeValueIntoArena(size_t n, Arena & ar
{
static constexpr auto s = sizeof(UInt8);
auto pos = arena.allocContinue(s, begin);
auto * pos = arena.allocContinue(s, begin);
UInt8 flag = (n == getNullValueIndex() ? 1 : 0);
unalignedStore<UInt8>(pos, flag);
@ -491,7 +491,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
return nullptr;
};
if (auto * nullable_column = checkAndGetColumn<ColumnNullable>(src))
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(src))
{
src_column = typeid_cast<const ColumnType *>(&nullable_column->getNestedColumn());
null_map = &nullable_column->getNullMapData();
@ -555,7 +555,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
template <typename ColumnType>
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
{
auto callForType = [this, &src, start, length](auto x) -> MutableColumnPtr
auto call_for_type = [this, &src, start, length](auto x) -> MutableColumnPtr
{
size_t size = getRawColumnPtr()->size();
@ -571,13 +571,13 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn &
MutableColumnPtr positions_column;
if (!positions_column)
positions_column = callForType(UInt8());
positions_column = call_for_type(UInt8());
if (!positions_column)
positions_column = callForType(UInt16());
positions_column = call_for_type(UInt16());
if (!positions_column)
positions_column = callForType(UInt32());
positions_column = call_for_type(UInt32());
if (!positions_column)
positions_column = callForType(UInt64());
positions_column = call_for_type(UInt64());
if (!positions_column)
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);
@ -598,7 +598,7 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
if (!overflowed_keys_ptr)
throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR);
auto callForType = [this, &src, start, length, overflowed_keys_ptr, max_dictionary_size](auto x) -> MutableColumnPtr
auto call_for_type = [this, &src, start, length, overflowed_keys_ptr, max_dictionary_size](auto x) -> MutableColumnPtr
{
size_t size = getRawColumnPtr()->size();
@ -617,13 +617,13 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
MutableColumnPtr positions_column;
if (!positions_column)
positions_column = callForType(UInt8());
positions_column = call_for_type(UInt8());
if (!positions_column)
positions_column = callForType(UInt16());
positions_column = call_for_type(UInt16());
if (!positions_column)
positions_column = callForType(UInt32());
positions_column = call_for_type(UInt32());
if (!positions_column)
positions_column = callForType(UInt64());
positions_column = call_for_type(UInt64());
if (!positions_column)
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);

View File

@ -127,8 +127,8 @@ public:
using Container = PaddedPODArray<ValueType>;
private:
ColumnVector() {}
ColumnVector(const size_t n) : data(n) {}
ColumnVector() = default;
explicit ColumnVector(const size_t n) : data(n) {}
ColumnVector(const size_t n, const ValueType x) : data(n, x) {}
ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {}

View File

@ -96,13 +96,13 @@ ColumnPtr selectIndexImpl(const Column & column, const IColumn & indexes, size_t
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH,
"Size of indexes ({}) is less than required ({})", indexes.size(), limit);
if (auto * data_uint8 = detail::getIndexesData<UInt8>(indexes))
if (const auto * data_uint8 = detail::getIndexesData<UInt8>(indexes))
return column.template indexImpl<UInt8>(*data_uint8, limit);
else if (auto * data_uint16 = detail::getIndexesData<UInt16>(indexes))
else if (const auto * data_uint16 = detail::getIndexesData<UInt16>(indexes))
return column.template indexImpl<UInt16>(*data_uint16, limit);
else if (auto * data_uint32 = detail::getIndexesData<UInt32>(indexes))
else if (const auto * data_uint32 = detail::getIndexesData<UInt32>(indexes))
return column.template indexImpl<UInt32>(*data_uint32, limit);
else if (auto * data_uint64 = detail::getIndexesData<UInt64>(indexes))
else if (const auto * data_uint64 = detail::getIndexesData<UInt64>(indexes))
return column.template indexImpl<UInt64>(*data_uint64, limit);
else
throw Exception("Indexes column for IColumn::select must be ColumnUInt, got " + indexes.getName(),
@ -118,6 +118,7 @@ ColumnPtr permuteImpl(const Column & column, const IColumn::Permutation & perm,
return column.indexImpl(perm, limit);
}
/// NOLINTNEXTLINE
#define INSTANTIATE_INDEX_IMPL(Column) \
template ColumnPtr Column::indexImpl<UInt8>(const PaddedPODArray<UInt8> & indexes, size_t limit) const; \
template ColumnPtr Column::indexImpl<UInt16>(const PaddedPODArray<UInt16> & indexes, size_t limit) const; \

View File

@ -413,7 +413,7 @@ public:
/// Returns ration of values in column, that equal to default value of column.
/// Checks only @sample_ratio ratio of rows.
virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0;
virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// NOLINT
/// Returns indices of values in column, that not equal to default value of column.
virtual void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const = 0;

View File

@ -24,9 +24,8 @@ class IColumnDummy : public IColumn
{
public:
IColumnDummy() : s(0) {}
IColumnDummy(size_t s_) : s(s_) {}
explicit IColumnDummy(size_t s_) : s(s_) {}
public:
virtual MutableColumnPtr cloneDummy(size_t s_) const = 0;
MutableColumnPtr cloneResized(size_t s_) const override { return cloneDummy(s_); }

View File

@ -17,300 +17,298 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
namespace
template <typename ColumnType, bool with_saved_hash, bool has_base_index>
struct ReverseIndexHashTableState;
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ false>
{
template <typename ColumnType, bool with_saved_hash, bool has_base_index>
struct ReverseIndexHashTableState;
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = false;
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ false>
ColumnType * index_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = true;
ColumnType * index_column;
size_t base_index;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = false;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = true;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
size_t base_index;
};
struct ReverseIndexHash
{
template <typename T>
size_t operator()(T) const
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = false;
ColumnType * index_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = true;
ColumnType * index_column;
size_t base_index;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = false;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = true;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
size_t base_index;
};
struct ReverseIndexHash
{
template <typename T>
size_t operator()(T) const
{
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
}
};
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
struct ReverseIndexHashTableCell
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
{
using Base = HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>;
using State = typename Base::State;
using Base::Base;
using Base::key;
using Base::keyEquals;
using Base::isZero;
template <typename T>
static bool isZero(const T &, const State & /*state*/)
{
/// 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>);
return false;
}
/// Special case when we want to compare with something not in index_column.
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
{
auto index = key;
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index);
else
return object == state.index_column->getDataAt(index);
}
size_t getHash(const Hash & hash) const
{
auto index = key;
/// Hack. HashTable is Hash itself.
const auto & state = static_cast<const State &>(static_cast<const HashTable &>(hash));
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return (*state.saved_hash_column)[index];
else
{
using ValueType = typename ColumnType::ValueType;
ValueType value = unalignedLoad<ValueType>(state.index_column->getDataAt(index).data);
return DefaultHash<ValueType>()(value);
}
}
};
/**
* ReverseIndexHashTableBase implements a special hash table interface for
* reverse index.
*
* The following requirements are different compared to a plain hash table:
*
* 1) Provide public access to 'hash table state' that contains
* additional data needed to calculate cell hashes.
*
* 2) Support emplace() and find() with a Key different from the resulting
* hash table key. This means emplace() accepts a different kind of object
* as a key, and then the real key can be read from the returned cell iterator.
*
* These requirements are unique to ReverseIndex and are in conflict with
* supporting hash tables that use alternative key storage, such as FixedHashMap
* or StringHashMap. Therefore, we implement an interface for ReverseIndex
* separately.
*/
template <typename Key, typename Cell, typename Hash>
class ReverseIndexHashTableBase : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
{
using State = typename Cell::State;
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
public:
using Base::Base;
using iterator = typename Base::iterator;
using LookupResult = typename Base::LookupResult;
State & getState() { return *this; }
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x,
size_t hash_value, size_t place_value) const
{
while (!this->buf[place_value].isZero(*this)
&& !this->buf[place_value].keyEquals(x, hash_value, *this))
{
place_value = this->grower.next(place_value);
}
return place_value;
}
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplaceNonZero(const Key & key, LookupResult & it,
bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
size_t place_value = reverseIndexFindCell(object, hash_value,
this->grower.place(hash_value));
// emplaceNonZeroImpl() might need to re-find the cell if the table grows,
// but it will find it correctly by the key alone, so we don't have to
// pass it the 'object'.
this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value);
}
/// Searches position by object.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted,
size_t hash_value, const ObjectToCompareWith& object)
{
LookupResult impl_it = nullptr;
if (!this->emplaceIfZero(key, impl_it, inserted, hash_value))
{
reverseIndexEmplaceNonZero(key, impl_it, inserted, hash_value, object);
}
assert(impl_it != nullptr);
it = iterator(this, impl_it);
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? this->iteratorToZero() : this->end();
size_t place_value = reverseIndexFindCell(x, hash_value,
this->grower.place(hash_value));
return !this->buf[place_value].isZero(*this)
? iterator(this, &this->buf[place_value])
: this->end();
}
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexStringHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexHash>;
public:
using Base::Base;
friend struct ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexHash>;
public:
using Base::Base;
friend struct ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index, bool is_numeric_column>
struct SelectReverseIndexHashTable;
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, true>
{
using Type = ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, false>
{
using Type = ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>;
};
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
}
};
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
struct ReverseIndexHashTableCell
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
{
using Base = HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>;
using State = typename Base::State;
using Base::Base;
using Base::isZero;
using Base::key;
using Base::keyEquals;
template <typename T>
constexpr bool isNumericColumn(const T *) { return false; }
static bool isZero(const T &, const State & /*state*/)
{
/// 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>);
return false;
}
template <typename T>
constexpr bool isNumericColumn(const ColumnVector<T> *) { return true; }
/// Special case when we want to compare with something not in index_column.
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
{
auto index = key;
if constexpr (has_base_index)
index -= state.base_index;
static_assert(isNumericColumn(static_cast<ColumnVector<UInt8> *>(nullptr)));
static_assert(!isNumericColumn(static_cast<ColumnString *>(nullptr)));
if constexpr (string_hash)
return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index);
else
return object == state.index_column->getDataAt(index);
}
size_t getHash(const Hash & hash) const
{
auto index = key;
/// Hack. HashTable is Hash itself.
const auto & state = static_cast<const State &>(static_cast<const HashTable &>(hash));
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return (*state.saved_hash_column)[index];
else
{
using ValueType = typename ColumnType::ValueType;
ValueType value = unalignedLoad<ValueType>(state.index_column->getDataAt(index).data);
return DefaultHash<ValueType>()(value);
}
}
};
template <typename IndexType, typename ColumnType, bool has_base_index>
using ReverseIndexHashTable = typename SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index,
isNumericColumn(static_cast<ColumnType *>(nullptr))>::Type;
/**
* ReverseIndexHashTableBase implements a special hash table interface for
* reverse index.
*
* The following requirements are different compared to a plain hash table:
*
* 1) Provide public access to 'hash table state' that contains
* additional data needed to calculate cell hashes.
*
* 2) Support emplace() and find() with a Key different from the resulting
* hash table key. This means emplace() accepts a different kind of object
* as a key, and then the real key can be read from the returned cell iterator.
*
* These requirements are unique to ReverseIndex and are in conflict with
* supporting hash tables that use alternative key storage, such as FixedHashMap
* or StringHashMap. Therefore, we implement an interface for ReverseIndex
* separately.
*/
template <typename Key, typename Cell, typename Hash>
class ReverseIndexHashTableBase : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
{
using State = typename Cell::State;
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
public:
using Base::Base;
using iterator = typename Base::iterator;
using LookupResult = typename Base::LookupResult;
State & getState() { return *this; }
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE reverseIndexFindCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const
{
while (!this->buf[place_value].isZero(*this) && !this->buf[place_value].keyEquals(x, hash_value, *this))
{
place_value = this->grower.next(place_value);
}
return place_value;
}
template <typename ObjectToCompareWith>
void ALWAYS_INLINE
reverseIndexEmplaceNonZero(const Key & key, LookupResult & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
size_t place_value = reverseIndexFindCell(object, hash_value, this->grower.place(hash_value));
// emplaceNonZeroImpl() might need to re-find the cell if the table grows,
// but it will find it correctly by the key alone, so we don't have to
// pass it the 'object'.
this->emplaceNonZeroImpl(place_value, key, it, inserted, hash_value);
}
/// Searches position by object.
template <typename ObjectToCompareWith>
void ALWAYS_INLINE reverseIndexEmplace(Key key, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
{
LookupResult impl_it = nullptr;
if (!this->emplaceIfZero(key, impl_it, inserted, hash_value))
{
reverseIndexEmplaceNonZero(key, impl_it, inserted, hash_value, object);
}
assert(impl_it != nullptr);
it = iterator(this, impl_it);
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE reverseIndexFind(ObjectToCompareWith x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? this->iteratorToZero() : this->end();
size_t place_value = reverseIndexFindCell(x, hash_value, this->grower.place(hash_value));
return !this->buf[place_value].isZero(*this) ? iterator(this, &this->buf[place_value]) : this->end();
}
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexStringHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexHash>;
public:
using Base::Base;
friend struct ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexNumberHashTable : public ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexHash>
{
using Base = ReverseIndexHashTableBase<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexHash>;
public:
using Base::Base;
friend struct ReverseIndexHashTableCell<
IndexType,
ReverseIndexHash,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index, bool is_numeric_column>
struct SelectReverseIndexHashTable;
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, true>
{
using Type = ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, false>
{
using Type = ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename T>
constexpr bool isNumericColumn(const T *)
{
return false;
}
template <typename T>
constexpr bool isNumericColumn(const ColumnVector<T> *)
{
return true;
}
static_assert(isNumericColumn(static_cast<ColumnVector<UInt8> *>(nullptr)));
static_assert(!isNumericColumn(static_cast<ColumnString *>(nullptr)));
template <typename IndexType, typename ColumnType, bool has_base_index>
using ReverseIndexHashTable =
typename SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, isNumericColumn(static_cast<ColumnType *>(nullptr))>::Type;
template <typename IndexType, typename ColumnType>
class ReverseIndex

View File

@ -21,9 +21,9 @@ private:
void dealloc();
public:
AlignedBuffer() {}
AlignedBuffer() = default;
AlignedBuffer(size_t size, size_t alignment);
AlignedBuffer(AlignedBuffer && old) { std::swap(buf, old.buf); }
AlignedBuffer(AlignedBuffer && old) noexcept { std::swap(buf, old.buf); }
~AlignedBuffer();
void reset(size_t size, size_t alignment);

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