Use std::popcount, ::countl_zero, ::countr_zero functions

- Introduced with the C++20 <bit> header

- The problem with __builtin_c(l|t)z() is that 0 as input has an
  undefined result (*) and the code did not always check. The std::
  versions do not have this issue.

- In some cases, we continue to use buildin_c(l|t)z(), (e.g. in
  src/Common/BitHelpers.h) because the std:: versions only accept
  unsigned inputs (and they also check that) and the casting would be
  ugly.

(*) https://gcc.gnu.org/onlinedocs/gcc/Other-Builtins.html
This commit is contained in:
Robert Schulze 2022-07-31 14:34:05 +00:00
parent c9e6850306
commit a7734672b9
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
20 changed files with 68 additions and 55 deletions

View File

@ -46,6 +46,7 @@
#include <boost/algorithm/string.hpp>
#include <boost/container/flat_map.hpp>
#include <Common/TerminalSize.h>
#include <bit>
static const char * documentation = R"(
@ -186,7 +187,7 @@ static UInt64 transform(UInt64 x, UInt64 seed)
if (x == 2 || x == 3)
return x ^ (seed & 1);
size_t num_leading_zeros = __builtin_clzll(x);
size_t num_leading_zeros = std::countl_zero(x);
return feistelNetwork(x, 64 - num_leading_zeros - 1, seed);
}

View File

