#include "ColumnVector.h" #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #if defined(__SSE2__) # include #endif #if USE_EMBEDDED_COMPILER #include #include #endif namespace DB { namespace ErrorCodes { extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; } template StringRef ColumnVector::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { auto * pos = arena.allocContinue(sizeof(T), begin); unalignedStore(pos, data[n]); return StringRef(pos, sizeof(T)); } template const char * ColumnVector::deserializeAndInsertFromArena(const char * pos) { data.emplace_back(unalignedLoad(pos)); return pos + sizeof(T); } template const char * ColumnVector::skipSerializedInArena(const char * pos) const { return pos + sizeof(T); } template void ColumnVector::updateHashWithValue(size_t n, SipHash & hash) const { hash.update(data[n]); } template void ColumnVector::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); const T * begin = data.data(); const T * end = begin + s; UInt32 * hash_data = hash.getData().data(); while (begin < end) { *hash_data = intHashCRC32(*begin, *hash_data); ++begin; ++hash_data; } } template void ColumnVector::updateHashFast(SipHash & hash) const { hash.update(reinterpret_cast(data.data()), size() * sizeof(data[0])); } template struct ColumnVector::less { 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::less(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; template struct ColumnVector::less_stable { const Self & parent; int nan_direction_hint; less_stable(const Self & parent_, int nan_direction_hint_) : parent(parent_), nan_direction_hint(nan_direction_hint_) {} bool operator()(size_t lhs, size_t rhs) const { if (unlikely(parent.data[lhs] == parent.data[rhs])) return lhs < rhs; if constexpr (std::is_floating_point_v) { if (unlikely(std::isnan(parent.data[lhs]) && std::isnan(parent.data[rhs]))) { return lhs < rhs; } } return CompareHelper::less(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; template struct ColumnVector::greater { 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::greater(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; template struct ColumnVector::greater_stable { const Self & parent; int nan_direction_hint; greater_stable(const Self & parent_, int nan_direction_hint_) : parent(parent_), nan_direction_hint(nan_direction_hint_) {} bool operator()(size_t lhs, size_t rhs) const { if (unlikely(parent.data[lhs] == parent.data[rhs])) return lhs < rhs; if constexpr (std::is_floating_point_v) { if (unlikely(std::isnan(parent.data[lhs]) && std::isnan(parent.data[rhs]))) { return lhs < rhs; } } return CompareHelper::greater(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; template struct ColumnVector::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::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); } }; namespace { template struct ValueWithIndex { T value; UInt32 index; }; template struct RadixSortTraits : RadixSortNumTraits { using Element = ValueWithIndex; using Result = size_t; static T & extractKey(Element & elem) { return elem.value; } static size_t extractResult(Element & elem) { return elem.index; } }; } #if USE_EMBEDDED_COMPILER template bool ColumnVector::isComparatorCompilable() const { /// TODO: for std::is_floating_point_v we need implement is_nan in LLVM IR. return std::is_integral_v; } template llvm::Value * ColumnVector::compileComparator(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs, llvm::Value *) const { llvm::IRBuilder<> & b = static_cast &>(builder); if constexpr (std::is_integral_v) { // a > b ? 1 : (a < b ? -1 : 0); bool is_signed = std::is_signed_v; auto * lhs_greater_than_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), 1); auto * lhs_less_than_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), -1); auto * lhs_equals_rhs_result = llvm::ConstantInt::getSigned(b.getInt8Ty(), 0); auto * lhs_greater_than_rhs = is_signed ? b.CreateICmpSGT(lhs, rhs) : b.CreateICmpUGT(lhs, rhs); auto * lhs_less_than_rhs = is_signed ? b.CreateICmpSLT(lhs, rhs) : b.CreateICmpULT(lhs, rhs); auto * if_lhs_less_than_rhs_result = b.CreateSelect(lhs_less_than_rhs, lhs_less_than_rhs_result, lhs_equals_rhs_result); return b.CreateSelect(lhs_greater_than_rhs, lhs_greater_than_rhs_result, if_lhs_less_than_rhs_result); } else { throw Exception(ErrorCodes::LOGICAL_ERROR, "Method compileComparator is not supported for type {}", TypeName); } } #endif template void ColumnVector::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const { size_t s = data.size(); res.resize(s); if (s == 0) return; if (limit >= s) limit = 0; if (limit) { for (size_t i = 0; i < s; ++i) res[i] = i; if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) ::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) ::partial_sort(res.begin(), res.begin() + limit, res.end(), less_stable(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) ::partial_sort(res.begin(), res.begin() + limit, res.end(), greater_stable(*this, nan_direction_hint)); } else { /// A case for radix sort /// LSD RadixSort is stable if constexpr (is_arithmetic_v && !is_big_int_v) { bool reverse = direction == IColumn::PermutationSortDirection::Descending; bool ascending = direction == IColumn::PermutationSortDirection::Ascending; bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable; /// TODO: LSD RadixSort is currently not stable if direction is descending, or value is floating point bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v) || !sort_is_stable; /// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters. if (s >= 256 && s <= std::numeric_limits::max() && use_radix_sort) { PaddedPODArray> pairs(s); for (UInt32 i = 0; i < static_cast(s); ++i) pairs[i] = {data[i], i}; RadixSort>::executeLSD(pairs.data(), s, reverse, res.data()); /// 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 && nan_direction_hint < 0) { size_t nans_to_move = 0; for (size_t i = 0; i < s; ++i) { if (isNaN(data[res[reverse ? i : s - 1 - i]])) ++nans_to_move; else break; } if (nans_to_move) { std::rotate(std::begin(res), std::begin(res) + (reverse ? nans_to_move : s - nans_to_move), std::end(res)); } } return; } } /// Default sorting algorithm. for (size_t i = 0; i < s; ++i) res[i] = i; if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) ::sort(res.begin(), res.end(), less(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) ::sort(res.begin(), res.end(), less_stable(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) ::sort(res.begin(), res.end(), greater(*this, nan_direction_hint)); else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) ::sort(res.begin(), res.end(), greater_stable(*this, nan_direction_hint)); } } template void ColumnVector::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); }; auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); }; if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable) { this->updatePermutationImpl( limit, res, equal_ranges, less(*this, nan_direction_hint), equals(*this, nan_direction_hint), sort, partial_sort); } else if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Stable) { this->updatePermutationImpl( limit, res, equal_ranges, less_stable(*this, nan_direction_hint), equals(*this, nan_direction_hint), sort, partial_sort); } else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable) { this->updatePermutationImpl( limit, res, equal_ranges, greater(*this, nan_direction_hint), equals(*this, nan_direction_hint), sort, partial_sort); } else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable) { this->updatePermutationImpl( limit, res, equal_ranges, greater_stable(*this, nan_direction_hint), equals(*this, nan_direction_hint), sort, partial_sort); } } template MutableColumnPtr ColumnVector::cloneResized(size_t size) const { auto res = this->create(); if (size > 0) { auto & new_col = static_cast(*res); new_col.data.resize(size); size_t count = std::min(this->size(), size); memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); if (size > count) memset(static_cast(&new_col.data[count]), 0, (size - count) * sizeof(ValueType)); } return res; } template UInt64 ColumnVector::get64(size_t n [[maybe_unused]]) const { if constexpr (is_arithmetic_v) return bit_cast(data[n]); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as UInt64", TypeName); } template inline Float64 ColumnVector::getFloat64(size_t n [[maybe_unused]]) const { if constexpr (is_arithmetic_v) return static_cast(data[n]); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float64", TypeName); } template Float32 ColumnVector::getFloat32(size_t n [[maybe_unused]]) const { if constexpr (is_arithmetic_v) return static_cast(data[n]); else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float32", TypeName); } template void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnVector & src_vec = assert_cast(src); if (start + length > src_vec.data.size()) throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length) + " are out of bound in ColumnVector::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); memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0])); } template ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const { size_t size = data.size(); if (size != filt.size()) throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size); auto res = this->create(); Container & res_data = res->getData(); if (result_size_hint) res_data.reserve(result_size_hint > 0 ? result_size_hint : size); 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) { 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); #endif } } filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } while (filt_pos < filt_end) { if (*filt_pos) res_data.push_back(*data_pos); ++filt_pos; ++data_pos; } return res; } template void ColumnVector::expand(const IColumn::Filter & mask, bool inverted) { expandDataByMask(data, mask, inverted); } template void ColumnVector::applyZeroMap(const IColumn::Filter & filt, bool inverted) { size_t size = data.size(); if (size != filt.size()) throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Size of filter ({}) doesn't match size of column ({})", filt.size(), size); 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; } } template ColumnPtr ColumnVector::permute(const IColumn::Permutation & perm, size_t limit) const { return permuteImpl(*this, perm, limit); } template ColumnPtr ColumnVector::index(const IColumn & indexes, size_t limit) const { return selectIndexImpl(*this, indexes, limit); } template ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const { #ifdef __SSE2__ if constexpr (std::is_same_v) { 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); if (0 == size) return this->create(); auto res = this->create(offsets.back()); 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]; } return res; } #ifdef __SSE2__ template ColumnPtr ColumnVector::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 ///Column using padded pod array. Don't worry about the 4 conitnues op will out of range if constexpr (std::is_same_v) { 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(&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(&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(&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 template void ColumnVector::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); } template void ColumnVector::getExtremes(Field & min, Field & max) const { 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 cur_max = NaNOrZero(); 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; } min = NearestFieldType(cur_min); max = NearestFieldType(cur_max); } #pragma GCC diagnostic ignored "-Wold-style-cast" template ColumnPtr ColumnVector::compress() const { const size_t data_size = data.size(); const size_t source_size = data_size * sizeof(T); /// Don't compress small blocks. if (source_size < 4096) /// A wild guess. return ColumnCompressed::wrap(this->getPtr()); auto compressed = ColumnCompressed::compressBuffer(data.data(), source_size, false); if (!compressed) return ColumnCompressed::wrap(this->getPtr()); const size_t compressed_size = compressed->size(); return ColumnCompressed::create(data_size, compressed_size, [compressed = std::move(compressed), column_size = data_size] { auto res = ColumnVector::create(column_size); ColumnCompressed::decompressBuffer( compressed->data(), res->getData().data(), compressed->size(), column_size * sizeof(T)); return res; }); } template ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible sizes of offsets ({}), shift ({}) and size of column {}", offsets.size(), shift, size()); auto res = this->create(); auto & res_data = res->getData(); T default_value = safeGet(default_field); res_data.resize_fill(total_rows, default_value); for (size_t i = 0; i < offsets.size(); ++i) res_data[offsets[i]] = data[i + shift]; return res; } /// Explicit template instantiations - to avoid code bloat in headers. template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; }