Unification [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-09-15 15:16:12 +03:00
parent e29b0e1750
commit 86c46ad1bd
51 changed files with 409 additions and 409 deletions

View File

@ -8,7 +8,7 @@ namespace DB
namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, class Data, typename ... TArgs>
template <template <typename, typename> class AggregateFunctionTemplate, typename Data, typename ... TArgs>
static IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type, TArgs && ... args)
{
if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);

View File

@ -47,7 +47,7 @@ bool areComparable(UInt64 a, UInt64 b)
* Source: "Updating formulae and a pairwise algorithm for computing sample variances"
* (Chan et al., Stanford University, 12.1979)
*/
template<typename T, typename Op>
template <typename T, typename Op>
class AggregateFunctionVarianceData
{
public:
@ -109,7 +109,7 @@ private:
/** The main code for the implementation of varSamp, stddevSamp, varPop, stddevPop.
*/
template<typename T, typename Op>
template <typename T, typename Op>
class AggregateFunctionVariance final
: public IUnaryAggregateFunction<AggregateFunctionVarianceData<T, Op>,
AggregateFunctionVariance<T, Op>>
@ -214,7 +214,7 @@ struct AggregateFunctionStdDevPopImpl
/** If `compute_marginal_moments` flag is set this class provides the successor
* CovarianceData support of marginal moments for calculating the correlation.
*/
template<bool compute_marginal_moments>
template <bool compute_marginal_moments>
class BaseCovarianceData
{
protected:
@ -224,7 +224,7 @@ protected:
void deserialize(const ReadBuffer & buf) {}
};
template<>
template <>
class BaseCovarianceData<true>
{
protected:
@ -262,7 +262,7 @@ protected:
* (J. Bennett et al., Sandia National Laboratories,
* 2009 IEEE International Conference on Cluster Computing)
*/
template<typename T, typename U, typename Op, bool compute_marginal_moments>
template <typename T, typename U, typename Op, bool compute_marginal_moments>
class CovarianceData : public BaseCovarianceData<compute_marginal_moments>
{
private:
@ -348,13 +348,13 @@ public:
Base::deserialize(buf);
}
template<bool compute = compute_marginal_moments>
template <bool compute = compute_marginal_moments>
void publish(IColumn & to, typename std::enable_if<compute>::type * = nullptr) const
{
static_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count));
}
template<bool compute = compute_marginal_moments>
template <bool compute = compute_marginal_moments>
void publish(IColumn & to, typename std::enable_if<!compute>::type * = nullptr) const
{
static_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, count));
@ -367,7 +367,7 @@ private:
Float64 co_moment = 0.0;
};
template<typename T, typename U, typename Op, bool compute_marginal_moments = false>
template <typename T, typename U, typename Op, bool compute_marginal_moments = false>
class AggregateFunctionCovariance final
: public IBinaryAggregateFunction<
CovarianceData<T, U, Op, compute_marginal_moments>,
@ -465,25 +465,25 @@ struct AggregateFunctionCorrImpl
}
};
template<typename T>
template <typename T>
using AggregateFunctionVarSamp = AggregateFunctionVariance<T, AggregateFunctionVarSampImpl>;
template<typename T>
template <typename T>
using AggregateFunctionStdDevSamp = AggregateFunctionVariance<T, AggregateFunctionStdDevSampImpl>;
template<typename T>
template <typename T>
using AggregateFunctionVarPop = AggregateFunctionVariance<T, AggregateFunctionVarPopImpl>;
template<typename T>
template <typename T>
using AggregateFunctionStdDevPop = AggregateFunctionVariance<T, AggregateFunctionStdDevPopImpl>;
template<typename T, typename U>
template <typename T, typename U>
using AggregateFunctionCovarSamp = AggregateFunctionCovariance<T, U, AggregateFunctionCovarSampImpl>;
template<typename T, typename U>
template <typename T, typename U>
using AggregateFunctionCovarPop = AggregateFunctionCovariance<T, U, AggregateFunctionCovarPopImpl>;
template<typename T, typename U>
template <typename T, typename U>
using AggregateFunctionCorr = AggregateFunctionCovariance<T, U, AggregateFunctionCorrImpl, true>;
}

View File