@ -298,7 +298,7 @@ ColumnPtr ColumnDecimal<T>::filter(const IColumn::Filter & filt, ssize_t result_
{
while (mask)
{
size_t index = __builtin_ctzll(mask);
size_t index = std::countr_zero(mask);
res_data.push_back(data_pos[index]);
#ifdef __BMI__
mask = _blsr_u64(mask);

View File

@ -240,7 +240,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
size_t res_chars_size = res->chars.size();
while (mask)
{
size_t index = __builtin_ctzll(mask);
size_t index = std::countr_zero(mask);
res->chars.resize(res_chars_size + n);
memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n);
res_chars_size += n;

View File

@ -508,7 +508,7 @@ ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_s
{
while (mask)
{
size_t index = __builtin_ctzll(mask);
size_t index = std::countr_zero(mask);
res_data.push_back(data_pos[index]);
#ifdef __BMI__
mask = _blsr_u64(mask);

View File

@ -2,13 +2,14 @@
#include <Columns/ColumnVector.h>
#include <Common/typeid_cast.h>
#include <Common/HashTable/HashSet.h>
#include <bit>
#include "ColumnsCommon.h"
namespace DB
{
#if defined(__SSE2__) && defined(__POPCNT__)
#if defined(__SSE2__)
/// Transform 64-byte mask to 64-bit mask.
static UInt64 toBits64(const Int8 * bytes64)
{
@ -41,11 +42,11 @@ size_t countBytesInFilter(const UInt8 * filt, size_t start, size_t end)
const Int8 * end_pos = pos + (end - start);
#if defined(__SSE2__) && defined(__POPCNT__)
#if defined(__SSE2__)
const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
for (; pos < end_pos64; pos += 64)
count += __builtin_popcountll(toBits64(pos));
count += std::popcount(toBits64(pos));
/// TODO Add duff device for tail?
#endif
@ -74,11 +75,11 @@ size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * nu
const Int8 * pos2 = reinterpret_cast<const Int8 *>(null_map) + start;
const Int8 * end_pos = pos + (end - start);
#if defined(__SSE2__) && defined(__POPCNT__)
#if defined(__SSE2__)
const Int8 * end_pos64 = pos + (end - start) / 64 * 64;
for (; pos < end_pos64; pos += 64, pos2 += 64)
count += __builtin_popcountll(toBits64(pos) & ~toBits64(pos2));
count += std::popcount(toBits64(pos) & ~toBits64(pos2));
/// TODO Add duff device for tail?
#endif
@ -259,7 +260,7 @@ namespace
{
while (mask)
{
size_t index = __builtin_ctzll(mask);
size_t index = std::countr_zero(mask);
copy_array(offsets_pos + index);
#ifdef __BMI__
mask = _blsr_u64(mask);

View File

@ -36,7 +36,7 @@ inline UInt64 bytes64MaskToBits64Mask(const UInt8 * bytes64)
_mm256_loadu_si256(reinterpret_cast<const __m256i *>(bytes64)), zero32))) & 0xffffffff)
| (static_cast<UInt64>(_mm256_movemask_epi8(_mm256_cmpeq_epi8(
_mm256_loadu_si256(reinterpret_cast<const __m256i *>(bytes64+32)), zero32))) << 32);
#elif defined(__SSE2__) && defined(__POPCNT__)
#elif defined(__SSE2__)
static const __m128i zero16 = _mm_setzero_si128();
UInt64 res =
(static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpeq_epi8(

View File

@ -11,6 +11,7 @@
#include <IO/WriteHelpers.h>
#include <Core/Defines.h>
#include <bit>
#include <cmath>
#include <cstring>
@ -205,7 +206,7 @@ struct TrailingZerosCounter<UInt32>
{
static int apply(UInt32 val)
{
return __builtin_ctz(val);
return std::countr_zero(val);
}
};
@ -214,7 +215,7 @@ struct TrailingZerosCounter<UInt64>
{
static int apply(UInt64 val)
{
return __builtin_ctzll(val);
return std::countr_zero(val);
}
};

View File

@ -5,6 +5,7 @@
#include <Common/formatIPv6.h>
#include <cstring>
#include <bit>
namespace DB
@ -89,7 +90,7 @@ bool matchIPv6Subnet(const uint8_t * addr, const uint8_t * cidr_addr, UInt8 pref
if (mask)
{
auto offset = __builtin_ctz(mask);
auto offset = std::countr_zero(mask);
if (prefix / 8 != offset)
return prefix / 8 < offset;

View File

@ -78,7 +78,7 @@ private:
constexpr uint64_t nextAlphaSize(uint64_t x)
{
constexpr uint64_t alpha_map_elements_per_counter = 6;
return 1ULL << (sizeof(uint64_t) * 8 - __builtin_clzll(x * alpha_map_elements_per_counter));
return 1ULL << (sizeof(uint64_t) * 8 - std::countl_zero(x * alpha_map_elements_per_counter));
}
public:

View File

@ -2,6 +2,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <widechar_width.h>
#include <bit>
namespace DB
@ -124,7 +125,7 @@ size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t l
if (non_regular_width_mask)
{
auto num_regular_chars = __builtin_ctz(non_regular_width_mask);
auto num_regular_chars = std::countr_zero(non_regular_width_mask);
width += num_regular_chars;
i += num_regular_chars;
break;

View File

@ -83,7 +83,7 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
const auto threshold = vdupq_n_s8(0xBF);
for (; data < src_end_sse; data += bytes_sse)
res += __builtin_popcountll(get_nibble_mask(vcgtq_s8(vld1q_s8(reinterpret_cast<const int8_t *>(data)), threshold)));
res += std::popcount(get_nibble_mask(vcgtq_s8(vld1q_s8(reinterpret_cast<const int8_t *>(data)), threshold)));
res >>= 2;
#endif

View File

@ -1,5 +1,6 @@
#include <iostream>
#include <string>
#include <bit>
#include <fmt/format.h>
@ -561,7 +562,7 @@ int main(int argc, char ** argv)
/// Fill source data
for (size_t i = 0; i < size; ++i)
{
keys[i] = __builtin_ctz(i + 1); /// Make keys to have just slightly more realistic distribution.
keys[i] = std::countr_zero(i + 1); /// Make keys to have just slightly more realistic distribution.
values[i] = 1234.5; /// The distribution of values does not affect execution speed.
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <algorithm>
#include <bit>
#include <cstdint>
#include <Core/Defines.h>
@ -50,7 +51,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= min_size)
break;
@ -82,7 +83,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= min_size)
break;
@ -123,7 +124,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= max_size)
return 0;
@ -150,7 +151,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= size)
return 0;
@ -180,7 +181,7 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return offset >= a_size;
}
}
@ -203,7 +204,7 @@ inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return detail::cmp(a[offset], b[offset]);
}
}
@ -222,7 +223,7 @@ inline int memcmp16(const Char * a, const Char * b)
if (mask)
{
auto offset = __builtin_ctz(mask);
auto offset = std::countr_zero(mask);
return detail::cmp(a[offset], b[offset]);
}
@ -252,7 +253,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return offset >= size;
}
}
@ -285,7 +286,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= min_size)
break;
@ -317,7 +318,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= min_size)
break;
@ -359,7 +360,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= max_size)
return 0;
@ -386,7 +387,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t siz
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
if (offset >= size)
return 0;
@ -416,7 +417,7 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return offset >= a_size;
}
}
@ -439,7 +440,7 @@ inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return detail::cmp(a[offset], b[offset]);
}
}
@ -459,7 +460,7 @@ inline int memcmp16(const Char * a, const Char * b)
if (mask)
{
auto offset = __builtin_ctz(mask);
auto offset = std::countr_zero(mask);
return detail::cmp(a[offset], b[offset]);
}
@ -490,7 +491,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
if (mask)
{
offset += __builtin_ctz(mask);
offset += std::countr_zero(mask);
return offset >= size;
}
}
@ -523,7 +524,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
if (offset >= min_size)
break;
@ -548,7 +549,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
if (offset >= min_size)
break;
@ -589,7 +590,7 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
if (offset >= max_size)
return 0;
@ -611,7 +612,7 @@ inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t siz
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
if (offset >= size)
return 0;
@ -637,7 +638,7 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
return offset >= a_size;
}
}
@ -656,7 +657,7 @@ inline int memcmpSmallMultipleOf16(const Char * a, const Char * b, size_t size)
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
return detail::cmp(a[offset], b[offset]);
}
}
@ -672,7 +673,7 @@ inline int memcmp16(const Char * a, const Char * b)
mask = ~mask;
if (mask)
{
auto offset = __builtin_ctzll(mask) >> 2;
auto offset = std::countr_zero(mask) >> 2;
return detail::cmp(a[offset], b[offset]);
}
return 0;
@ -694,7 +695,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
if (mask)
{
offset += __builtin_ctzll(mask) >> 2;
offset += std::countr_zero(mask) >> 2;
return offset >= size;
}
}

