ClickHouse/src/Columns/ColumnVector.cpp

645 lines
20 KiB
C++
Raw Normal View History

2018-11-16 12:22:51 +00:00
#include "ColumnVector.h"
#include <Columns/ColumnsCommon.h>
2021-02-12 00:25:00 +00:00
#include <Columns/ColumnCompressed.h>
#include <Columns/MaskOperations.h>
2021-10-08 14:03:54 +00:00
#include <Processors/Transforms/ColumnGathererTransform.h>
#include <IO/WriteHelpers.h>
#include <Common/Arena.h>
#include <Common/Exception.h>
#include <Common/HashTable/Hash.h>
#include <Common/NaNUtils.h>
2019-02-18 16:29:37 +00:00
#include <Common/RadixSort.h>
#include <Common/SipHash.h>
2020-03-13 17:31:50 +00:00
#include <Common/WeakHash.h>
#include <Common/assert_cast.h>
2021-10-02 07:13:14 +00:00
#include <base/sort.h>
#include <base/unaligned.h>
#include <base/bit_cast.h>
#include <base/scope_guard.h>
2017-03-11 01:29:45 +00:00
#include <cmath>
#include <cstring>
#if defined(__SSE2__)
# include <emmintrin.h>
2017-03-11 01:12:51 +00:00
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
2020-03-19 17:35:08 +00:00
extern const int LOGICAL_ERROR;
2021-05-03 23:46:11 +00:00
extern const int NOT_IMPLEMENTED;
2017-03-11 01:12:51 +00:00
}
2018-08-07 13:57:28 +00:00
template <typename T>
StringRef ColumnVector<T>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
2017-03-11 01:12:51 +00:00
{
2021-01-26 18:22:40 +00:00
auto * pos = arena.allocContinue(sizeof(T), begin);
unalignedStore<T>(pos, data[n]);
return StringRef(pos, sizeof(T));
2017-03-11 01:12:51 +00:00
}
2018-08-07 13:57:28 +00:00
template <typename T>
const char * ColumnVector<T>::deserializeAndInsertFromArena(const char * pos)
2017-03-11 01:12:51 +00:00
{
2021-01-26 18:22:40 +00:00
data.emplace_back(unalignedLoad<T>(pos));
return pos + sizeof(T);
2017-03-11 01:12:51 +00:00
}
template <typename T>
const char * ColumnVector<T>::skipSerializedInArena(const char * pos) const
{
return pos + sizeof(T);
}
2018-08-07 13:57:28 +00:00
template <typename T>
void ColumnVector<T>::updateHashWithValue(size_t n, SipHash & hash) const
2017-03-11 01:12:51 +00:00
{
2018-03-03 15:36:20 +00:00
hash.update(data[n]);
2017-03-11 01:12:51 +00:00
}
2020-03-13 17:31:50 +00:00
template <typename T>
void ColumnVector<T>::updateWeakHash32(WeakHash32 & hash) const
{
auto s = data.size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
2020-03-25 11:14:11 +00:00
const T * begin = data.data();
2020-03-13 17:31:50 +00:00
const T * end = begin + s;
2020-03-25 11:14:11 +00:00
UInt32 * hash_data = hash.getData().data();
2020-03-13 17:31:50 +00:00
while (begin < end)
{
2020-03-23 19:18:35 +00:00
*hash_data = intHashCRC32(*begin, *hash_data);
2020-03-13 17:31:50 +00:00
++begin;
++hash_data;
}
}
template <typename T>
void ColumnVector<T>::updateHashFast(SipHash & hash) const
{
hash.update(reinterpret_cast<const char *>(data.data()), size() * sizeof(data[0]));
}
2018-08-07 13:57:28 +00:00
template <typename T>
struct ColumnVector<T>::less
2017-03-11 01:12:51 +00:00
{
const Self & parent;
int nan_direction_hint;
less(const Self & parent_, int nan_direction_hint_) : parent(parent_), nan_direction_hint(nan_direction_hint_) {}
bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::less(parent.data[lhs], parent.data[rhs], nan_direction_hint); }
2017-03-11 01:12:51 +00:00
};
2018-08-07 13:57:28 +00:00
template <typename T>
struct ColumnVector<T>::greater
2017-03-11 01:12:51 +00:00
{
const Self & parent;
int nan_direction_hint;
greater(const Self & parent_, int nan_direction_hint_) : parent(parent_), nan_direction_hint(nan_direction_hint_) {}
bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::greater(parent.data[lhs], parent.data[rhs], nan_direction_hint); }
2017-03-11 01:12:51 +00:00
};
template <typename T>
struct ColumnVector<T>::equals
{
const Self & parent;
int nan_direction_hint;
equals(const Self & parent_, int nan_direction_hint_) : parent(parent_), nan_direction_hint(nan_direction_hint_) {}
bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); }
};
2019-04-25 01:16:26 +00:00
namespace
{
template <typename T>
struct ValueWithIndex
{
T value;
UInt32 index;
};
template <typename T>
struct RadixSortTraits : RadixSortNumTraits<T>
{
using Element = ValueWithIndex<T>;
2020-05-23 14:28:05 +00:00
using Result = size_t;
2020-05-23 15:22:04 +00:00
2019-04-25 01:16:26 +00:00
static T & extractKey(Element & elem) { return elem.value; }
2020-05-23 14:28:05 +00:00
static size_t extractResult(Element & elem) { return elem.index; }
2019-04-25 01:16:26 +00:00
};
}
2020-05-18 21:41:23 +00:00
2018-08-07 13:57:28 +00:00
template <typename T>
void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const
2017-03-11 01:12:51 +00:00
{
size_t s = data.size();
res.resize(s);
2019-04-25 01:16:26 +00:00
if (s == 0)
return;
if (limit >= s)
limit = 0;
if (limit)
{
2019-02-18 16:29:37 +00:00
for (size_t i = 0; i < s; ++i)
res[i] = i;
if (reverse)
2022-01-30 19:49:48 +00:00
::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint));
else
2022-01-30 19:49:48 +00:00
::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint));
}
else
{
2019-04-25 01:16:26 +00:00
/// A case for radix sort
2021-05-06 00:31:09 +00:00
if constexpr (is_arithmetic_v<T> && !is_big_int_v<T>)
2019-02-18 16:29:37 +00:00
{
2019-04-25 01:16:26 +00:00
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
if (s >= 256 && s <= std::numeric_limits<UInt32>::max())
{
PaddedPODArray<ValueWithIndex<T>> pairs(s);
2020-03-18 16:50:08 +00:00
for (UInt32 i = 0; i < UInt32(s); ++i)
2019-04-25 01:16:26 +00:00
pairs[i] = {data[i], i};
RadixSort<RadixSortTraits<T>>::executeLSD(pairs.data(), s, reverse, res.data());
2019-04-25 01:16:26 +00:00
/// Radix sort treats all NaNs to be greater than all numbers.
/// If the user needs the opposite, we must move them accordingly.
if (std::is_floating_point_v<T> && nan_direction_hint < 0)
{
size_t nans_to_move = 0;
for (size_t i = 0; i < s; ++i)
2019-04-25 01:16:26 +00:00
{
if (isNaN(data[res[reverse ? i : s - 1 - i]]))
2019-04-25 01:16:26 +00:00
++nans_to_move;
else
break;
}
if (nans_to_move)
{
2020-05-21 14:10:47 +00:00
std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res));
2019-04-25 01:16:26 +00:00
}
}
return;
}
2019-02-18 16:29:37 +00:00
}
2019-04-25 01:16:26 +00:00
/// Default sorting algorithm.
for (size_t i = 0; i < s; ++i)
res[i] = i;
if (reverse)
2022-01-30 19:49:48 +00:00
::sort(res.begin(), res.end(), greater(*this, nan_direction_hint));
2019-04-25 01:16:26 +00:00
else
2022-01-30 19:49:48 +00:00
::sort(res.begin(), res.end(), less(*this, nan_direction_hint));
}
2017-03-11 01:12:51 +00:00
}
2020-05-12 00:58:58 +00:00
template <typename T>
2020-05-25 14:33:31 +00:00
void ColumnVector<T>::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const
2020-05-12 00:58:58 +00:00
{
2022-01-30 19:49:48 +00:00
auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); };
2021-10-01 16:43:58 +00:00
auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); };
if (reverse)
2021-10-01 16:43:58 +00:00
this->updatePermutationImpl(
limit, res, equal_range,
greater(*this, nan_direction_hint),
equals(*this, nan_direction_hint),
sort, partial_sort);
else
2021-10-01 16:43:58 +00:00
this->updatePermutationImpl(
limit, res, equal_range,
less(*this, nan_direction_hint),
equals(*this, nan_direction_hint),
sort, partial_sort);
2020-05-12 00:58:58 +00:00
}
2018-08-07 13:57:28 +00:00
template <typename T>
MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
2017-03-11 01:12:51 +00:00
{
auto res = this->create();
2017-03-11 01:12:51 +00:00
if (size > 0)
{
auto & new_col = static_cast<Self &>(*res);
new_col.data.resize(size);
2017-03-11 01:12:51 +00:00
size_t count = std::min(this->size(), size);
2021-01-26 18:22:40 +00:00
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
2017-03-11 01:12:51 +00:00
2021-01-26 18:22:40 +00:00
if (size > count)
2021-05-03 22:46:51 +00:00
memset(static_cast<void *>(&new_col.data[count]), 0, (size - count) * sizeof(ValueType));
}
2017-03-11 01:12:51 +00:00
return res;
2017-03-11 01:12:51 +00:00
}
2018-08-07 13:57:28 +00:00
template <typename T>
2021-05-06 22:57:41 +00:00
UInt64 ColumnVector<T>::get64(size_t n [[maybe_unused]]) const
2017-03-11 01:12:51 +00:00
{
2021-05-09 21:26:34 +00:00
if constexpr (is_arithmetic_v<T>)
2021-06-15 19:55:21 +00:00
return bit_cast<UInt64>(data[n]);
2021-05-03 22:46:51 +00:00
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as UInt64", TypeName<T>);
2017-03-11 01:12:51 +00:00
}
template <typename T>
2021-05-06 22:57:41 +00:00
inline Float64 ColumnVector<T>::getFloat64(size_t n [[maybe_unused]]) const
{
2021-05-09 21:26:34 +00:00
if constexpr (is_arithmetic_v<T>)
2021-05-03 22:46:51 +00:00
return static_cast<Float64>(data[n]);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float64", TypeName<T>);
}
template <typename T>
2021-05-06 22:57:41 +00:00
Float32 ColumnVector<T>::getFloat32(size_t n [[maybe_unused]]) const
{
2021-05-09 21:26:34 +00:00
if constexpr (is_arithmetic_v<T>)
2021-05-03 22:46:51 +00:00
return static_cast<Float32>(data[n]);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float32", TypeName<T>);
}
2018-08-07 13:57:28 +00:00
template <typename T>
void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t length)
2017-03-11 01:12:51 +00:00
{
const ColumnVector & src_vec = assert_cast<const ColumnVector &>(src);
if (start + length > src_vec.data.size())
throw Exception("Parameters start = "
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in ColumnVector<T>::insertRangeFrom method"
" (data.size() = " + toString(src_vec.data.size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t old_size = data.size();
data.resize(old_size + length);
2021-01-26 18:22:40 +00:00
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
2017-03-11 01:12:51 +00:00
}
2018-08-07 13:57:28 +00:00
template <typename T>
2021-08-10 11:31:15 +00:00
ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
2017-03-11 01:12:51 +00:00
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
2017-03-11 01:12:51 +00:00
auto res = this->create();
Container & res_data = res->getData();
2017-03-11 01:12:51 +00:00
if (result_size_hint)
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
2017-03-11 01:12:51 +00:00
2021-01-26 18:22:40 +00:00
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = data.data();
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.
* Therefore, we will optimistically check the parts of `SIMD_BYTES` values.
*/
static constexpr size_t SIMD_BYTES = 64;
const UInt8 * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
while (filt_pos < filt_end_aligned)
2021-10-12 02:54:47 +00:00
{
2021-11-20 12:58:44 +00:00
UInt64 mask = bytes64MaskToBits64Mask(filt_pos);
if (0xffffffffffffffff == mask)
{
res_data.insert(data_pos, data_pos + SIMD_BYTES);
}
else
{
while (mask)
{
size_t index = __builtin_ctzll(mask);
res_data.push_back(data_pos[index]);
#ifdef __BMI__
mask = _blsr_u64(mask);
#else
mask = mask & (mask-1);
2021-10-12 02:54:47 +00:00
#endif
}
}
filt_pos += SIMD_BYTES;
data_pos += SIMD_BYTES;
}
2021-01-26 18:22:40 +00:00
while (filt_pos < filt_end)
{
2021-08-10 11:31:15 +00:00
if (*filt_pos)
2021-01-26 18:22:40 +00:00
res_data.push_back(*data_pos);
2017-03-11 01:12:51 +00:00
2021-01-26 18:22:40 +00:00
++filt_pos;
++data_pos;
}
2017-03-11 01:12:51 +00:00
return res;
2017-03-11 01:12:51 +00:00
}
template <typename T>
void ColumnVector<T>::expand(const IColumn::Filter & mask, bool inverted)
{
expandDataByMask<T>(data, mask, inverted);
}
2020-06-23 21:06:32 +00:00
template <typename T>
void ColumnVector<T>::applyZeroMap(const IColumn::Filter & filt, bool inverted)
{
size_t size = data.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
const UInt8 * filt_pos = filt.data();
const UInt8 * filt_end = filt_pos + size;
T * data_pos = data.data();
if (inverted)
{
for (; filt_pos < filt_end; ++filt_pos, ++data_pos)
if (!*filt_pos)
*data_pos = 0;
}
else
{
for (; filt_pos < filt_end; ++filt_pos, ++data_pos)
if (*filt_pos)
*data_pos = 0;
}
}
2018-08-07 13:57:28 +00:00
template <typename T>
ColumnPtr ColumnVector<T>::permute(const IColumn::Permutation & perm, size_t limit) const
2017-03-11 01:12:51 +00:00
{
2021-09-29 17:51:58 +00:00
return permuteImpl(*this, perm, limit);
2017-03-11 01:12:51 +00:00
}
2018-04-18 21:00:47 +00:00
template <typename T>
ColumnPtr ColumnVector<T>::index(const IColumn & indexes, size_t limit) const
2018-04-18 21:00:47 +00:00
{
return selectIndexImpl(*this, indexes, limit);
}
2018-08-07 13:57:28 +00:00
template <typename T>
ColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets & offsets) const
2017-03-11 01:12:51 +00:00
{
#ifdef __SSE4_2__
if constexpr (std::is_same_v<T, UInt32>)
{
return replicateSSE2(offsets);
}
#endif
const size_t size = data.size();
if (size != offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
2017-03-11 01:12:51 +00:00
if (0 == size)
return this->create();
2017-03-11 01:12:51 +00:00
2020-02-21 19:47:20 +00:00
auto res = this->create(offsets.back());
2017-03-11 01:12:51 +00:00
auto it = res->getData().begin(); // NOLINT
for (size_t i = 0; i < size; ++i)
{
const auto span_end = res->getData().begin() + offsets[i]; // NOLINT
for (; it != span_end; ++it)
*it = data[i];
}
2017-03-11 01:12:51 +00:00
return res;
2017-03-11 01:12:51 +00:00
}
#ifdef __SSE4_2__
template <typename T>
ColumnPtr ColumnVector<T>::replicateSSE2(const IColumn::Offsets & offsets) const
{
const size_t size = data.size();
if (size != offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (0 == size)
return this->create();
auto res = this->create(offsets.back());
auto it = res->getData().begin(); // NOLINT
2022-05-16 15:31:54 +00:00
///column using padded pod array. Don't worry about the 4 conitnues op will out of range
if constexpr (std::is_same_v<T, UInt32>)
{
size_t prev_offset = 0;
int cp_begin = -1;
for (size_t i = 0; i < size; ++i)
{
size_t span = offsets[i] - prev_offset;
prev_offset = offsets[i];
if (span == 1)
{
if (cp_begin == -1)
cp_begin = i;
continue;
}
///data : 11 22 33 44 55
///offsets: 0 1 2 3 3
///res: 22 33 44
size_t cpsz = (!(cp_begin == -1)) * (i - cp_begin);
bool remain = (cpsz & 3);
size_t sse_cp_counter = (cpsz >> 2);
sse_cp_counter = remain * (sse_cp_counter + 1) + (!remain) * (sse_cp_counter);
auto it_tmp = it;
size_t data_start = cp_begin;
cp_begin = -1;
constexpr const int msk_cp = (_MM_SHUFFLE(3, 2, 1, 0));
while (sse_cp_counter--)
{
__m128i cdata = _mm_loadu_si128(reinterpret_cast<const __m128i *>(&data[data_start]));
auto cres = _mm_shuffle_epi32(cdata, msk_cp);
_mm_storeu_si128(reinterpret_cast<__m128i *>(it_tmp), cres);
it_tmp += 4;
data_start += 4;
}
it += cpsz;
if (span == 0)
{
continue;
}
///data : 11 22 33
///offsets: 0 0 4
///res: 33 33 33 33
size_t shuffle_sz = span;
bool shuffle_remain = (shuffle_sz & 3);
size_t sse_shuffle_counter = (shuffle_sz >> 2);
sse_shuffle_counter = shuffle_remain * (sse_shuffle_counter + 1) + (!shuffle_remain) * (sse_shuffle_counter);
it_tmp = it;
constexpr const int msk_shuffle = (_MM_SHUFFLE(0, 0, 0, 0));
__m128i cdata = _mm_loadu_si128(reinterpret_cast<const __m128i *>(&data[i]));
while (sse_shuffle_counter--)
{
auto cres = _mm_shuffle_epi32(cdata, msk_shuffle);
_mm_storeu_si128(reinterpret_cast<__m128i *>(it_tmp), cres);
it_tmp += 4;
}
it += shuffle_sz;
}
///data : 11 22 33 44 55
///offsets: 1 2 3 4 5
///res: 11 22 33 44 55
if (cp_begin != -1)
{
size_t cpsz = (size - cp_begin);
bool remain = (cpsz & 3);
size_t sse_cp_counter = (cpsz >> 2);
sse_cp_counter = remain * (sse_cp_counter + 1) + (!remain) * (sse_cp_counter);
auto it_tmp = it;
size_t data_start = cp_begin;
constexpr const int msk_cp = (_MM_SHUFFLE(3, 2, 1, 0));
while (sse_cp_counter--)
{
__m128i cdata = _mm_loadu_si128(reinterpret_cast<const __m128i *>(&data[data_start]));
auto cres = _mm_shuffle_epi32(cdata, msk_cp);
_mm_storeu_si128(reinterpret_cast<__m128i *>(it_tmp), cres);
it_tmp += 4;
data_start += 4;
}
it += cpsz;
}
}
return res;
}
#endif
2018-08-07 13:57:28 +00:00
template <typename T>
void ColumnVector<T>::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
}
2018-08-07 13:57:28 +00:00
template <typename T>
void ColumnVector<T>::getExtremes(Field & min, Field & max) const
2017-03-11 01:12:51 +00:00
{
size_t size = data.size();
if (size == 0)
{
min = T(0);
max = T(0);
return;
}
bool has_value = false;
/** Skip all NaNs in extremes calculation.
* If all values are NaNs, then return NaN.
* NOTE: There exist many different NaNs.
* Different NaN could be returned: not bit-exact value as one of NaNs from column.
*/
T cur_min = NaNOrZero<T>();
T cur_max = NaNOrZero<T>();
2021-01-26 19:39:03 +00:00
for (const T & x : data)
{
if (isNaN(x))
continue;
if (!has_value)
{
cur_min = x;
cur_max = x;
has_value = true;
continue;
}
if (x < cur_min)
cur_min = x;
else if (x > cur_max)
cur_max = x;
}
2018-11-20 20:09:20 +00:00
min = NearestFieldType<T>(cur_min);
max = NearestFieldType<T>(cur_max);
2017-03-11 01:12:51 +00:00
}
2021-02-07 01:41:31 +00:00
#pragma GCC diagnostic ignored "-Wold-style-cast"
template <typename T>
2021-02-12 00:25:00 +00:00
ColumnPtr ColumnVector<T>::compress() const
2021-02-07 01:41:31 +00:00
{
const size_t data_size = data.size();
const size_t source_size = data_size * sizeof(T);
2021-02-10 18:53:31 +00:00
/// Don't compress small blocks.
if (source_size < 4096) /// A wild guess.
2021-02-12 00:25:00 +00:00
return ColumnCompressed::wrap(this->getPtr());
2021-02-10 18:53:31 +00:00
2021-02-18 00:52:09 +00:00
auto compressed = ColumnCompressed::compressBuffer(data.data(), source_size, false);
2021-02-07 01:41:31 +00:00
2021-02-12 00:25:00 +00:00
if (!compressed)
return ColumnCompressed::wrap(this->getPtr());
2021-02-07 01:41:31 +00:00
const size_t compressed_size = compressed->size();
return ColumnCompressed::create(data_size, compressed_size,
[compressed = std::move(compressed), column_size = data_size]
2021-02-12 00:25:00 +00:00
{
auto res = ColumnVector<T>::create(column_size);
ColumnCompressed::decompressBuffer(
compressed->data(), res->getData().data(), compressed->size(), column_size * sizeof(T));
return res;
});
2021-02-07 01:41:31 +00:00
}
2021-04-01 18:18:28 +00:00
template <typename T>
2021-09-16 13:57:45 +00:00
ColumnPtr ColumnVector<T>::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const
2021-04-01 18:18:28 +00:00
{
2021-05-21 00:57:11 +00:00
if (offsets.size() + shift != size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Incompatible sizes of offsets ({}), shift ({}) and size of column {}", offsets.size(), shift, size());
2021-04-01 18:18:28 +00:00
auto res = this->create();
auto & res_data = res->getData();
2021-09-16 13:57:45 +00:00
T default_value = safeGet<T>(default_field);
res_data.resize_fill(total_rows, default_value);
2021-04-01 18:18:28 +00:00
for (size_t i = 0; i < offsets.size(); ++i)
2021-05-21 00:57:11 +00:00
res_data[offsets[i]] = data[i + shift];
2021-04-01 18:18:28 +00:00
return res;
}
/// Explicit template instantiations - to avoid code bloat in headers.
2018-08-07 13:57:28 +00:00
template class ColumnVector<UInt8>;
template class ColumnVector<UInt16>;
template class ColumnVector<UInt32>;
template class ColumnVector<UInt64>;
template class ColumnVector<UInt128>;
template class ColumnVector<UInt256>;
2018-08-07 13:57:28 +00:00
template class ColumnVector<Int8>;
template class ColumnVector<Int16>;
template class ColumnVector<Int32>;
template class ColumnVector<Int64>;
template class ColumnVector<Int128>;
template class ColumnVector<Int256>;
2018-08-07 13:57:28 +00:00
template class ColumnVector<Float32>;
template class ColumnVector<Float64>;
2021-05-03 22:46:51 +00:00
template class ColumnVector<UUID>;
2021-02-11 21:54:50 +00:00
2017-03-11 01:12:51 +00:00
}