2012-08-23 23:49:28 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-01-17 18:03:32 +00:00
|
|
|
#include <array>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <Common/Arena.h>
|
2017-11-09 23:37:57 +00:00
|
|
|
#include <Common/HashTable/Hash.h>
|
2018-09-02 21:28:25 +00:00
|
|
|
#include <Common/memcpySmall.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Defines.h>
|
2017-06-23 20:22:35 +00:00
|
|
|
#include <common/StringRef.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/IColumn.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
2018-09-27 15:55:22 +00:00
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2021-02-13 22:56:04 +00:00
|
|
|
#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
|
|
|
|
#include <tmmintrin.h>
|
|
|
|
#endif
|
2014-04-29 00:28:18 +00:00
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2020-02-25 18:10:48 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2016-05-28 10:35:44 +00:00
|
|
|
using Sizes = std::vector<size_t>;
|
2012-08-23 23:49:28 +00:00
|
|
|
|
2016-09-16 16:11:36 +00:00
|
|
|
/// When packing the values of nullable columns at a given row, we have to
|
|
|
|
/// store the fact that these values are nullable or not. This is achieved
|
|
|
|
/// by encoding this information as a bitmap. Let S be the size in bytes of
|
|
|
|
/// a packed values binary blob and T the number of bytes we may place into
|
|
|
|
/// this blob, the size that the bitmap shall occupy in the blob is equal to:
|
|
|
|
/// ceil(T/8). Thus we must have: S = T + ceil(T/8). Below we indicate for
|
|
|
|
/// each value of S, the corresponding value of T, and the bitmap size:
|
|
|
|
///
|
2016-10-18 10:09:48 +00:00
|
|
|
/// 32,28,4
|
2016-09-16 16:11:36 +00:00
|
|
|
/// 16,14,2
|
|
|
|
/// 8,7,1
|
|
|
|
/// 4,3,1
|
|
|
|
/// 2,1,1
|
|
|
|
///
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
constexpr auto getBitmapSize()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return
|
|
|
|
(sizeof(T) == 32) ?
|
|
|
|
4 :
|
|
|
|
(sizeof(T) == 16) ?
|
|
|
|
2 :
|
|
|
|
((sizeof(T) == 8) ?
|
|
|
|
1 :
|
|
|
|
((sizeof(T) == 4) ?
|
|
|
|
1 :
|
|
|
|
((sizeof(T) == 2) ?
|
|
|
|
1 :
|
|
|
|
0)));
|
2016-09-16 16:11:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
2012-08-23 23:49:28 +00:00
|
|
|
|
2021-02-25 15:51:01 +00:00
|
|
|
template<typename T, size_t step>
|
|
|
|
void fillFixedBatch(size_t num_rows, const T * source, T * dest)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < num_rows; ++i)
|
|
|
|
{
|
|
|
|
*dest = *source;
|
|
|
|
++source;
|
|
|
|
dest += step;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-10 11:50:51 +00:00
|
|
|
/// Move keys of size T into binary blob, starting from offset.
|
|
|
|
/// It is assumed that offset is aligned to sizeof(T).
|
|
|
|
/// Example: sizeof(key) = 16, sizeof(T) = 4, offset = 8
|
|
|
|
/// out[0] : [--------****----]
|
|
|
|
/// out[1] : [--------****----]
|
|
|
|
/// ...
|
2021-02-25 15:51:01 +00:00
|
|
|
template<typename T, typename Key>
|
|
|
|
void fillFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, PaddedPODArray<Key> & out, size_t & offset)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < keys_size; ++i)
|
|
|
|
{
|
|
|
|
if (key_sizes[i] == sizeof(T))
|
|
|
|
{
|
|
|
|
const auto * column = key_columns[i];
|
|
|
|
size_t num_rows = column->size();
|
2021-02-25 19:02:58 +00:00
|
|
|
out.resize_fill(num_rows);
|
2021-02-25 15:51:01 +00:00
|
|
|
|
2021-03-10 11:50:51 +00:00
|
|
|
/// Note: here we violate strict aliasing.
|
|
|
|
/// It should be ok as log as we do not reffer to any value from `out` before filling.
|
2021-02-25 15:51:01 +00:00
|
|
|
const char * source = static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<sizeof(T)>();
|
|
|
|
T * dest = reinterpret_cast<T *>(reinterpret_cast<char *>(out.data()) + offset);
|
|
|
|
fillFixedBatch<T, sizeof(Key) / sizeof(T)>(num_rows, reinterpret_cast<const T *>(source), dest);
|
|
|
|
offset += sizeof(T);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-10 11:50:51 +00:00
|
|
|
/// Pack into a binary blob of type T a set of fixed-size keys. Granted that all the keys fit into the
|
|
|
|
/// binary blob. Keys are placed starting from the longest one.
|
2021-02-25 15:51:01 +00:00
|
|
|
template <typename T>
|
|
|
|
void packFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, PaddedPODArray<T> & out)
|
|
|
|
{
|
|
|
|
size_t offset = 0;
|
2021-02-26 10:17:00 +00:00
|
|
|
fillFixedBatch<UInt128>(keys_size, key_columns, key_sizes, out, offset);
|
2021-02-25 15:51:01 +00:00
|
|
|
fillFixedBatch<UInt64>(keys_size, key_columns, key_sizes, out, offset);
|
|
|
|
fillFixedBatch<UInt32>(keys_size, key_columns, key_sizes, out, offset);
|
|
|
|
fillFixedBatch<UInt16>(keys_size, key_columns, key_sizes, out, offset);
|
|
|
|
fillFixedBatch<UInt8>(keys_size, key_columns, key_sizes, out, offset);
|
|
|
|
}
|
|
|
|
|
2016-09-16 16:11:36 +00:00
|
|
|
template <typename T>
|
|
|
|
using KeysNullMap = std::array<UInt8, getBitmapSize<T>()>;
|
|
|
|
|
|
|
|
/// Pack into a binary blob of type T a set of fixed-size keys. Granted that all the keys fit into the
|
|
|
|
/// binary blob, they are disposed in it consecutively.
|
2018-09-14 13:02:03 +00:00
|
|
|
template <typename T, bool has_low_cardinality = false>
|
2015-02-22 05:53:16 +00:00
|
|
|
static inline T ALWAYS_INLINE packFixed(
|
2018-09-14 13:02:03 +00:00
|
|
|
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
|
|
|
|
const ColumnRawPtrs * low_cardinality_positions [[maybe_unused]] = nullptr,
|
|
|
|
const Sizes * low_cardinality_sizes [[maybe_unused]] = nullptr)
|
2012-08-23 23:49:28 +00:00
|
|
|
{
|
2021-02-12 21:30:58 +00:00
|
|
|
T key{};
|
|
|
|
char * bytes = reinterpret_cast<char *>(&key);
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t offset = 0;
|
|
|
|
|
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
{
|
2018-09-14 13:02:03 +00:00
|
|
|
size_t index = i;
|
|
|
|
const IColumn * column = key_columns[j];
|
|
|
|
if constexpr (has_low_cardinality)
|
|
|
|
{
|
|
|
|
if (const IColumn * positions = (*low_cardinality_positions)[j])
|
|
|
|
{
|
|
|
|
switch ((*low_cardinality_sizes)[j])
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
case sizeof(UInt8): index = assert_cast<const ColumnUInt8 *>(positions)->getElement(i); break;
|
|
|
|
case sizeof(UInt16): index = assert_cast<const ColumnUInt16 *>(positions)->getElement(i); break;
|
|
|
|
case sizeof(UInt32): index = assert_cast<const ColumnUInt32 *>(positions)->getElement(i); break;
|
|
|
|
case sizeof(UInt64): index = assert_cast<const ColumnUInt64 *>(positions)->getElement(i); break;
|
2018-09-14 13:02:03 +00:00
|
|
|
default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
switch (key_sizes[j])
|
|
|
|
{
|
|
|
|
case 1:
|
2020-05-09 23:54:04 +00:00
|
|
|
{
|
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<1>() + index, 1);
|
|
|
|
offset += 1;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
case 2:
|
2020-05-09 23:54:04 +00:00
|
|
|
if constexpr (sizeof(T) >= 2) /// To avoid warning about memcpy exceeding object size.
|
|
|
|
{
|
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<2>() + index * 2, 2);
|
|
|
|
offset += 2;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
case 4:
|
2020-05-09 23:54:04 +00:00
|
|
|
if constexpr (sizeof(T) >= 4)
|
|
|
|
{
|
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<4>() + index * 4, 4);
|
|
|
|
offset += 4;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
case 8:
|
2020-05-09 23:54:04 +00:00
|
|
|
if constexpr (sizeof(T) >= 8)
|
|
|
|
{
|
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<8>() + index * 8, 8);
|
|
|
|
offset += 8;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
default:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(column)->getRawDataBegin<1>() + index * key_sizes[j], key_sizes[j]);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += key_sizes[j];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return key;
|
2016-09-16 16:11:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Similar as above but supports nullable values.
|
|
|
|
template <typename T>
|
|
|
|
static inline T ALWAYS_INLINE packFixed(
|
2017-12-13 01:27:53 +00:00
|
|
|
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, const Sizes & key_sizes,
|
2017-04-01 07:20:54 +00:00
|
|
|
const KeysNullMap<T> & bitmap)
|
2016-09-16 16:11:36 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
union
|
|
|
|
{
|
|
|
|
T key;
|
|
|
|
char bytes[sizeof(key)] = {};
|
|
|
|
};
|
|
|
|
|
|
|
|
size_t offset = 0;
|
|
|
|
|
|
|
|
static constexpr auto bitmap_size = std::tuple_size<KeysNullMap<T>>::value;
|
|
|
|
static constexpr bool has_bitmap = bitmap_size > 0;
|
|
|
|
|
|
|
|
if (has_bitmap)
|
|
|
|
{
|
|
|
|
memcpy(bytes + offset, bitmap.data(), bitmap_size * sizeof(UInt8));
|
|
|
|
offset += bitmap_size;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
{
|
|
|
|
bool is_null;
|
|
|
|
|
|
|
|
if (!has_bitmap)
|
|
|
|
is_null = false;
|
|
|
|
else
|
|
|
|
{
|
|
|
|
size_t bucket = j / 8;
|
|
|
|
size_t off = j % 8;
|
|
|
|
is_null = ((bitmap[bucket] >> off) & 1) == 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (is_null)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
switch (key_sizes[j])
|
|
|
|
{
|
|
|
|
case 1:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<1>() + i, 1);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += 1;
|
|
|
|
break;
|
|
|
|
case 2:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<2>() + i * 2, 2);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += 2;
|
|
|
|
break;
|
|
|
|
case 4:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<4>() + i * 4, 4);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += 4;
|
|
|
|
break;
|
|
|
|
case 8:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<8>() + i * 8, 8);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += 8;
|
|
|
|
break;
|
|
|
|
default:
|
2019-08-21 05:47:09 +00:00
|
|
|
memcpy(bytes + offset, static_cast<const ColumnVectorHelper *>(key_columns[j])->getRawDataBegin<1>() + i * key_sizes[j], key_sizes[j]);
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += key_sizes[j];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return key;
|
2013-06-30 16:56:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-09-16 16:11:36 +00:00
|
|
|
/// Hash a set of keys into a UInt128 value.
|
2014-10-06 22:48:20 +00:00
|
|
|
static inline UInt128 ALWAYS_INLINE hash128(
|
2017-12-13 01:27:53 +00:00
|
|
|
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns)
|
2013-02-17 23:28:52 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt128 key;
|
|
|
|
SipHash hash;
|
2013-02-17 23:28:52 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
key_columns[j]->updateHashWithValue(i, hash);
|
2013-02-17 23:28:52 +00:00
|
|
|
|
2021-01-27 00:54:57 +00:00
|
|
|
hash.get128(key);
|
2013-06-30 16:56:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return key;
|
2013-06-30 16:56:00 +00:00
|
|
|
}
|
|
|
|
|
2013-02-17 23:28:52 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Copy keys to the pool. Then put into pool StringRefs to them and return the pointer to the first.
|
2014-10-06 22:48:20 +00:00
|
|
|
static inline StringRef * ALWAYS_INLINE placeKeysInPool(
|
2017-12-01 17:49:12 +00:00
|
|
|
size_t keys_size, StringRefs & keys, Arena & pool)
|
2013-06-30 16:56:00 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
{
|
|
|
|
char * place = pool.alloc(keys[j].size);
|
2018-09-02 21:28:25 +00:00
|
|
|
memcpySmallAllowReadWriteOverflow15(place, keys[j].data, keys[j].size);
|
2017-04-01 07:20:54 +00:00
|
|
|
keys[j].data = place;
|
|
|
|
}
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Place the StringRefs on the newly copied keys in the pool.
|
2018-09-02 21:28:25 +00:00
|
|
|
char * res = pool.alignedAlloc(keys_size * sizeof(StringRef), alignof(StringRef));
|
|
|
|
memcpySmallAllowReadWriteOverflow15(res, keys.data(), keys_size * sizeof(StringRef));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return reinterpret_cast<StringRef *>(res);
|
2013-09-15 10:53:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Serialize keys into a continuous chunk of memory.
|
2015-10-04 06:10:48 +00:00
|
|
|
*/
|
|
|
|
static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous(
|
2017-12-13 01:27:53 +00:00
|
|
|
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, Arena & pool)
|
2015-10-04 06:10:48 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
const char * begin = nullptr;
|
2015-10-04 06:10:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t sum_size = 0;
|
|
|
|
for (size_t j = 0; j < keys_size; ++j)
|
|
|
|
sum_size += key_columns[j]->serializeValueIntoArena(i, pool, begin).size;
|
2015-10-04 06:10:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return {begin, sum_size};
|
2015-10-04 06:10:48 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-02-13 22:56:04 +00:00
|
|
|
/** Pack elements with shuffle instruction.
|
|
|
|
* See the explanation in ColumnsHashing.h
|
|
|
|
*/
|
|
|
|
#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
|
|
|
|
template <typename T>
|
2021-02-14 16:00:47 +00:00
|
|
|
static T inline packFixedShuffle(
|
2021-02-13 22:56:04 +00:00
|
|
|
const char * __restrict * __restrict srcs,
|
|
|
|
size_t num_srcs,
|
|
|
|
const size_t * __restrict elem_sizes,
|
|
|
|
size_t idx,
|
|
|
|
const uint8_t * __restrict masks)
|
|
|
|
{
|
2021-02-19 19:01:45 +00:00
|
|
|
assert(num_srcs > 0);
|
2021-02-13 22:56:04 +00:00
|
|
|
|
2021-02-19 19:01:45 +00:00
|
|
|
__m128i res = _mm_shuffle_epi8(
|
|
|
|
_mm_loadu_si128(reinterpret_cast<const __m128i *>(srcs[0] + elem_sizes[0] * idx)),
|
|
|
|
_mm_loadu_si128(reinterpret_cast<const __m128i *>(masks)));
|
|
|
|
|
|
|
|
for (size_t i = 1; i < num_srcs; ++i)
|
2021-02-13 22:56:04 +00:00
|
|
|
{
|
|
|
|
res = _mm_xor_si128(res,
|
|
|
|
_mm_shuffle_epi8(
|
|
|
|
_mm_loadu_si128(reinterpret_cast<const __m128i *>(srcs[i] + elem_sizes[i] * idx)),
|
|
|
|
_mm_loadu_si128(reinterpret_cast<const __m128i *>(&masks[i * sizeof(T)]))));
|
|
|
|
}
|
|
|
|
|
|
|
|
T out;
|
|
|
|
__builtin_memcpy(&out, &res, sizeof(T));
|
|
|
|
return out;
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|