View File

@ -1,5 +1,6 @@
#include "CompressedReadBufferBase.h"
#include <bit>
#include <cstring>
#include <cassert>
#include <city.h>
@ -93,8 +94,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c
}
/// Check if the difference caused by single bit flip in stored checksum.
size_t difference = __builtin_popcountll(expected_checksum.first ^ calculated_checksum.first)
+ __builtin_popcountll(expected_checksum.second ^ calculated_checksum.second);
size_t difference = std::popcount(expected_checksum.first ^ calculated_checksum.first)
+ std::popcount(expected_checksum.second ^ calculated_checksum.second);
if (difference == 1)
{

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTFunction.h>
#include <IO/WriteHelpers.h>
#include <Core/Types.h>
#include <bit>
namespace DB
@ -413,7 +414,7 @@ UInt32 getValuableBitsNumber(UInt64 min, UInt64 max)
{
UInt64 diff_bits = min ^ max;
if (diff_bits)
return 64 - __builtin_clzll(diff_bits);
return 64 - std::countl_zero(diff_bits);
return 0;
}

View File

@ -8,6 +8,7 @@
#include <Functions/IFunction.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
#include <bit>
namespace DB
@ -285,7 +286,7 @@ public:
{
while (x)
{
result_array_values_data.push_back(getTrailingZeroBitsUnsafe(x));
result_array_values_data.push_back(std::countr_zero(x));
x &= (x - 1);
}
}

View File

@ -1,5 +1,6 @@
#include <Functions/FunctionBinaryArithmetic.h>
#include <Functions/FunctionFactory.h>
#include <bit>
namespace DB
{
@ -14,7 +15,7 @@ struct BitHammingDistanceImpl
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)
{
UInt64 res = static_cast<UInt64>(a) ^ static_cast<UInt64>(b);
return __builtin_popcountll(res);
return std::popcount(res);
}
#if USE_EMBEDDED_COMPILER

View File

@ -10,6 +10,7 @@
#include <IO/Operators.h>
#include <base/find_symbols.h>
#include <cstdlib>
#include <bit>
#ifdef __SSE2__
#include <emmintrin.h>
@ -698,7 +699,7 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
uint16_t bit_mask = _mm_movemask_epi8(eq);
if (bit_mask)
{
next_pos += __builtin_ctz(bit_mask);
next_pos += std::countr_zero(bit_mask);
return;
}
}
@ -716,7 +717,7 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
uint64_t bit_mask = get_nibble_mask(eq);
if (bit_mask)
{
next_pos += __builtin_ctzll(bit_mask) >> 2;
next_pos += std::countr_zero(bit_mask) >> 2;
return;
}
}

View File

@ -33,7 +33,7 @@ static UInt32 toPowerOfTwo(UInt32 x)
{
if (x <= 1)
return 1;
return static_cast<UInt32>(1) << (32 - __builtin_clz(x - 1));
return static_cast<UInt32>(1) << (32 - std::countl_zero(x - 1));
}
ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr<TableJoin> table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_)

View File

@ -8,6 +8,7 @@
#include <base/range.h>
#include <Interpreters/castColumn.h>
#include <DataTypes/DataTypeNothing.h>
#include <bit>
#ifdef __SSE2__
#include <emmintrin.h>
@ -473,7 +474,7 @@ size_t numZerosInTail(const UInt8 * begin, const UInt8 * end)
count += 64;
else
{
count += __builtin_clzll(val);
count += std::countl_zero(val);
return count;
}
}
@ -507,7 +508,7 @@ size_t numZerosInTail(const UInt8 * begin, const UInt8 * end)
count += 64;
else
{
count += __builtin_clzll(val);
count += std::countl_zero(val);
return count;
}
}
@ -531,7 +532,7 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con
size_t count = 0;
#if defined(__SSE2__) && defined(__POPCNT__)
#if defined(__SSE2__)
const __m128i zero16 = _mm_setzero_si128();
while (end - begin >= 64)
{
@ -555,7 +556,7 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con
count += 64;
else
{
count += __builtin_clzll(val);
count += std::countl_zero(val);
return count;
}
}
@ -583,7 +584,7 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con
count += 64;
else
{
count += __builtin_clzll(val);
count += std::countl_zero(val);
return count;
}
}