@ -34,7 +34,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, class Data>
template <template <typename, typename> class AggregateFunctionTemplate, typename Data>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>;
@ -53,7 +53,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, class Data, typename ... TArgs>
template <template <typename, typename> class AggregateFunctionTemplate, typename Data, typename ... TArgs>
static IAggregateFunction * createWithNumericType(const IDataType & argument_type, TArgs && ... args)
{
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>(std::forward<TArgs>(args)...);

View File

@ -30,7 +30,7 @@ namespace ReservoirSamplerOnEmpty
};
}
template<typename ResultType, bool IsFloatingPoint>
template <typename ResultType, bool IsFloatingPoint>
struct NanLikeValueConstructor
{
static ResultType getValue()
@ -38,7 +38,7 @@ struct NanLikeValueConstructor
return std::numeric_limits<ResultType>::quiet_NaN();
}
};
template<typename ResultType>
template <typename ResultType>
struct NanLikeValueConstructor<ResultType, false>
{
static ResultType getValue()
@ -47,7 +47,7 @@ struct NanLikeValueConstructor<ResultType, false>
}
};
template<typename T, ReservoirSamplerOnEmpty::Enum OnEmpty = ReservoirSamplerOnEmpty::THROW, typename Comparer = std::less<T>>
template <typename T, ReservoirSamplerOnEmpty::Enum OnEmpty = ReservoirSamplerOnEmpty::THROW, typename Comparer = std::less<T>>
class ReservoirSampler
{
public:

View File

@ -287,13 +287,13 @@ private:
}
}
template<typename T>
template <typename T>
inline T & getContainer()
{
return *reinterpret_cast<T *>(address & mask);
}
template<typename T>
template <typename T>
inline const T & getContainer() const
{
return *reinterpret_cast<T *>(address & mask);

View File

@ -11,10 +11,10 @@
namespace detail
{
template <class T, bool is_nothrow_move_assignable = std::is_nothrow_move_assignable<T>::value>
template <typename T, bool is_nothrow_move_assignable = std::is_nothrow_move_assignable<T>::value>
struct MoveOrCopyIfThrow;
template <class T>
template <typename T>
struct MoveOrCopyIfThrow<T, true>
{
void operator()(T && src, T & dst) const
@ -23,7 +23,7 @@ namespace detail
}
};
template <class T>
template <typename T>
struct MoveOrCopyIfThrow<T, false>
{
void operator()(T && src, T & dst) const
@ -32,7 +32,7 @@ namespace detail
}
};
template <class T>
template <typename T>
void moveOrCopyIfThrow(T && src, T & dst)
{
MoveOrCopyIfThrow<T>()(std::forward<T>(src), dst);

View File

@ -36,7 +36,7 @@ namespace details
{
/// Look-up table of logarithms for integer numbers, used in HyperLogLogCounter.
template<UInt8 K>
template <UInt8 K>
struct LogLUT
{
LogLUT()
@ -60,15 +60,15 @@ private:
double log_table[M + 1];
};
template<UInt8 K> struct MinCounterTypeHelper;
template<> struct MinCounterTypeHelper<0> { using Type = UInt8; };
template<> struct MinCounterTypeHelper<1> { using Type = UInt16; };
template<> struct MinCounterTypeHelper<2> { using Type = UInt32; };
template<> struct MinCounterTypeHelper<3> { using Type = UInt64; };
template <UInt8 K> struct MinCounterTypeHelper;
template <> struct MinCounterTypeHelper<0> { using Type = UInt8; };
template <> struct MinCounterTypeHelper<1> { using Type = UInt16; };
template <> struct MinCounterTypeHelper<2> { using Type = UInt32; };
template <> struct MinCounterTypeHelper<3> { using Type = UInt64; };
/// Auxiliary structure for automatic determining minimum size of counter's type depending on its maximum value.
/// Used in HyperLogLogCounter in order to spend memory efficiently.
template<UInt64 MaxValue> struct MinCounterType
template <UInt64 MaxValue> struct MinCounterType
{
using Type = typename MinCounterTypeHelper<
(MaxValue >= 1 << 8) +
@ -78,7 +78,7 @@ template<UInt64 MaxValue> struct MinCounterType
};
/// Denominator of expression for HyperLogLog algorithm.
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode, typename Enable = void>
class __attribute__ ((packed)) Denominator;
@ -118,7 +118,7 @@ struct IntermediateDenominator<HashValueType, DenominatorType, DenominatorMode::
/// "Lightweight" implementation of expression's denominator for HyperLogLog algorithm.
/// Uses minimum amount of memory, but estimates may be unstable.
/// Satisfiable when rank storage is small enough.
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode>
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
denominator_mode,
@ -162,7 +162,7 @@ private:
/// Fully-functional version of expression's denominator for HyperLogLog algorithm.
/// Spends more space that lightweight version. Estimates will always be stable.
/// Used when rank storage is big.
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
template <UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
DenominatorMode denominator_mode>
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
denominator_mode,

View File

@ -137,7 +137,7 @@ protected:
Logger * log;
};
template<typename TNestedPool>
template <typename TNestedPool>
typename TNestedPool::Entry
PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority)
{
@ -149,7 +149,7 @@ PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, co
return results[0].entry;
}
template<typename TNestedPool>
template <typename TNestedPool>
std::vector<typename PoolWithFailoverBase<TNestedPool>::TryResult>
PoolWithFailoverBase<TNestedPool>::getMany(
size_t min_entries, size_t max_entries,
@ -293,7 +293,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
return try_results;
}
template<typename TNestedPool>
template <typename TNestedPool>
void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
{
for (size_t i = 0; i < nested_pools.size(); ++i)
@ -308,7 +308,7 @@ void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
throw DB::Exception("Can't find pool to report error.");
}
template<typename TNestedPool>
template <typename TNestedPool>
struct PoolWithFailoverBase<TNestedPool>::PoolState
{
UInt64 error_count = 0;
@ -330,7 +330,7 @@ private:
std::minstd_rand rng = std::minstd_rand(randomSeed());
};
template<typename TNestedPool>
template <typename TNestedPool>
typename PoolWithFailoverBase<TNestedPool>::PoolStates
PoolWithFailoverBase<TNestedPool>::updatePoolStates()
{

View File

@ -50,7 +50,7 @@ struct UInt128
template <typename T> bool inline operator<= (const T rhs) const { return *this <= UInt128(rhs); }
template <typename T> bool inline operator< (const T rhs) const { return *this < UInt128(rhs); }
template<typename T> explicit operator T() const { return static_cast<T>(low); }
template <typename T> explicit operator T() const { return static_cast<T>(low); }
#if !__clang__
#pragma GCC diagnostic pop

View File

@ -19,7 +19,7 @@ public:
ZooKeeperHolder() = default;
/// вызывать из одного потока - не thread safe
template <class... Args>
template <typename... Args>
void init(Args&&... args);
/// был ли класс инициализирован
bool isInitialized() const { return ptr != nullptr; }
@ -75,7 +75,7 @@ private:
static std::string nullptr_exception_message;
};
template <class... Args>
template <typename... Args>
void ZooKeeperHolder::init(Args&&... args)
{
ptr = std::make_shared<ZooKeeper>(std::forward<Args>(args)...);

View File

@ -38,7 +38,7 @@ inline uint32_t getbit ( const void * block, int len, uint32_t bit )
return 0;
}
template< typename T >
template < typename T >
inline uint32_t getbit ( T & blob, uint32_t bit )
{
return getbit(&blob,sizeof(blob),bit);
@ -54,7 +54,7 @@ inline void flipbit ( void * block, int len, uint32_t bit )
if(byte < len) b[byte] ^= (1 << bit);
}
template< typename T >
template < typename T >
inline void flipbit ( T & blob, uint32_t bit )
{
flipbit(&blob,sizeof(blob),bit);
@ -146,7 +146,7 @@ bool AvalancheTest ( pfHash hash, const int reps )
// BIC test variant - store all intermediate data in a table, draw diagram
// afterwards (much faster)
template< typename keytype, typename hashtype >
template < typename keytype, typename hashtype >
void BicTest3 ( pfHash hash, const int reps, bool verbose = true )
{
const int keybytes = sizeof(keytype);

View File

@ -146,7 +146,7 @@ private:
* Grow the map by GrowthFactor::num/GrowthFactor::den and use a modulo to map a hash
* to a bucket. Slower but it can be usefull if you want a slower growth.
*/
template<class GrowthFactor = std::ratio<3, 2>>
template <typename GrowthFactor = std::ratio<3, 2>>
class mod_growth_policy {
public:
mod_growth_policy(std::size_t& min_bucket_count_in_out) {
@ -212,7 +212,7 @@ static constexpr const std::array<std::size_t, 39> PRIMES = {{
1610612741ul, 3221225473ul, 4294967291ul
}};
template<unsigned int IPrime>
template <unsigned int IPrime>
static std::size_t mod(std::size_t hash) { return hash % PRIMES[IPrime]; }
// MOD_PRIME[iprime](hash) returns hash % PRIMES[iprime]. This table allows for faster modulo as the
@ -274,26 +274,26 @@ namespace detail_hopscotch_hash {
template<typename T>
template <typename T>
struct make_void {
using type = void;
};
template<typename T, typename = void>
template <typename T, typename = void>
struct has_is_transparent : std::false_type {
};
template<typename T>
template <typename T>
struct has_is_transparent<T, typename make_void<typename T::is_transparent>::type> : std::true_type {
};
template<typename T, typename = void>
template <typename T, typename = void>
struct has_key_compare : std::false_type {
};
template<typename T>
template <typename T>
struct has_key_compare<T, typename make_void<typename T::key_compare>::type> : std::true_type {
};
@ -305,29 +305,29 @@ struct has_key_compare<T, typename make_void<typename T::key_compare>::type> : s
* smallest_type_for_min_bits::type returns the smallest type that can fit MinBits.
*/
static const size_t SMALLEST_TYPE_MAX_BITS_SUPPORTED = 64;
template<unsigned int MinBits, typename Enable = void>
template <unsigned int MinBits, typename Enable = void>
class smallest_type_for_min_bits {
};
template<unsigned int MinBits>
template <unsigned int MinBits>
class smallest_type_for_min_bits<MinBits, typename std::enable_if<(MinBits > 0) && (MinBits <= 8)>::type> {
public:
using type = std::uint_least8_t;
};
template<unsigned int MinBits>
template <unsigned int MinBits>
class smallest_type_for_min_bits<MinBits, typename std::enable_if<(MinBits > 8) && (MinBits <= 16)>::type> {
public:
using type = std::uint_least16_t;
};
template<unsigned int MinBits>
template <unsigned int MinBits>
class smallest_type_for_min_bits<MinBits, typename std::enable_if<(MinBits > 16) && (MinBits <= 32)>::type> {
public:
using type = std::uint_least32_t;
};
template<unsigned int MinBits>
template <unsigned int MinBits>
class smallest_type_for_min_bits<MinBits, typename std::enable_if<(MinBits > 32) && (MinBits <= 64)>::type> {
public:
using type = std::uint_least64_t;
@ -353,7 +353,7 @@ public:
static const std::size_t NB_RESERVED_BITS_IN_NEIGHBORHOOD = 2;
template<bool StoreHash>
template <bool StoreHash>
class hopscotch_bucket_hash {
public:
using hash_type = std::false_type;
@ -375,7 +375,7 @@ protected:
}
};
template<>
template <>
class hopscotch_bucket_hash<true> {
public:
using hash_type = std::uint_least32_t;
@ -402,7 +402,7 @@ private:
hash_type m_hash;
};
template<typename ValueType, unsigned int NeighborhoodSize, bool StoreHash>
template <typename ValueType, unsigned int NeighborhoodSize, bool StoreHash>
class hopscotch_bucket: public hopscotch_bucket_hash<StoreHash> {
private:
static const size_t MIN_NEIGHBORHOOD_SIZE = 4;
@ -529,7 +529,7 @@ public:
return *reinterpret_cast<const value_type*>(std::addressof(m_value));
}
template<typename... Args>
template <typename... Args>
void set_value_of_empty_bucket(std::size_t hash, Args&&... value_type_args) {
tsl_assert(empty());
@ -617,7 +617,7 @@ private:
* OverflowContainer will be used as containers for overflown elements. Usually it should be a list<ValueType>
* or a set<Key>/map<Key, T>.
*/
template<class ValueType,
template <typename ValueType,
class KeySelect,
class ValueSelect,
class Hash,
@ -629,11 +629,11 @@ template<class ValueType,
class OverflowContainer>
class hopscotch_hash: private Hash, private KeyEqual, private GrowthPolicy {
private:
template<typename U>
template <typename U>
using has_mapped_type = typename std::integral_constant<bool, !std::is_same<U, void>::value>;
public:
template<bool is_const>
template <bool is_const>
class hopscotch_iterator;
using key_type = typename KeySelect::key_type;
@ -680,7 +680,7 @@ public:
* In case of a map, to get a modifiable reference to the value associated to a key (the '.second' in the
* stored pair), you have to call 'value()'.
*/
template<bool is_const>
template <bool is_const>
class hopscotch_iterator {
friend class hopscotch_hash;
private:
@ -724,7 +724,7 @@ public:
return KeySelect()(*m_overflow_iterator);
}
template<class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
typename std::conditional<
is_const,
const typename U::value_type&,
@ -791,7 +791,7 @@ public:
public:
template<class OC = OverflowContainer, typename std::enable_if<!has_key_compare<OC>::value>::type* = nullptr>
template <typename OC = OverflowContainer, typename std::enable_if<!has_key_compare<OC>::value>::type* = nullptr>
hopscotch_hash(size_type bucket_count,
const Hash& hash,
const KeyEqual& equal,
@ -814,7 +814,7 @@ public:
this->max_load_factor(max_load_factor);
}
template<class OC = OverflowContainer, typename std::enable_if<has_key_compare<OC>::value>::type* = nullptr>
template <typename OC = OverflowContainer, typename std::enable_if<has_key_compare<OC>::value>::type* = nullptr>
hopscotch_hash(size_type bucket_count,
const Hash& hash,
const KeyEqual& equal,
@ -949,7 +949,7 @@ public:
return insert_impl(value);
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
std::pair<iterator, bool> insert(P&& value) {
return emplace(std::forward<P>(value));
}
@ -967,7 +967,7 @@ public:
return insert(value).first;
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
iterator insert(const_iterator hint, P&& value) {
return emplace_hint(hint, std::forward<P>(value));
}
@ -981,7 +981,7 @@ public:
}
template<class InputIt>
template <typename InputIt>
void insert(InputIt first, InputIt last) {
if(std::is_base_of<std::forward_iterator_tag,
typename std::iterator_traits<InputIt>::iterator_category>::value)
@ -1003,18 +1003,18 @@ public:
}
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(const key_type& k, M&& obj) {
return insert_or_assign_impl(k, std::forward<M>(obj));
}
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(key_type&& k, M&& obj) {
return insert_or_assign_impl(std::move(k), std::forward<M>(obj));
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, const key_type& k, M&& obj) {
if(hint != cend() && compare_keys(KeySelect()(*hint), k)) {
auto it = mutable_iterator(hint);
@ -1026,7 +1026,7 @@ public:
return insert_or_assign(k, std::forward<M>(obj)).first;
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, key_type&& k, M&& obj) {
if(hint != cend() && compare_keys(KeySelect()(*hint), k)) {
auto it = mutable_iterator(hint);
@ -1039,27 +1039,27 @@ public:
}
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> emplace(Args&&... args) {
return insert(value_type(std::forward<Args>(args)...));
}
template<class... Args>
template <typename... Args>
iterator emplace_hint(const_iterator hint, Args&&... args) {
return insert(hint, value_type(std::forward<Args>(args)...));
}
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(const key_type& k, Args&&... args) {
return try_emplace_impl(k, std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(key_type&& k, Args&&... args) {
return try_emplace_impl(std::move(k), std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, const key_type& k, Args&&... args) {
if(hint != cend() && compare_keys(KeySelect()(*hint), k)) {
return mutable_iterator(hint);
@ -1068,7 +1068,7 @@ public:
return try_emplace(k, std::forward<Args>(args)...).first;
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, key_type&& k, Args&&... args) {
if(hint != cend() && compare_keys(KeySelect()(*hint), k)) {
return mutable_iterator(hint);
@ -1111,12 +1111,12 @@ public:
return to_delete;
}
template<class K>
template <typename K>
size_type erase(const K& key) {
return erase(key, hash_key(key));
}
template<class K>
template <typename K>
size_type erase(const K& key, std::size_t hash) {
const std::size_t ibucket_for_hash = bucket_for_hash(hash);
@ -1157,23 +1157,23 @@ public:
/*
* Lookup
*/
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
typename U::value_type& at(const K& key) {
return at(key, hash_key(key));
}
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
typename U::value_type& at(const K& key, std::size_t hash) {
return const_cast<typename U::value_type&>(static_cast<const hopscotch_hash*>(this)->at(key, hash));
}
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
const typename U::value_type& at(const K& key) const {
return at(key, hash_key(key));
}
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
const typename U::value_type& at(const K& key, std::size_t hash) const {
using T = typename U::value_type;
@ -1187,7 +1187,7 @@ public:
}
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
typename U::value_type& operator[](K&& key) {
using T = typename U::value_type;
@ -1206,57 +1206,57 @@ public:
}
template<class K>
template <typename K>
size_type count(const K& key) const {
return count(key, hash_key(key));
}
template<class K>
template <typename K>
size_type count(const K& key, std::size_t hash) const {
return count_impl(key, hash, m_buckets.cbegin() + bucket_for_hash(hash));
}
template<class K>
template <typename K>
iterator find(const K& key) {
return find(key, hash_key(key));
}
template<class K>
template <typename K>
iterator find(const K& key, std::size_t hash) {
return find_impl(key, hash, m_buckets.begin() + bucket_for_hash(hash));
}
template<class K>
template <typename K>
const_iterator find(const K& key) const {
return find(key, hash_key(key));
}
template<class K>
template <typename K>
const_iterator find(const K& key, std::size_t hash) const {
return find_impl(key, hash, m_buckets.begin() + bucket_for_hash(hash));
}
template<class K>
template <typename K>
std::pair<iterator, iterator> equal_range(const K& key) {
return equal_range(key, hash_key(key));
}
template<class K>
template <typename K>
std::pair<iterator, iterator> equal_range(const K& key, std::size_t hash) {
iterator it = find(key, hash);
return std::make_pair(it, (it == end())?it:std::next(it));
}
template<class K>
template <typename K>
std::pair<const_iterator, const_iterator> equal_range(const K& key) const {
return equal_range(key, hash_key(key));
}
template<class K>
template <typename K>
std::pair<const_iterator, const_iterator> equal_range(const K& key, std::size_t hash) const {
const_iterator it = find(key, hash);
return std::make_pair(it, (it == cend())?it:std::next(it));
@ -1339,19 +1339,19 @@ public:
return m_overflow_elements.size();
}
template<class U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
template <typename U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
typename U::key_compare key_comp() const {
return m_overflow_elements.key_comp();
}
private:
template<class K>
template <typename K>
std::size_t hash_key(const K& key) const {
return Hash::operator()(key);
}
template<class K1, class K2>
template <typename K1, typename K2>
bool compare_keys(const K1& key1, const K2& key2) const {
return KeyEqual::operator()(key1, key2);
}
@ -1365,7 +1365,7 @@ private:
std::is_copy_constructible<value_type>::value,
"value_type must be either copy constructible or nothrow move constructible.");
template<typename U = value_type,
template <typename U = value_type,
typename std::enable_if<std::is_nothrow_move_constructible<U>::value>::type* = nullptr>
void rehash_impl(size_type count) {
hopscotch_hash new_map = new_hopscotch_hash(count);
@ -1425,7 +1425,7 @@ private:
new_map.swap(*this);
}
template<typename U = value_type,
template <typename U = value_type,
typename std::enable_if<std::is_copy_constructible<U>::value &&
!std::is_nothrow_move_constructible<U>::value>::type* = nullptr>
void rehash_impl(size_type count) {
@ -1499,7 +1499,7 @@ private:
template<class K, class M>
template <typename K, typename M>
std::pair<iterator, bool> insert_or_assign_impl(K&& key, M&& obj) {
auto it = try_emplace_impl(std::forward<K>(key), std::forward<M>(obj));
if(!it.second) {
@ -1509,7 +1509,7 @@ private:
return it;
}
template<typename P, class... Args>
template <typename P, typename... Args>
std::pair<iterator, bool> try_emplace_impl(P&& key, Args&&... args_value) {
const std::size_t hash = hash_key(key);
const std::size_t ibucket_for_hash = bucket_for_hash(hash);
@ -1525,7 +1525,7 @@ private:
std::forward_as_tuple(std::forward<Args>(args_value)...));
}
template<typename P>
template <typename P>
std::pair<iterator, bool> insert_impl(P&& value) {
const std::size_t hash = hash_key(KeySelect()(value));
const std::size_t ibucket_for_hash = bucket_for_hash(hash);
@ -1540,7 +1540,7 @@ private:
return insert_impl(ibucket_for_hash, hash, std::forward<P>(value));
}
template<typename... Args>
template <typename... Args>
std::pair<iterator, bool> insert_impl(std::size_t ibucket_for_hash, std::size_t hash, Args&&... value_type_args) {
if((m_nb_elements - m_overflow_elements.size()) >= m_load_threshold) {
rehash(GrowthPolicy::next_bucket_count());
@ -1624,7 +1624,7 @@ private:
*
* Return bucket iterator to ibucket_empty
*/
template<typename... Args>
template <typename... Args>
iterator_buckets insert_in_bucket(std::size_t ibucket_empty, std::size_t ibucket_for_hash,
std::size_t hash, Args&&... value_type_args)
{
@ -1682,7 +1682,7 @@ private:
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
typename U::value_type* find_value_impl(const K& key, std::size_t hash, iterator_buckets it_bucket) {
return const_cast<typename U::value_type*>(
static_cast<const hopscotch_hash*>(this)->find_value_impl(key, hash, it_bucket));
@ -1693,7 +1693,7 @@ private:
*
* Return null if no value for key (TODO use std::optional when available).
*/
template<class K, class U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
template <typename K, typename U = ValueSelect, typename std::enable_if<has_mapped_type<U>::value>::type* = nullptr>
const typename U::value_type* find_value_impl(const K& key, std::size_t hash,
const_iterator_buckets it_bucket) const
{
@ -1712,7 +1712,7 @@ private:
return nullptr;
}
template<class K>
template <typename K>
size_type count_impl(const K& key, std::size_t hash, const_iterator_buckets it_bucket) const {
if(find_in_buckets(key, hash, it_bucket) != m_buckets.cend()) {
return 1;
@ -1725,7 +1725,7 @@ private:
}
}
template<class K>
template <typename K>
iterator find_impl(const K& key, std::size_t hash, iterator_buckets it_bucket) {
auto it = find_in_buckets(key, hash, it_bucket);
if(it != m_buckets.end()) {
@ -1739,7 +1739,7 @@ private:
return iterator(m_buckets.end(), m_buckets.end(), find_in_overflow(key));
}
template<class K>
template <typename K>
const_iterator find_impl(const K& key, std::size_t hash, const_iterator_buckets it_bucket) const {
auto it = find_in_buckets(key, hash, it_bucket);
if(it != m_buckets.cend()) {
@ -1754,14 +1754,14 @@ private:
return const_iterator(m_buckets.cend(), m_buckets.cend(), find_in_overflow(key));
}
template<class K>
template <typename K>
iterator_buckets find_in_buckets(const K& key, std::size_t hash, iterator_buckets it_bucket) {
auto it_find = static_cast<const hopscotch_hash*>(this)->find_in_buckets(key, hash, it_bucket);
return m_buckets.begin() + std::distance(m_buckets.cbegin(), it_find);
}
template<class K>
template <typename K>
const_iterator_buckets find_in_buckets(const K& key, std::size_t hash, const_iterator_buckets it_bucket) const {
(void) hash; // Avoid warning of unused variable when StoreHash is false;
@ -1791,7 +1791,7 @@ private:
template<class K, class U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
template <typename K, typename U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
iterator_overflow find_in_overflow(const K& key) {
return std::find_if(m_overflow_elements.begin(), m_overflow_elements.end(),
[&](const value_type& value) {
@ -1799,7 +1799,7 @@ private:
});
}
template<class K, class U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
template <typename K, typename U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
const_iterator_overflow find_in_overflow(const K& key) const {
return std::find_if(m_overflow_elements.cbegin(), m_overflow_elements.cend(),
[&](const value_type& value) {
@ -1807,37 +1807,37 @@ private:
});
}
template<class K, class U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
template <typename K, typename U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
iterator_overflow find_in_overflow(const K& key) {
return m_overflow_elements.find(key);
}
template<class K, class U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
template <typename K, typename U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
const_iterator_overflow find_in_overflow(const K& key) const {
return m_overflow_elements.find(key);
}
template<class... Args, class U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
template <typename... Args, typename U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
iterator_overflow insert_in_overflow(Args&&... value_type_args) {
return m_overflow_elements.emplace(m_overflow_elements.end(), std::forward<Args>(value_type_args)...);
}
template<class... Args, class U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
template <typename... Args, typename U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
iterator_overflow insert_in_overflow(Args&&... value_type_args) {
return m_overflow_elements.emplace(std::forward<Args>(value_type_args)...).first;
}
template<class U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
template <typename U = OverflowContainer, typename std::enable_if<!has_key_compare<U>::value>::type* = nullptr>
hopscotch_hash new_hopscotch_hash(size_type bucket_count) {
return hopscotch_hash(bucket_count, static_cast<Hash&>(*this), static_cast<KeyEqual&>(*this),
get_allocator(), m_max_load_factor);
}
template<class U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
template <typename U = OverflowContainer, typename std::enable_if<has_key_compare<U>::value>::type* = nullptr>
hopscotch_hash new_hopscotch_hash(size_type bucket_count) {
return hopscotch_hash(bucket_count, static_cast<Hash&>(*this), static_cast<KeyEqual&>(*this),
get_allocator(), m_max_load_factor, m_overflow_elements.key_comp());

View File

@ -68,7 +68,7 @@ namespace tsl {
* insert will invalidate the iterators). Or if there is a rehash.
* - erase: iterator on the erased element is the only one which become invalid.
*/
template<class Key,
template <typename Key,
class T,
class Hash = std::hash<Key>,
class KeyEqual = std::equal_to<Key>,
@ -78,7 +78,7 @@ template<class Key,
class GrowthPolicy = tsl::power_of_two_growth_policy>
class hopscotch_map {
private:
template<typename U>
template <typename U>
using has_is_transparent = tsl::detail_hopscotch_hash::has_is_transparent<U>;
class KeySelect {
@ -161,7 +161,7 @@ public:
explicit hopscotch_map(const Allocator& alloc) : hopscotch_map(ht::DEFAULT_INIT_BUCKETS_SIZE, alloc) {
}
template<class InputIt>
template <typename InputIt>
hopscotch_map(InputIt first, InputIt last,
size_type bucket_count = ht::DEFAULT_INIT_BUCKETS_SIZE,
const Hash& hash = Hash(),
@ -171,14 +171,14 @@ public:
insert(first, last);
}
template<class InputIt>
template <typename InputIt>
hopscotch_map(InputIt first, InputIt last,
size_type bucket_count,
const Allocator& alloc) : hopscotch_map(first, last, bucket_count, Hash(), KeyEqual(), alloc)
{
}
template<class InputIt>
template <typename InputIt>
hopscotch_map(InputIt first, InputIt last,
size_type bucket_count,
const Hash& hash,
@ -254,7 +254,7 @@ public:
return m_ht.insert(value);
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
std::pair<iterator, bool> insert(P&& value) {
return m_ht.insert(std::forward<P>(value));
}
@ -268,7 +268,7 @@ public:
return m_ht.insert(hint, value);
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
iterator insert(const_iterator hint, P&& value) {
return m_ht.insert(hint, std::forward<P>(value));
}
@ -278,7 +278,7 @@ public:
}
template<class InputIt>
template <typename InputIt>
void insert(InputIt first, InputIt last) {
m_ht.insert(first, last);
}
@ -290,22 +290,22 @@ public:
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(const key_type& k, M&& obj) {
return m_ht.insert_or_assign(k, std::forward<M>(obj));
}
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(key_type&& k, M&& obj) {
return m_ht.insert_or_assign(std::move(k), std::forward<M>(obj));
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, const key_type& k, M&& obj) {
return m_ht.insert_or_assign(hint, k, std::forward<M>(obj));
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, key_type&& k, M&& obj) {
return m_ht.insert_or_assign(hint, std::move(k), std::forward<M>(obj));
}
@ -319,7 +319,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> emplace(Args&&... args) {
return m_ht.emplace(std::forward<Args>(args)...);
}
@ -333,7 +333,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
iterator emplace_hint(const_iterator hint, Args&&... args) {
return m_ht.emplace_hint(hint, std::forward<Args>(args)...);
}
@ -341,22 +341,22 @@ public:
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(const key_type& k, Args&&... args) {
return m_ht.try_emplace(k, std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(key_type&& k, Args&&... args) {
return m_ht.try_emplace(std::move(k), std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, const key_type& k, Args&&... args) {
return m_ht.try_emplace(hint, k, std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, key_type&& k, Args&&... args) {
return m_ht.try_emplace(hint, std::move(k), std::forward<Args>(args)...);
}
@ -381,7 +381,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type erase(const K& key) { return m_ht.erase(key); }
/**
@ -390,7 +390,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup to the value if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type erase(const K& key, std::size_t precalculated_hash) {
return m_ht.erase(key, precalculated_hash);
}
@ -424,7 +424,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
T& at(const K& key) { return m_ht.at(key); }
/**
@ -433,20 +433,20 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
T& at(const K& key, std::size_t precalculated_hash) { return m_ht.at(key, precalculated_hash); }
/**
* @copydoc at(const K& key)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const T& at(const K& key) const { return m_ht.at(key); }
/**
* @copydoc at(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const T& at(const K& key, std::size_t precalculated_hash) const { return m_ht.at(key, precalculated_hash); }
@ -472,7 +472,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type count(const K& key) const { return m_ht.count(key); }
/**
@ -481,7 +481,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type count(const K& key, std::size_t precalculated_hash) const { return m_ht.count(key, precalculated_hash); }
@ -508,7 +508,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
iterator find(const K& key) { return m_ht.find(key); }
/**
@ -517,13 +517,13 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
iterator find(const K& key, std::size_t precalculated_hash) { return m_ht.find(key, precalculated_hash); }
/**
* @copydoc find(const K& key)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const_iterator find(const K& key) const { return m_ht.find(key); }
/**
@ -532,7 +532,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const_iterator find(const K& key, std::size_t precalculated_hash) const {
return m_ht.find(key, precalculated_hash);
}
@ -563,7 +563,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key) { return m_ht.equal_range(key); }
@ -573,7 +573,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key, std::size_t precalculated_hash) {
return m_ht.equal_range(key, precalculated_hash);
}
@ -581,13 +581,13 @@ public:
/**
* @copydoc equal_range(const K& key)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key) const { return m_ht.equal_range(key); }
/**
* @copydoc equal_range(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key, std::size_t precalculated_hash) const {
return m_ht.equal_range(key, precalculated_hash);
}

View File

@ -53,7 +53,7 @@ namespace tsl {
*
* @copydoc hopscotch_map
*/
template<class Key,
template <typename Key,
class T,
class Hash = std::hash<Key>,
class KeyEqual = std::equal_to<Key>,
@ -64,7 +64,7 @@ template<class Key,
class GrowthPolicy = tsl::power_of_two_growth_policy>
class hopscotch_sc_map {
private:
template<typename U>
template <typename U>
using has_is_transparent = tsl::detail_hopscotch_hash::has_is_transparent<U>;
class KeySelect {
@ -150,7 +150,7 @@ public:
explicit hopscotch_sc_map(const Allocator& alloc) : hopscotch_sc_map(ht::DEFAULT_INIT_BUCKETS_SIZE, alloc) {
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_map(InputIt first, InputIt last,
size_type bucket_count = ht::DEFAULT_INIT_BUCKETS_SIZE,
const Hash& hash = Hash(),
@ -160,14 +160,14 @@ public:
insert(first, last);
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_map(InputIt first, InputIt last,
size_type bucket_count,
const Allocator& alloc) : hopscotch_sc_map(first, last, bucket_count, Hash(), KeyEqual(), alloc)
{
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_map(InputIt first, InputIt last,
size_type bucket_count,
const Hash& hash,
@ -243,7 +243,7 @@ public:
return m_ht.insert(value);
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
std::pair<iterator, bool> insert(P&& value) {
return m_ht.insert(std::forward<P>(value));
}
@ -257,7 +257,7 @@ public:
return m_ht.insert(hint, value);
}
template<class P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
template <typename P, typename std::enable_if<std::is_constructible<value_type, P&&>::value>::type* = nullptr>
iterator insert(const_iterator hint, P&& value) {
return m_ht.insert(hint, std::forward<P>(value));
}
@ -267,7 +267,7 @@ public:
}
template<class InputIt>
template <typename InputIt>
void insert(InputIt first, InputIt last) {
m_ht.insert(first, last);
}
@ -279,22 +279,22 @@ public:
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(const key_type& k, M&& obj) {
return m_ht.insert_or_assign(k, std::forward<M>(obj));
}
template<class M>
template <typename M>
std::pair<iterator, bool> insert_or_assign(key_type&& k, M&& obj) {
return m_ht.insert_or_assign(std::move(k), std::forward<M>(obj));
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, const key_type& k, M&& obj) {
return m_ht.insert_or_assign(hint, k, std::forward<M>(obj));
}
template<class M>
template <typename M>
iterator insert_or_assign(const_iterator hint, key_type&& k, M&& obj) {
return m_ht.insert_or_assign(hint, std::move(k), std::forward<M>(obj));
}
@ -307,7 +307,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> emplace(Args&&... args) {
return m_ht.emplace(std::forward<Args>(args)...);
}
@ -321,7 +321,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
iterator emplace_hint(const_iterator hint, Args&&... args) {
return m_ht.emplace_hint(hint, std::forward<Args>(args)...);
}
@ -329,22 +329,22 @@ public:
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(const key_type& k, Args&&... args) {
return m_ht.try_emplace(k, std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> try_emplace(key_type&& k, Args&&... args) {
return m_ht.try_emplace(std::move(k), std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, const key_type& k, Args&&... args) {
return m_ht.try_emplace(hint, k, std::forward<Args>(args)...);
}
template<class... Args>
template <typename... Args>
iterator try_emplace(const_iterator hint, key_type&& k, Args&&... args) {
return m_ht.try_emplace(hint, std::move(k), std::forward<Args>(args)...);
}
@ -370,7 +370,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type erase(const K& key) { return m_ht.erase(key); }
@ -380,7 +380,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup to the value if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type erase(const K& key, std::size_t precalculated_hash) { return m_ht.erase(key, precalculated_hash); }
@ -412,7 +412,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
T& at(const K& key) { return m_ht.at(key); }
@ -422,21 +422,21 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
T& at(const K& key, std::size_t precalculated_hash) { return m_ht.at(key, precalculated_hash); }
/**
* @copydoc at(const K& key)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const T& at(const K& key) const { return m_ht.at(key); }
/**
* @copydoc at(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const T& at(const K& key, std::size_t precalculated_hash) const { return m_ht.at(key, precalculated_hash); }
@ -462,7 +462,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type count(const K& key) const { return m_ht.count(key); }
@ -472,7 +472,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type count(const K& key, std::size_t precalculated_hash) const { return m_ht.count(key, precalculated_hash); }
@ -499,7 +499,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
iterator find(const K& key) { return m_ht.find(key); }
@ -509,21 +509,21 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
iterator find(const K& key, std::size_t precalculated_hash) { return m_ht.find(key, precalculated_hash); }
/**
* @copydoc find(const K& key)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const_iterator find(const K& key) const { return m_ht.find(key); }
/**
* @copydoc find(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const_iterator find(const K& key, std::size_t precalculated_hash) const { return m_ht.find(key, precalculated_hash); }
@ -554,7 +554,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key) { return m_ht.equal_range(key); }
@ -564,7 +564,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key, std::size_t precalculated_hash) {
return m_ht.equal_range(key, precalculated_hash);
@ -573,14 +573,14 @@ public:
/**
* @copydoc equal_range(const K& key)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key) const { return m_ht.equal_range(key); }
/**
* @copydoc equal_range(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key, std::size_t precalculated_hash) const {
return m_ht.equal_range(key, precalculated_hash);

View File

@ -53,7 +53,7 @@ namespace tsl {
*
* @copydoc hopscotch_set
*/
template<class Key,
template <typename Key,
class Hash = std::hash<Key>,
class KeyEqual = std::equal_to<Key>,
class Compare = std::less<Key>,
@ -63,7 +63,7 @@ template<class Key,
class GrowthPolicy = tsl::power_of_two_growth_policy>
class hopscotch_sc_set {
private:
template<typename U>
template <typename U>
using has_is_transparent = tsl::detail_hopscotch_hash::has_is_transparent<U>;
class KeySelect {
@ -133,7 +133,7 @@ public:
explicit hopscotch_sc_set(const Allocator& alloc) : hopscotch_sc_set(ht::DEFAULT_INIT_BUCKETS_SIZE, alloc) {
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_set(InputIt first, InputIt last,
size_type bucket_count = ht::DEFAULT_INIT_BUCKETS_SIZE,
const Hash& hash = Hash(),
@ -143,14 +143,14 @@ public:
insert(first, last);
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_set(InputIt first, InputIt last,
size_type bucket_count,
const Allocator& alloc) : hopscotch_sc_set(first, last, bucket_count, Hash(), KeyEqual(), alloc)
{
}
template<class InputIt>
template <typename InputIt>
hopscotch_sc_set(InputIt first, InputIt last,
size_type bucket_count,
const Hash& hash,
@ -228,7 +228,7 @@ public:
iterator insert(const_iterator hint, const value_type& value) { return m_ht.insert(hint, value); }
iterator insert(const_iterator hint, value_type&& value) { return m_ht.insert(hint, std::move(value)); }
template<class InputIt>
template <typename InputIt>
void insert(InputIt first, InputIt last) { m_ht.insert(first, last); }
void insert(std::initializer_list<value_type> ilist) { m_ht.insert(ilist.begin(), ilist.end()); }
@ -241,7 +241,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> emplace(Args&&... args) { return m_ht.emplace(std::forward<Args>(args)...); }
@ -253,7 +253,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
iterator emplace_hint(const_iterator hint, Args&&... args) {
return m_ht.emplace_hint(hint, std::forward<Args>(args)...);
}
@ -279,7 +279,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type erase(const K& key) { return m_ht.erase(key); }
@ -289,7 +289,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup to the value if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type erase(const K& key, std::size_t precalculated_hash) { return m_ht.erase(key, precalculated_hash); }
@ -315,7 +315,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type count(const K& key) const { return m_ht.count(key); }
@ -325,7 +325,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
size_type count(const K& key, std::size_t precalculated_hash) const { return m_ht.count(key, precalculated_hash); }
@ -352,7 +352,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
iterator find(const K& key) { return m_ht.find(key); }
@ -362,14 +362,14 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
iterator find(const K& key, std::size_t precalculated_hash) { return m_ht.find(key, precalculated_hash); }
/**
* @copydoc find(const K& key)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const_iterator find(const K& key) const { return m_ht.find(key); }
@ -379,7 +379,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
const_iterator find(const K& key, std::size_t precalculated_hash) const { return m_ht.find(key, precalculated_hash); }
@ -410,7 +410,7 @@ public:
* and Compare::is_transparent exist.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key) { return m_ht.equal_range(key); }
@ -420,7 +420,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key, std::size_t precalculated_hash) {
return m_ht.equal_range(key, precalculated_hash);
@ -429,14 +429,14 @@ public:
/**
* @copydoc equal_range(const K& key)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key) const { return m_ht.equal_range(key); }
/**
* @copydoc equal_range(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, class CP = Compare,
template <typename K, typename KE = KeyEqual, typename CP = Compare,
typename std::enable_if<has_is_transparent<KE>::value && has_is_transparent<CP>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key, std::size_t precalculated_hash) const {
return m_ht.equal_range(key, precalculated_hash);

View File

@ -68,7 +68,7 @@ namespace tsl {
* insert will invalidate the iterators). Or if there is a rehash.
* - erase: iterator on the erased element is the only one which become invalid.
*/
template<class Key,
template <typename Key,
class Hash = std::hash<Key>,
class KeyEqual = std::equal_to<Key>,
class Allocator = std::allocator<Key>,
@ -77,7 +77,7 @@ template<class Key,
class GrowthPolicy = tsl::power_of_two_growth_policy>
class hopscotch_set {
private:
template<typename U>
template <typename U>
using has_is_transparent = tsl::detail_hopscotch_hash::has_is_transparent<U>;
class KeySelect {
@ -145,7 +145,7 @@ public:
explicit hopscotch_set(const Allocator& alloc) : hopscotch_set(ht::DEFAULT_INIT_BUCKETS_SIZE, alloc) {
}
template<class InputIt>
template <typename InputIt>
hopscotch_set(InputIt first, InputIt last,
size_type bucket_count = ht::DEFAULT_INIT_BUCKETS_SIZE,
const Hash& hash = Hash(),
@ -155,14 +155,14 @@ public:
insert(first, last);
}
template<class InputIt>
template <typename InputIt>
hopscotch_set(InputIt first, InputIt last,
size_type bucket_count,
const Allocator& alloc) : hopscotch_set(first, last, bucket_count, Hash(), KeyEqual(), alloc)
{
}
template<class InputIt>
template <typename InputIt>
hopscotch_set(InputIt first, InputIt last,
size_type bucket_count,
const Hash& hash,
@ -240,7 +240,7 @@ public:
iterator insert(const_iterator hint, const value_type& value) { return m_ht.insert(hint, value); }
iterator insert(const_iterator hint, value_type&& value) { return m_ht.insert(hint, std::move(value)); }
template<class InputIt>
template <typename InputIt>
void insert(InputIt first, InputIt last) { m_ht.insert(first, last); }
void insert(std::initializer_list<value_type> ilist) { m_ht.insert(ilist.begin(), ilist.end()); }
@ -253,7 +253,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
std::pair<iterator, bool> emplace(Args&&... args) { return m_ht.emplace(std::forward<Args>(args)...); }
@ -265,7 +265,7 @@ public:
*
* Mainly here for compatibility with the std::unordered_map interface.
*/
template<class... Args>
template <typename... Args>
iterator emplace_hint(const_iterator hint, Args&&... args) {
return m_ht.emplace_hint(hint, std::forward<Args>(args)...);
}
@ -290,7 +290,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type erase(const K& key) { return m_ht.erase(key); }
/**
@ -299,7 +299,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup to the value if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type erase(const K& key, std::size_t precalculated_hash) {
return m_ht.erase(key, precalculated_hash);
}
@ -325,7 +325,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type count(const K& key) const { return m_ht.count(key); }
/**
@ -334,7 +334,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
size_type count(const K& key, std::size_t precalculated_hash) const { return m_ht.count(key, precalculated_hash); }
@ -359,7 +359,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
iterator find(const K& key) { return m_ht.find(key); }
/**
@ -368,13 +368,13 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
iterator find(const K& key, std::size_t precalculated_hash) { return m_ht.find(key, precalculated_hash); }
/**
* @copydoc find(const K& key)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const_iterator find(const K& key) const { return m_ht.find(key); }
/**
@ -383,7 +383,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
const_iterator find(const K& key, std::size_t precalculated_hash) const { return m_ht.find(key, precalculated_hash); }
@ -412,7 +412,7 @@ public:
* This overload only participates in the overload resolution if the typedef KeyEqual::is_transparent exists.
* If so, K must be hashable and comparable to Key.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key) { return m_ht.equal_range(key); }
/**
@ -421,7 +421,7 @@ public:
* Use the hash value 'precalculated_hash' instead of hashing the key. The hash value should be the same
* as hash_function()(key). Usefull to speed-up the lookup if you already have the hash.
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<iterator, iterator> equal_range(const K& key, std::size_t precalculated_hash) {
return m_ht.equal_range(key, precalculated_hash);
}
@ -429,13 +429,13 @@ public:
/**
* @copydoc equal_range(const K& key)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key) const { return m_ht.equal_range(key); }
/**
* @copydoc equal_range(const K& key, std::size_t precalculated_hash)
*/
template<class K, class KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
template <typename K, typename KE = KeyEqual, typename std::enable_if<has_is_transparent<KE>::value>::type* = nullptr>
std::pair<const_iterator, const_iterator> equal_range(const K& key, std::size_t precalculated_hash) const {
return m_ht.equal_range(key, precalculated_hash);
}

View File

@ -161,7 +161,7 @@ inline bool_if_safe_conversion<A, B> greaterOp(A a, B b)
constexpr DB::Int64 MAX_INT64_WITH_EXACT_FLOAT64_REPR = 9007199254740992LL; // 2^53
template<>
template <>
inline bool greaterOp<DB::Float64, DB::Int64>(DB::Float64 f, DB::Int64 i)
{
if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR)
@ -171,7 +171,7 @@ inline bool greaterOp<DB::Float64, DB::Int64>(DB::Float64 f, DB::Int64 i)
|| (f > static_cast<DB::Float64>(std::numeric_limits<DB::Int64>::min()) && static_cast<DB::Int64>(f) > i);
}
template<>
template <>
inline bool greaterOp<DB::Int64, DB::Float64>(DB::Int64 i, DB::Float64 f)
{
if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR)
@ -181,7 +181,7 @@ inline bool greaterOp<DB::Int64, DB::Float64>(DB::Int64 i, DB::Float64 f)
|| (f < static_cast<DB::Float64>(std::numeric_limits<DB::Int64>::max()) && i > static_cast<DB::Int64>(f));
}
template<>
template <>
inline bool greaterOp<DB::Float64, DB::UInt64>(DB::Float64 f, DB::UInt64 u)
{
if (u <= static_cast<DB::UInt64>(MAX_INT64_WITH_EXACT_FLOAT64_REPR))
@ -191,7 +191,7 @@ inline bool greaterOp<DB::Float64, DB::UInt64>(DB::Float64 f, DB::UInt64 u)
|| (f >= 0 && static_cast<DB::UInt64>(f) > u);
}
template<>
template <>
inline bool greaterOp<DB::UInt64, DB::Float64>(DB::UInt64 u, DB::Float64 f)
{
if (u <= static_cast<DB::UInt64>(MAX_INT64_WITH_EXACT_FLOAT64_REPR))
@ -202,49 +202,49 @@ inline bool greaterOp<DB::UInt64, DB::Float64>(DB::UInt64 u, DB::Float64 f)
}
// Case 3b for float32
template<>
template <>
inline bool greaterOp<DB::Float32, DB::Int64>(DB::Float32 f, DB::Int64 i)
{
return greaterOp(static_cast<DB::Float64>(f), i);
}
template<>
template <>
inline bool greaterOp<DB::Int64, DB::Float32>(DB::Int64 i, DB::Float32 f)
{
return greaterOp(i, static_cast<DB::Float64>(f));
}
template<>
template <>
inline bool greaterOp<DB::Float32, DB::UInt64>(DB::Float32 f, DB::UInt64 u)
{
return greaterOp(static_cast<DB::Float64>(f), u);
}
template<>
template <>
inline bool greaterOp<DB::UInt64, DB::Float32>(DB::UInt64 u, DB::Float32 f)
{
return greaterOp(u, static_cast<DB::Float64>(f));
}
template<>
template <>
inline bool greaterOp<DB::Float64, DB::UInt128>(DB::Float64 f, DB::UInt128 u)
{
return u.low == 0 && greaterOp(f, u.high);
}
template<>
template <>
inline bool greaterOp<DB::UInt128, DB::Float64>(DB::UInt128 u, DB::Float64 f)
{
return u.low != 0 || greaterOp(u.high, f);
}
template<>
template <>
inline bool greaterOp<DB::Float32, DB::UInt128>(DB::Float32 f, DB::UInt128 u)
{
return greaterOp(static_cast<DB::Float64>(f), u);
}
template<>
template <>
inline bool greaterOp<DB::UInt128, DB::Float32>(DB::UInt128 u, DB::Float32 f)
{
return greaterOp(u, static_cast<DB::Float64>(f));
@ -262,73 +262,73 @@ inline bool_if_safe_conversion<A, B> equalsOp(A a, B b)
return a == b;
}
template<>
template <>
inline bool equalsOp<DB::Float64, DB::UInt64>(DB::Float64 f, DB::UInt64 u)
{
return static_cast<DB::UInt64>(f) == u && f == static_cast<DB::Float64>(u);
}
template<>
template <>
inline bool equalsOp<DB::UInt64, DB::Float64>(DB::UInt64 u, DB::Float64 f)
{
return u == static_cast<DB::UInt64>(f) && static_cast<DB::Float64>(u) == f;
}
template<>
template <>
inline bool equalsOp<DB::Float64, DB::Int64>(DB::Float64 f, DB::Int64 u)
{
return static_cast<DB::Int64>(f) == u && f == static_cast<DB::Float64>(u);
}
template<>
template <>
inline bool equalsOp<DB::Int64, DB::Float64>(DB::Int64 u, DB::Float64 f)
{
return u == static_cast<DB::Int64>(f) && static_cast<DB::Float64>(u) == f;
}
template<>
template <>
inline bool equalsOp<DB::Float32, DB::UInt64>(DB::Float32 f, DB::UInt64 u)
{
return static_cast<DB::UInt64>(f) == u && f == static_cast<DB::Float32>(u);
}
template<>
template <>
inline bool equalsOp<DB::UInt64, DB::Float32>(DB::UInt64 u, DB::Float32 f)
{
return u == static_cast<DB::UInt64>(f) && static_cast<DB::Float32>(u) == f;
}
template<>
template <>
inline bool equalsOp<DB::Float32, DB::Int64>(DB::Float32 f, DB::Int64 u)
{
return static_cast<DB::Int64>(f) == u && f == static_cast<DB::Float32>(u);
}
template<>
template <>
inline bool equalsOp<DB::Int64, DB::Float32>(DB::Int64 u, DB::Float32 f)
{
return u == static_cast<DB::Int64>(f) && static_cast<DB::Float32>(u) == f;
}
template<>
template <>
inline bool equalsOp<DB::UInt128, DB::Float64>(DB::UInt128 u, DB::Float64 f)
{
return u.low == 0 && equalsOp(static_cast<UInt64>(u.high), f);
}
template<>
template <>
inline bool equalsOp<DB::UInt128, DB::Float32>(DB::UInt128 u, DB::Float32 f)
{
return equalsOp(u, static_cast<DB::Float64>(f));
}
template<>
template <>
inline bool equalsOp<DB::Float64, DB::UInt128>(DB::Float64 f, DB::UInt128 u)
{
return equalsOp(u, f);
}
template<>
template <>
inline bool equalsOp<DB::Float32, DB::UInt128>(DB::Float32 f, DB::UInt128 u)
{
return equalsOp(static_cast<DB::Float64>(f), u);

View File

@ -36,7 +36,7 @@ struct QualifiedTableName
namespace std
{
template<> struct hash<DB::QualifiedTableName>
template <> struct hash<DB::QualifiedTableName>
{
using argument_type = DB::QualifiedTableName;
using result_type = size_t;

View File

@ -68,7 +68,7 @@ Block AggregatingSortedBlockInputStream::readImpl()
}
template<class TSortCursor>
template <typename TSortCursor>
void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
{
size_t merged_rows = 0;
@ -133,7 +133,7 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
}
template <class TSortCursor>
template <typename TSortCursor>
void AggregatingSortedBlockInputStream::addRow(TSortCursor & cursor)
{
for (size_t i = 0, size = column_numbers_to_aggregate.size(); i < size; ++i)

View File

@ -70,12 +70,12 @@ private:
/** We support two different cursors - with Collation and without.
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
*/
template <class TSortCursor>
template <typename TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/** Extract all states of aggregate functions and merge them with the current group.
*/
template <class TSortCursor>
template <typename TSortCursor>
void addRow(TSortCursor & cursor);
};

View File

@ -136,7 +136,7 @@ Block CollapsingSortedBlockInputStream::readImpl()
}
template<class TSortCursor>
template <typename TSortCursor>
void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
{
size_t merged_rows = 0;

View File

@ -90,7 +90,7 @@ private:
/** We support two different cursors - with Collation and without.
* Templates are used instead of polymorphic SortCursors and calls to virtual functions.
*/
template<class TSortCursor>
template <typename TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/// Output to result rows for the current primary key.

View File

@ -68,7 +68,7 @@ public:
void readSuffixImpl() override;
/// for use in implementations of IColumn::gather()
template<typename Column>
template <typename Column>
void gather(Column & column_res);
private:
@ -110,7 +110,7 @@ private:
Poco::Logger * log;
};
template<typename Column>
template <typename Column>
void ColumnGathererStream::gather(Column & column_res)
{
if (source_to_fully_copy) /// Was set on a previous iteration

View File

@ -218,7 +218,7 @@ void GraphiteRollupSortedBlockInputStream::merge(ColumnPlainPtrs & merged_column
}
template <class TSortCursor>
template <typename TSortCursor>
void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor, const Graphite::Pattern * next_pattern)
{
/// Copy unmodified column values.

View File

@ -199,7 +199,7 @@ private:
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/// Insert the values into the resulting columns, which will not be changed in the future.
template <class TSortCursor>
template <typename TSortCursor>
void startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor, const Graphite::Pattern * next_pattern);
/// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows.

View File

@ -153,7 +153,7 @@ protected:
/// These methods are used in Collapsing/Summing/Aggregating... SortedBlockInputStream-s.
/// Save the row pointed to by cursor in `row`.
template <class TSortCursor>
template <typename TSortCursor>
void setRow(Row & row, TSortCursor & cursor)
{
for (size_t i = 0; i < num_columns; ++i)
@ -185,7 +185,7 @@ protected:
}
}
template <class TSortCursor>
template <typename TSortCursor>
void setRowRef(RowRef & row_ref, TSortCursor & cursor)
{
row_ref.row_num = cursor.impl->pos;
@ -195,7 +195,7 @@ protected:
row_ref.columns[i] = cursor->all_columns[i];
}
template <class TSortCursor>
template <typename TSortCursor>
void setPrimaryKeyRef(RowRef & row_ref, TSortCursor & cursor)
{
row_ref.row_num = cursor.impl->pos;

View File

@ -58,7 +58,7 @@ Block ReplacingSortedBlockInputStream::readImpl()
}
template<class TSortCursor>
template <typename TSortCursor>
void ReplacingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
{
size_t merged_rows = 0;

View File

@ -63,7 +63,7 @@ private:
PODArray<RowSourcePart> current_row_sources; /// Sources of rows with the current primary key
template<class TSortCursor>
template <typename TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/// Output into result the rows for current primary key.

View File

@ -176,7 +176,7 @@ Block SummingSortedBlockInputStream::readImpl()
}
template <class TSortCursor>
template <typename TSortCursor>
void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
{
size_t merged_rows = 0;
@ -269,7 +269,7 @@ public:
};
template <class TSortCursor>
template <typename TSortCursor>
bool SummingSortedBlockInputStream::mergeMaps(Row & row, TSortCursor & cursor)
{
bool non_empty_map_present = false;
@ -283,7 +283,7 @@ bool SummingSortedBlockInputStream::mergeMaps(Row & row, TSortCursor & cursor)
}
template <class TSortCursor>
template <typename TSortCursor>
bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row & row, TSortCursor & cursor)
{
/// Strongly non-optimal.
@ -367,7 +367,7 @@ bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row &
}
template <class TSortCursor>
template <typename TSortCursor>
bool SummingSortedBlockInputStream::addRow(Row & row, TSortCursor & cursor)
{
bool res = mergeMaps(row, cursor); /// Is there at least one non-zero number or non-empty array

View File

@ -90,7 +90,7 @@ private:
/** We support two different cursors - with Collation and without.
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.
*/
template <class TSortCursor>
template <typename TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/// Insert the summed row for the current group into the result.
@ -99,16 +99,16 @@ private:
/** For nested Map, a merge by key is performed with the ejection of rows of nested arrays, in which
* all items are zero.
*/
template <class TSortCursor>
template <typename TSortCursor>
bool mergeMaps(Row & row, TSortCursor & cursor);
template <class TSortCursor>
template <typename TSortCursor>
bool mergeMap(const MapDescription & map, Row & row, TSortCursor & cursor);
/** Add the row under the cursor to the `row`.
* Returns false if the result is zero.
*/
template <class TSortCursor>
template <typename TSortCursor>
bool addRow(Row & row, TSortCursor & cursor);
};

View File

@ -237,13 +237,13 @@ protected:
}
private:
template<StreamUnionMode mode2 = mode>
template <StreamUnionMode mode2 = mode>
BlockExtraInfo doGetBlockExtraInfo(typename std::enable_if<mode2 == StreamUnionMode::ExtraInfo>::type * = nullptr) const
{
return received_payload.extra_info;
}
template<StreamUnionMode mode2 = mode>
template <StreamUnionMode mode2 = mode>
BlockExtraInfo doGetBlockExtraInfo(typename std::enable_if<mode2 == StreamUnionMode::Basic>::type * = nullptr) const
{
throw Exception("Method getBlockExtraInfo is not supported for mode StreamUnionMode::Basic",

View File

@ -19,7 +19,7 @@ namespace DB
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
*/
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
@ -51,13 +51,13 @@ protected:
private:
// pointer types to getXXX functions
// for single key dictionaries
template <class Type>
template <typename Type>
using DictionaryGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, PaddedPODArray<Type> &) const;
using DictionaryStringGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, ColumnString *) const;
// for complex complex key dictionaries
template <class Type>
template <typename Type>
using GetterByKey = void (DictionaryType::*)(
const std::string &, const Columns &, const DataTypes &, PaddedPODArray<Type> & out) const;
using StringGetterByKey = void (DictionaryType::*)(
@ -65,34 +65,34 @@ private:
// call getXXX
// for single key dictionaries
template <class Type, class Container>
template <typename Type, typename Container>
void callGetter(DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <class Container>
template <typename Container>
void callGetter(DictionaryStringGetter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
// for complex complex key dictionaries
template <class Type, class Container>
template <typename Type, typename Container>
void callGetter(GetterByKey<Type> getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <class Container>
template <typename Container>
void callGetter(StringGetterByKey getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <template <class> class Getter, class StringGetter>
template <template <typename> class Getter, typename StringGetter>
Block fillBlock(const PaddedPODArray<Key> & ids, const Columns & keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
template <class AttributeType, class Getter>
template <typename AttributeType, typename Getter>
ColumnPtr getColumnFromAttribute(Getter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <class Getter>
template <typename Getter>
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
@ -115,7 +115,7 @@ private:
GetColumnsFunction get_view_columns_function;
};
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
PaddedPODArray<Key> && ids, const Names& column_names)
@ -127,7 +127,7 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
{
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const std::vector<StringRef> & keys, const Names& column_names)
@ -140,7 +140,7 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
fillKeyColumns(keys, 0, keys.size(), dictionaty_structure, key_columns);
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const Columns & data_columns, const Names & column_names,
@ -155,7 +155,7 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
{
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
{
if (!key_columns.empty())
@ -197,8 +197,8 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
}
}
template <class DictionaryType, class Key>
template <class Type, class Container>
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -207,8 +207,8 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
(dictionary.*getter)(attribute.name, ids, container);
}
template <class DictionaryType, class Key>
template <class Container>
template <typename DictionaryType, typename Key>
template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryStringGetter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -217,8 +217,8 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
(dictionary.*getter)(attribute.name, ids, container);
}
template <class DictionaryType, class Key>
template <class Type, class Container>
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
GetterByKey<Type> getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -227,8 +227,8 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
(dictionary.*getter)(attribute.name, keys, data_types, container);
}
template <class DictionaryType, class Key>
template <class Container>
template <typename DictionaryType, typename Key>
template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
StringGetterByKey getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -237,8 +237,8 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
(dictionary.*getter)(attribute.name, keys, data_types, container);
}
template <class DictionaryType, class Key>
template <template <class> class Getter, class StringGetter>
template <typename DictionaryType, typename Key>
template <template <typename> class Getter, typename StringGetter>
Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key>& ids, const Columns& keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
{
@ -317,8 +317,8 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
return Block(block_columns);
}
template <class DictionaryType, class Key>
template <class AttributeType, class Getter>
template <typename DictionaryType, typename Key>
template <typename AttributeType, typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
Getter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -332,8 +332,8 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribut
return ColumnPtr(std::move(column_vector));
}
template <class DictionaryType, class Key>
template <class Getter>
template <typename DictionaryType, typename Key>
template <typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute(
Getter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
@ -345,7 +345,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAt
return column_string;
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key>& ids) const
{
auto column_vector = std::make_shared<ColumnVector<UInt64>>();
@ -357,7 +357,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(cons
return column_vector;
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
void DictionaryBlockInputStream<DictionaryType, Key>::fillKeyColumns(
const std::vector<StringRef> & keys, size_t start, size_t size,
const DictionaryStructure& dictionary_structure, ColumnsWithTypeAndName & columns) const

View File

@ -17,7 +17,7 @@ namespace DB
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
*/
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
@ -35,20 +35,20 @@ protected:
Block getBlock(size_t start, size_t length) const override;
private:
template <class Type>
template <typename Type>
using DictionaryGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &,
const PaddedPODArray<UInt16> &, PaddedPODArray<Type> &) const;
template <class AttributeType>
template <typename AttributeType>
ColumnPtr getColumnFromAttribute(DictionaryGetter<AttributeType> getter,
const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
template <class T>
template <typename T>
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T>& array) const;
template <class T>
template <typename T>
void addSpecialColumn(
const std::experimental::optional<DictionarySpecialAttribute>& attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names,
@ -65,7 +65,7 @@ private:
};
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
RangeDictionaryBlockInputStream<DictionaryType, Key>::RangeDictionaryBlockInputStream(
DictionatyPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<UInt16> && start_dates, PaddedPODArray<UInt16> && end_dates)
@ -75,7 +75,7 @@ RangeDictionaryBlockInputStream<DictionaryType, Key>::RangeDictionaryBlockInputS
{
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
{
PaddedPODArray<Key> block_ids;
@ -95,8 +95,8 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
return fillBlock(block_ids, block_start_dates, block_end_dates);
}
template <class DictionaryType, class Key>
template <class AttributeType>
template <typename DictionaryType, typename Key>
template <typename AttributeType>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key>& ids,
const PaddedPODArray<UInt16> & dates, const DictionaryAttribute& attribute, const DictionaryType& dictionary) const
@ -106,7 +106,7 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
return ColumnPtr(std::move(column_vector));
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const
@ -116,8 +116,8 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
return ColumnPtr(std::move(column_string));
}
template <class DictionaryType, class Key>
template <class T>
template <typename DictionaryType, typename Key>
template <typename T>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPODArray(const PaddedPODArray<T>& array) const
{
auto column_vector = std::make_unique<ColumnVector<T>>();
@ -130,8 +130,8 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPOD
}
template <class DictionaryType, class Key>
template <class T>
template <typename DictionaryType, typename Key>
template <typename T>
void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
const std::experimental::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::string& default_name, const std::unordered_set<std::string> & column_names,
@ -146,7 +146,7 @@ void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
}
}
template <class DictionaryType, class Key>
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key>& ids,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const

View File

@ -90,7 +90,7 @@ template <typename A, typename B> struct GreaterOrEqualsOp
};
template<typename A, typename B, typename Op>
template <typename A, typename B, typename Op>
struct NumComparisonImpl
{
/// If you don't specify NO_INLINE, the compiler will inline this function, but we don't need this as this function contains tight loop inside.

View File

@ -705,7 +705,7 @@ private:
}
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value ||
std::is_same<Name2, NameToUnixTimestamp>::value ||
@ -721,7 +721,7 @@ private:
/** Conversion of anything to String. For DateTime, it allows second optional argument - time zone.
*/
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<ToDataType2, DataTypeString>::value>::type * = nullptr) const
{
@ -747,7 +747,7 @@ private:
return std::make_shared<ToDataType2>();
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<Name2, NameToUnixTimestamp>::value, void>::type * = nullptr) const
{
@ -773,7 +773,7 @@ private:
return std::make_shared<ToDataType2>();
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
template <typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<Name2, NameToDate>::value>::type * = nullptr) const
{

View File

@ -466,7 +466,7 @@ struct ToYYYYMMDDhhmmssImpl
};
template<typename FromType, typename ToType, typename Transform>
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
static void vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const DateLUTImpl & time_zone)

View File

@ -167,7 +167,7 @@ protected:
// Match the type of the centrods array and convert them to Float64, because we
// don't want to have problems calculating negative distances of UInts
template<typename CentroidsType>
template <typename CentroidsType>
bool fillCentroids(const IColumn * centroids_array_untyped, std::vector<Float64> & centroids)
{
const ColumnConst * const_centroids_array = checkAndGetColumnConst<ColumnVector<Array>>(centroids_array_untyped);
@ -191,7 +191,7 @@ protected:
return true;
}
template<typename CentroidsType, typename OutputType>
template <typename CentroidsType, typename OutputType>
bool executeOperation(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * centroids_array_untyped)
{
// Match the type of the output
@ -220,7 +220,7 @@ protected:
return true;
}
template<typename InputType, typename OutputType, typename CentroidsType>
template <typename InputType, typename OutputType, typename CentroidsType>
bool executeOperationTyped(const IColumn * in_untyped, PaddedPODArray<OutputType> & dst, const IColumn * centroids_array_untyped)
{
const auto maybe_const = in_untyped->convertToFullColumnIfConst();

View File

@ -331,7 +331,7 @@ struct ArraySumImpl
throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <class Element, class Result>
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets_t & offsets, ColumnPtr & res_ptr)
{
const ColumnVector<Element> * column = checkAndGetColumn<ColumnVector<Element>>(&*mapped);

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
* Accept any numeric types, return a UInt8 containing 0 or 1.
*/
template<typename B>
template <typename B>
struct AndImpl
{
static inline bool isSaturable()
@ -43,7 +43,7 @@ struct AndImpl
}
};
template<typename B>
template <typename B>
struct OrImpl
{
static inline bool isSaturable()
@ -62,7 +62,7 @@ struct OrImpl
}
};
template<typename B>
template <typename B>
struct XorImpl
{
static inline bool isSaturable()
@ -81,7 +81,7 @@ struct XorImpl
}
};
template<typename A>
template <typename A>
struct NotImpl
{
using ResultType = UInt8;

View File

@ -27,7 +27,7 @@ namespace ErrorCodes
*/
template<typename Name>
template <typename Name>
class FunctionReinterpretAsStringImpl : public IFunction
{
public:
@ -52,7 +52,7 @@ public:
return std::make_shared<DataTypeString>();
}
template<typename T>
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
@ -112,7 +112,7 @@ public:
};
template<typename ToDataType, typename Name>
template <typename ToDataType, typename Name>
class FunctionReinterpretStringAs : public IFunction
{
public:

View File

@ -37,7 +37,7 @@ namespace ErrorCodes
* - N = 0: round to an integer
*/
template<typename A>
template <typename A>
struct RoundToExp2Impl
{
using ResultType = A;
@ -48,7 +48,7 @@ struct RoundToExp2Impl
}
};
template<>
template <>
struct RoundToExp2Impl<Float32>
{
using ResultType = Float32;
@ -59,7 +59,7 @@ struct RoundToExp2Impl<Float32>
}
};
template<>
template <>
struct RoundToExp2Impl<Float64>
{
using ResultType = Float64;
@ -70,7 +70,7 @@ struct RoundToExp2Impl<Float64>
}
};
template<typename A>
template <typename A>
struct RoundDurationImpl
{
using ResultType = UInt16;
@ -96,7 +96,7 @@ struct RoundDurationImpl
}
};
template<typename A>
template <typename A>
struct RoundAgeImpl
{
using ResultType = UInt8;
@ -116,17 +116,17 @@ struct RoundAgeImpl
/** Quick calculation of the remainder of the division to apply to the rounding of integers.
* Without verification, because the divisor is always positive.
*/
template<typename T, typename Enable = void>
template <typename T, typename Enable = void>
struct FastModulo;
template<typename T>
template <typename T>
struct FastModulo<T, typename std::enable_if<std::is_integral<T>::value>::type>
{
private:
template<typename InputType, typename Enable = void>
template <typename InputType, typename Enable = void>
struct Extend;
template<typename InputType>
template <typename InputType>
struct Extend<InputType,
typename std::enable_if<std::is_same<InputType, Int8>::value
|| std::is_same<InputType, Int16>::value>::type>
@ -134,7 +134,7 @@ private:
using Type = Int64;
};
template<typename InputType>
template <typename InputType>
struct Extend<InputType,
typename std::enable_if<std::is_same<InputType, UInt8>::value
|| std::is_same<InputType, UInt16>::value>::type>
@ -142,7 +142,7 @@ private:
using Type = UInt64;
};
template<typename InputType>
template <typename InputType>
struct Extend<InputType,
typename std::enable_if<std::is_integral<InputType>::value
&& (sizeof(InputType) >= 4)>::type>
@ -186,10 +186,10 @@ enum ScaleMode
/** Implementing low-level rounding functions for integer values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
template <typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
struct IntegerRoundingComputation;
template<typename T, int rounding_mode, ScaleMode scale_mode>
template <typename T, int rounding_mode, ScaleMode scale_mode>
struct IntegerRoundingComputation<T, rounding_mode, scale_mode,
typename std::enable_if<std::is_integral<T>::value
&& ((scale_mode == PositiveScale) || (scale_mode == ZeroScale))>::type>
@ -207,7 +207,7 @@ struct IntegerRoundingComputation<T, rounding_mode, scale_mode,
}
};
template<typename T>
template <typename T>
struct IntegerRoundingComputation<T, _MM_FROUND_NINT, NegativeScale,
typename std::enable_if<std::is_integral<T>::value>::type>
{
@ -234,7 +234,7 @@ struct IntegerRoundingComputation<T, _MM_FROUND_NINT, NegativeScale,
}
};
template<typename T>
template <typename T>
struct IntegerRoundingComputation<T, _MM_FROUND_CEIL, NegativeScale,
typename std::enable_if<std::is_integral<T>::value>::type>
{
@ -256,7 +256,7 @@ struct IntegerRoundingComputation<T, _MM_FROUND_CEIL, NegativeScale,
}
};
template<typename T>
template <typename T>
struct IntegerRoundingComputation<T, _MM_FROUND_FLOOR, NegativeScale,
typename std::enable_if<std::is_integral<T>::value>::type>
{
@ -360,10 +360,10 @@ protected:
/** Implementation of low-level round-off functions for floating-point values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
template <typename T, int rounding_mode, ScaleMode scale_mode>
class FloatRoundingComputation;
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float32, rounding_mode, PositiveScale>
: public BaseFloatRoundingComputation<Float32>
{
@ -389,7 +389,7 @@ public:
}
};
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float32, rounding_mode, NegativeScale>
: public BaseFloatRoundingComputation<Float32>
{
@ -433,7 +433,7 @@ private:
}
};
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float32, rounding_mode, ZeroScale>
: public BaseFloatRoundingComputation<Float32>
{
@ -454,7 +454,7 @@ public:
}
};
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float64, rounding_mode, PositiveScale>
: public BaseFloatRoundingComputation<Float64>
{
@ -480,7 +480,7 @@ public:
}
};
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float64, rounding_mode, NegativeScale>
: public BaseFloatRoundingComputation<Float64>
{
@ -524,7 +524,7 @@ private:
}
};
template<int rounding_mode>
template <int rounding_mode>
class FloatRoundingComputation<Float64, rounding_mode, ZeroScale>
: public BaseFloatRoundingComputation<Float64>
{
@ -622,12 +622,12 @@ public:
/** Implementing high-level rounding functions.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
template <typename T, int rounding_mode, ScaleMode scale_mode, typename Enable = void>
struct FunctionRoundingImpl;
/** Implement high-level rounding functions for integer values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
template <typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
typename std::enable_if<std::is_integral<T>::value && (scale_mode != NullScale)>::type>
{
@ -659,7 +659,7 @@ public:
/** Implement high-level round-off functions for floating-point values.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
template <typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
typename std::enable_if<std::is_floating_point<T>::value && (scale_mode != NullScale)>::type>
{
@ -739,7 +739,7 @@ public:
/** Implementation of high-level rounding functions in the case when a zero value is returned.
*/
template<typename T, int rounding_mode, ScaleMode scale_mode>
template <typename T, int rounding_mode, ScaleMode scale_mode>
struct FunctionRoundingImpl<T, rounding_mode, scale_mode,
typename std::enable_if<scale_mode == NullScale>::type>
{
@ -761,13 +761,13 @@ namespace
{
/// Individual degrees of the number 10.
template<size_t N>
template <size_t N>
struct PowerOf10
{
static const size_t value = 10 * PowerOf10<N - 1>::value;
};
template<>
template <>
struct PowerOf10<0>
{
static const size_t value = 1;
@ -776,30 +776,30 @@ namespace
/// Declaring and defining a container containing a table of powers of 10.
template<size_t... TArgs>
template <size_t... TArgs>
struct TableContainer
{
static const std::array<size_t, sizeof...(TArgs)> values;
};
template<size_t... TArgs>
template <size_t... TArgs>
const std::array<size_t, sizeof...(TArgs)> TableContainer<TArgs...>::values {{ TArgs... }};
/// The generator of the first N degrees.
template<size_t N, size_t... TArgs>
template <size_t N, size_t... TArgs>
struct FillArrayImpl
{
using result = typename FillArrayImpl<N - 1, PowerOf10<N>::value, TArgs...>::result;
};
template<size_t... TArgs>
template <size_t... TArgs>
struct FillArrayImpl<0, TArgs...>
{
using result = TableContainer<PowerOf10<0>::value, TArgs...>;
};
template<size_t N>
template <size_t N>
struct FillArray
{
using result = typename FillArrayImpl<N - 1>::result;
@ -809,10 +809,10 @@ struct FillArray
* then converts it to a value that can be used in operations of
* multiplication and division. Therefore, it is called a scale.
*/
template<typename T, typename U, typename Enable = void>
template <typename T, typename U, typename Enable = void>
struct ScaleForRightType;
template<typename T, typename U>
template <typename T, typename U>
struct ScaleForRightType<T, U,
typename std::enable_if<
std::is_floating_point<T>::value
@ -857,7 +857,7 @@ struct ScaleForRightType<T, U,
}
};
template<typename T, typename U>
template <typename T, typename U>
struct ScaleForRightType<T, U,
typename std::enable_if<
std::is_floating_point<T>::value
@ -888,7 +888,7 @@ struct ScaleForRightType<T, U,
}
};
template<typename T, typename U>
template <typename T, typename U>
struct ScaleForRightType<T, U,
typename std::enable_if<
std::is_integral<T>::value
@ -925,7 +925,7 @@ struct ScaleForRightType<T, U,
}
};
template<typename T, typename U>
template <typename T, typename U>
struct ScaleForRightType<T, U,
typename std::enable_if<
std::is_integral<T>::value
@ -946,7 +946,7 @@ struct ScaleForRightType<T, U,
/** Turn the precision parameter into a scale.
*/
template<typename T>
template <typename T>
struct ScaleForLeftType
{
static inline void apply(const ColumnPtr & column, ScaleMode & scale_mode, size_t & scale)
@ -1025,7 +1025,7 @@ struct Dispatcher
* (U)Int8/16/32/64 or Float32/64, and accept an additional optional
* parameter (default is 0).
*/
template<typename Name, int rounding_mode>
template <typename Name, int rounding_mode>
class FunctionRounding : public IFunction
{
public:
@ -1033,13 +1033,13 @@ public:
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionRounding>(); }
private:
template<typename T>
template <typename T>
bool checkType(const IDataType * type) const
{
return typeid_cast<const T *>(type);
}
template<typename T>
template <typename T>
bool executeForType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))

View File

@ -49,7 +49,7 @@ struct HasParam
}
};
template<typename NumericType>
template <typename NumericType>
struct ExtractNumericType
{
using ResultType = NumericType;
@ -239,7 +239,7 @@ struct ExtractParamImpl
/** For the case where the type of field to extract is a string.
*/
template<typename ParamExtractor>
template <typename ParamExtractor>
struct ExtractParamToStringImpl
{
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets,

View File

@ -18,7 +18,7 @@ namespace ErrorCodes
}
template<typename B>
template <typename B>
struct AndImpl
{
static inline UInt8 apply(UInt8 a, B b)
@ -27,7 +27,7 @@ struct AndImpl
}
};
template<typename B>
template <typename B>
struct OrImpl
{
static inline UInt8 apply(UInt8 a, B b)
@ -36,7 +36,7 @@ struct OrImpl
}
};
template<typename B>
template <typename B>
struct XorImpl
{
static inline UInt8 apply(UInt8 a, B b)

View File

@ -7,7 +7,7 @@ namespace DB
/// computation of the hash depends on the partitioning of blocks
/// so you need to compute a hash of n complete pieces and one incomplete
template <class Buffer>
template <typename Buffer>
void IHashingBuffer<Buffer>::calculateHash(DB::BufferBase::Position data, size_t len)
{
if (len)

View File

@ -11,7 +11,7 @@
namespace DB
{
template <class Buffer>
template <typename Buffer>
class IHashingBuffer : public BufferWithOwnMemory<Buffer>
{
public:

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
}
template <class IteratorSrc, class IteratorDst>
template <typename IteratorSrc, typename IteratorDst>
void parseHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes)
{
size_t src_pos = 0;

View File

@ -344,14 +344,14 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf)
x = -x;
}
template<typename T>
template <typename T>
void tryReadIntTextUnsafe(T & x, ReadBuffer & buf)
{
return readIntTextUnsafe<T, false>(x, buf);
}
template <bool throw_exception, class ExcepFun, class NoExcepFun, class... Args>
template <bool throw_exception, typename ExcepFun, typename NoExcepFun, typename... Args>
bool exceptionPolicySelector(ExcepFun && excep_f, NoExcepFun && no_excep_f, Args &&... args)
{
if (throw_exception)
@ -482,13 +482,13 @@ ReturnType readFloatTextImpl(T & x, ReadBuffer & buf)
return ReturnType(true);
}
template <class T>
template <typename T>
inline bool tryReadFloatText(T & x, ReadBuffer & buf)
{
return readFloatTextImpl<T, bool>(x, buf);
}
template <class T>
template <typename T>
inline void readFloatText(T & x, ReadBuffer & buf)
{
readFloatTextImpl<T, void>(x, buf);
@ -569,7 +569,7 @@ struct NullSink
void parseUUID(const UInt8 * src36, UInt8 * dst16);
void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16);
template<class IteratorSrc, class IteratorDst>
template <typename IteratorSrc, typename IteratorDst>
void formatHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes);
/// In YYYY-MM-DD format

View File

@ -56,7 +56,7 @@ protected:
};
template <class NameParser>
template <typename NameParser>
class IParserNameTypePair : public IParserBase
{
protected:
@ -69,7 +69,7 @@ using ParserNameTypePair = IParserNameTypePair<ParserIdentifier>;
/** Name and type separated by a space. The name can contain a dot. For example, Hits.URL String. */
using ParserCompoundNameTypePair = IParserNameTypePair<ParserCompoundIdentifier>;
template <class NameParser>
template <typename NameParser>
bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
NameParser name_parser;
@ -101,7 +101,7 @@ protected:
};
template <class NameParser>
template <typename NameParser>
class IParserColumnDeclaration : public IParserBase
{
protected:
@ -112,7 +112,7 @@ protected:
using ParserColumnDeclaration = IParserColumnDeclaration<ParserIdentifier>;
using ParserCompoundColumnDeclaration = IParserColumnDeclaration<ParserCompoundIdentifier>;
template <class NameParser>
template <typename NameParser>
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
NameParser name_parser;

View File

@ -21,7 +21,7 @@ enum class ColumnDefaultType
namespace std
{
template<> struct hash<DB::ColumnDefaultType>
template <> struct hash<DB::ColumnDefaultType>
{
size_t operator()(const DB::ColumnDefaultType type) const
{

View File

@ -53,7 +53,7 @@ public:
return key;
}
template<typename LoadFunc>
template <typename LoadFunc>
MappedPtr getOrSet(const Key & key, LoadFunc && load)
{
auto result = Base::getOrSet(key, load);

View File

@ -69,7 +69,7 @@ private:
void checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionaryStructure) const;
template <class ForwardIterator>
template <typename ForwardIterator>
std::string generateNamesAndTypesDescription(ForwardIterator begin, ForwardIterator end) const
{
if (begin == end)