diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 3e56441479b..fb5d603d6e8 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -380,6 +380,7 @@ if(ARROW_COMPUTE) compute/cast.cc compute/exec.cc compute/exec/aggregate_node.cc + compute/exec/bloom_filter.cc compute/exec/exec_plan.cc compute/exec/expression.cc compute/exec/filter_node.cc @@ -393,6 +394,7 @@ if(ARROW_COMPUTE) compute/exec/key_map.cc compute/exec/options.cc compute/exec/order_by_impl.cc + compute/exec/partition_util.cc compute/exec/project_node.cc compute/exec/sink_node.cc compute/exec/source_node.cc @@ -441,6 +443,7 @@ if(ARROW_COMPUTE) append_avx2_src(compute/kernels/aggregate_basic_avx2.cc) append_avx512_src(compute/kernels/aggregate_basic_avx512.cc) + append_avx2_src(compute/exec/bloom_filter_avx2.cc) append_avx2_src(compute/exec/key_compare_avx2.cc) append_avx2_src(compute/exec/key_encode_avx2.cc) append_avx2_src(compute/exec/key_hash_avx2.cc) diff --git a/cpp/src/arrow/compute/exec/CMakeLists.txt b/cpp/src/arrow/compute/exec/CMakeLists.txt index 3832273593d..53ba77b6088 100644 --- a/cpp/src/arrow/compute/exec/CMakeLists.txt +++ b/cpp/src/arrow/compute/exec/CMakeLists.txt @@ -25,9 +25,14 @@ add_arrow_compute_test(expression_test subtree_test.cc) add_arrow_compute_test(plan_test PREFIX "arrow-compute") -add_arrow_compute_test(hash_join_node_test PREFIX "arrow-compute") +add_arrow_compute_test(hash_join_node_test + PREFIX + "arrow-compute" + SOURCES + hash_join_node_test.cc + bloom_filter_test.cc + key_hash_test.cc) add_arrow_compute_test(union_node_test PREFIX "arrow-compute") - add_arrow_compute_test(util_test PREFIX "arrow-compute") add_arrow_benchmark(expression_benchmark PREFIX "arrow-compute") diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc new file mode 100644 index 00000000000..61031725457 --- /dev/null +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -0,0 +1,434 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/bloom_filter.h" +#include +#include "arrow/compute/exec/util.h" // PREFETCH +#include "arrow/util/bit_util.h" // Log2 +#include "arrow/util/bitmap_ops.h" // CountSetBits + +namespace arrow { +namespace compute { + +BloomFilterMasks::BloomFilterMasks() { + std::seed_seq seed{0, 0, 0, 0, 0, 0, 0, 0}; + std::mt19937 re(seed); + std::uniform_int_distribution rd; + auto random = [&re, &rd](int min_value, int max_value) { + return min_value + rd(re) % (max_value - min_value + 1); + }; + + memset(masks_, 0, kTotalBytes); + + // Prepare the first mask + // + int num_bits_set = static_cast(random(kMinBitsSet, kMaxBitsSet)); + for (int i = 0; i < num_bits_set; ++i) { + for (;;) { + int bit_pos = static_cast(random(0, kBitsPerMask - 1)); + if (!bit_util::GetBit(masks_, bit_pos)) { + bit_util::SetBit(masks_, bit_pos); + break; + } + } + } + + int64_t num_bits_total = kNumMasks + kBitsPerMask - 1; + + // The value num_bits_set will be updated in each iteration of the loop to + // represent the number of bits set in the entire mask directly preceding the + // currently processed bit. + // + for (int64_t i = kBitsPerMask; i < num_bits_total; ++i) { + // The value of the lowest bit of the previous mask that will be removed + // from the current mask as we move to the next position in the bit vector + // of masks. + // + int bit_leaving = bit_util::GetBit(masks_, i - kBitsPerMask) ? 1 : 0; + + // Next bit has to be 1 because of minimum bits in a mask requirement + // + if (bit_leaving == 1 && num_bits_set == kMinBitsSet) { + bit_util::SetBit(masks_, i); + continue; + } + + // Next bit has to be 0 because of maximum bits in a mask requirement + // + if (bit_leaving == 0 && num_bits_set == kMaxBitsSet) { + continue; + } + + // Next bit can be random. Use the expected number of bits set in a mask + // as a probability of 1. + // + if (random(0, kBitsPerMask * 2 - 1) < kMinBitsSet + kMaxBitsSet) { + bit_util::SetBit(masks_, i); + if (bit_leaving == 0) { + ++num_bits_set; + } + } else { + if (bit_leaving == 1) { + --num_bits_set; + } + } + } +} + +BloomFilterMasks BlockedBloomFilter::masks_; + +Status BlockedBloomFilter::CreateEmpty(int64_t num_rows_to_insert, MemoryPool* pool) { + // Compute the size + // + constexpr int64_t min_num_bits_per_key = 8; + constexpr int64_t min_num_bits = 512; + int64_t desired_num_bits = + std::max(min_num_bits, num_rows_to_insert * min_num_bits_per_key); + int log_num_bits = bit_util::Log2(desired_num_bits); + + log_num_blocks_ = log_num_bits - 6; + num_blocks_ = 1ULL << log_num_blocks_; + + // Allocate and zero out bit vector + // + int64_t buffer_size = num_blocks_ * sizeof(uint64_t); + ARROW_ASSIGN_OR_RAISE(buf_, AllocateBuffer(buffer_size, pool)); + blocks_ = reinterpret_cast(buf_->mutable_data()); + memset(blocks_, 0, buffer_size); + + return Status::OK(); +} + +template +void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { + for (int64_t i = 0; i < num_rows; ++i) { + Insert(hashes[i]); + } +} + +void BlockedBloomFilter::Insert(int64_t hardware_flags, int64_t num_rows, + const uint32_t* hashes) { + int64_t num_processed = 0; +#if defined(ARROW_HAVE_AVX2) + if (hardware_flags & arrow::internal::CpuInfo::AVX2) { + num_processed = Insert_avx2(num_rows, hashes); + } +#endif + InsertImp(num_rows - num_processed, hashes + num_processed); +} + +void BlockedBloomFilter::Insert(int64_t hardware_flags, int64_t num_rows, + const uint64_t* hashes) { + int64_t num_processed = 0; +#if defined(ARROW_HAVE_AVX2) + if (hardware_flags & arrow::internal::CpuInfo::AVX2) { + num_processed = Insert_avx2(num_rows, hashes); + } +#endif + InsertImp(num_rows - num_processed, hashes + num_processed); +} + +template +void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, + uint8_t* result_bit_vector, bool enable_prefetch) const { + int64_t num_processed = 0; + uint64_t bits = 0ULL; + + if (enable_prefetch && UsePrefetch()) { + constexpr int kPrefetchIterations = 16; + for (int64_t i = 0; i < num_rows - kPrefetchIterations; ++i) { + PREFETCH(blocks_ + block_id(hashes[i + kPrefetchIterations])); + uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; + bits |= result << (i & 63); + if ((i & 63) == 63) { + reinterpret_cast(result_bit_vector)[i / 64] = bits; + bits = 0ULL; + } + } + num_processed = num_rows - kPrefetchIterations; + } + + for (int64_t i = num_processed; i < num_rows; ++i) { + uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; + bits |= result << (i & 63); + if ((i & 63) == 63) { + reinterpret_cast(result_bit_vector)[i / 64] = bits; + bits = 0ULL; + } + } + + for (int i = 0; i < bit_util::CeilDiv(num_rows % 64, 8); ++i) { + result_bit_vector[num_rows / 64 * 8 + i] = static_cast(bits >> (i * 8)); + } +} + +void BlockedBloomFilter::Find(int64_t hardware_flags, int64_t num_rows, + const uint32_t* hashes, uint8_t* result_bit_vector, + bool enable_prefetch) const { + int64_t num_processed = 0; + +#if defined(ARROW_HAVE_AVX2) + if (!(enable_prefetch && UsePrefetch()) && + (hardware_flags & arrow::internal::CpuInfo::AVX2)) { + num_processed = Find_avx2(num_rows, hashes, result_bit_vector); + // Make sure that the results in bit vector for the remaining rows start at + // a byte boundary. + // + num_processed -= (num_processed % 8); + } +#endif + + ARROW_DCHECK(num_processed % 8 == 0); + FindImp(num_rows - num_processed, hashes + num_processed, + result_bit_vector + num_processed / 8, enable_prefetch); +} + +void BlockedBloomFilter::Find(int64_t hardware_flags, int64_t num_rows, + const uint64_t* hashes, uint8_t* result_bit_vector, + bool enable_prefetch) const { + int64_t num_processed = 0; + +#if defined(ARROW_HAVE_AVX2) + if (!(enable_prefetch && UsePrefetch()) && + (hardware_flags & arrow::internal::CpuInfo::AVX2)) { + num_processed = Find_avx2(num_rows, hashes, result_bit_vector); + num_processed -= (num_processed % 8); + } +#endif + + ARROW_DCHECK(num_processed % 8 == 0); + FindImp(num_rows - num_processed, hashes + num_processed, + result_bit_vector + num_processed / 8, enable_prefetch); +} + +void BlockedBloomFilter::Fold() { + // Keep repeating until one of the stop conditions checked inside the loop + // is met + for (;;) { + // If we reached the minimum size of blocked Bloom filter then stop + constexpr int log_num_blocks_min = 4; + if (log_num_blocks_ <= log_num_blocks_min) { + break; + } + + int64_t num_bits = num_blocks_ * 64; + + // Calculate the number of bits set in this blocked Bloom filter + int64_t num_bits_set = 0; + int batch_size_max = 65536; + for (int64_t i = 0; i < num_bits; i += batch_size_max) { + int batch_size = + static_cast(std::min(num_bits - i, static_cast(batch_size_max))); + num_bits_set += + arrow::internal::CountSetBits(reinterpret_cast(blocks_) + i / 8, + /*offset=*/0, batch_size); + } + + // If at least 1/4 of bits is set then stop + if (4 * num_bits_set >= num_bits) { + break; + } + + // Decide how many times to fold at once. + // The resulting size should not be less than log_num_bits_min. + int num_folds = 1; + + while ((log_num_blocks_ - num_folds) > log_num_blocks_min && + (4 * num_bits_set) < (num_bits >> num_folds)) { + ++num_folds; + } + + // Actual update to block Bloom filter bits + SingleFold(num_folds); + } +} + +void BlockedBloomFilter::SingleFold(int num_folds) { + // Calculate number of slices and size of a slice + // + int64_t num_slices = 1LL << num_folds; + int64_t num_slice_blocks = (num_blocks_ >> num_folds); + uint64_t* target_slice = blocks_; + + // OR bits of all the slices and store result in the first slice + // + for (int64_t slice = 1; slice < num_slices; ++slice) { + const uint64_t* source_slice = blocks_ + slice * num_slice_blocks; + for (int i = 0; i < num_slice_blocks; ++i) { + target_slice[i] |= source_slice[i]; + } + } + + log_num_blocks_ -= num_folds; + num_blocks_ = 1ULL << log_num_blocks_; +} + +int BlockedBloomFilter::NumHashBitsUsed() const { + constexpr int num_bits_for_mask = (BloomFilterMasks::kLogNumMasks + 6); + int num_bits_for_block = log_num_blocks(); + return num_bits_for_mask + num_bits_for_block; +} + +bool BlockedBloomFilter::IsSameAs(const BlockedBloomFilter* other) const { + if (log_num_blocks_ != other->log_num_blocks_ || num_blocks_ != other->num_blocks_) { + return false; + } + if (memcmp(blocks_, other->blocks_, num_blocks_ * sizeof(uint64_t)) != 0) { + return false; + } + return true; +} + +int64_t BlockedBloomFilter::NumBitsSet() const { + return arrow::internal::CountSetBits(reinterpret_cast(blocks_), + /*offset=*/0, (1LL << log_num_blocks()) * 64); +} + +Status BloomFilterBuilder_SingleThreaded::Begin(size_t /*num_threads*/, + int64_t hardware_flags, MemoryPool* pool, + int64_t num_rows, int64_t /*num_batches*/, + BlockedBloomFilter* build_target) { + hardware_flags_ = hardware_flags; + build_target_ = build_target; + + RETURN_NOT_OK(build_target->CreateEmpty(num_rows, pool)); + + return Status::OK(); +} + +Status BloomFilterBuilder_SingleThreaded::PushNextBatch(size_t /*thread_index*/, + int num_rows, + const uint32_t* hashes) { + PushNextBatchImp(num_rows, hashes); + return Status::OK(); +} + +Status BloomFilterBuilder_SingleThreaded::PushNextBatch(size_t /*thread_index*/, + int num_rows, + const uint64_t* hashes) { + PushNextBatchImp(num_rows, hashes); + return Status::OK(); +} + +template +void BloomFilterBuilder_SingleThreaded::PushNextBatchImp(int num_rows, const T* hashes) { + build_target_->Insert(hardware_flags_, num_rows, hashes); +} + +Status BloomFilterBuilder_Parallel::Begin(size_t num_threads, int64_t hardware_flags, + MemoryPool* pool, int64_t num_rows, + int64_t /*num_batches*/, + BlockedBloomFilter* build_target) { + hardware_flags_ = hardware_flags; + build_target_ = build_target; + + constexpr int kMaxLogNumPrtns = 8; + log_num_prtns_ = std::min(kMaxLogNumPrtns, bit_util::Log2(num_threads)); + + thread_local_states_.resize(num_threads); + prtn_locks_.Init(1 << log_num_prtns_); + + RETURN_NOT_OK(build_target->CreateEmpty(num_rows, pool)); + + return Status::OK(); +} + +Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int num_rows, + const uint32_t* hashes) { + PushNextBatchImp(thread_id, num_rows, hashes); + return Status::OK(); +} + +Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int num_rows, + const uint64_t* hashes) { + PushNextBatchImp(thread_id, num_rows, hashes); + return Status::OK(); +} + +template +void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int num_rows, + const T* hashes) { + int num_prtns = 1 << log_num_prtns_; + ThreadLocalState& local_state = thread_local_states_[thread_id]; + local_state.partition_ranges.resize(num_prtns + 1); + local_state.partitioned_hashes_64.resize(num_rows); + local_state.unprocessed_partition_ids.resize(num_prtns); + uint16_t* partition_ranges = local_state.partition_ranges.data(); + uint64_t* partitioned_hashes = local_state.partitioned_hashes_64.data(); + int* unprocessed_partition_ids = local_state.unprocessed_partition_ids.data(); + + PartitionSort::Eval( + num_rows, num_prtns, partition_ranges, + [hashes, num_prtns](int row_id) { + constexpr int kLogBlocksKeptTogether = 7; + constexpr int kPrtnIdBitOffset = + BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; + return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); + }, + [hashes, partitioned_hashes](int row_id, int output_pos) { + partitioned_hashes[output_pos] = hashes[row_id]; + }); + + int num_unprocessed_partitions = 0; + for (int i = 0; i < num_prtns; ++i) { + bool is_prtn_empty = (partition_ranges[i + 1] == partition_ranges[i]); + if (!is_prtn_empty) { + unprocessed_partition_ids[num_unprocessed_partitions++] = i; + } + } + while (num_unprocessed_partitions > 0) { + int locked_prtn_id; + int locked_prtn_id_pos; + prtn_locks_.AcquirePartitionLock(num_unprocessed_partitions, + unprocessed_partition_ids, + /*limit_retries=*/false, /*max_retries=*/-1, + &locked_prtn_id, &locked_prtn_id_pos); + build_target_->Insert( + hardware_flags_, + partition_ranges[locked_prtn_id + 1] - partition_ranges[locked_prtn_id], + partitioned_hashes + partition_ranges[locked_prtn_id]); + prtn_locks_.ReleasePartitionLock(locked_prtn_id); + if (locked_prtn_id_pos < num_unprocessed_partitions - 1) { + unprocessed_partition_ids[locked_prtn_id_pos] = + unprocessed_partition_ids[num_unprocessed_partitions - 1]; + } + --num_unprocessed_partitions; + } +} + +void BloomFilterBuilder_Parallel::CleanUp() { + thread_local_states_.clear(); + prtn_locks_.CleanUp(); +} + +std::unique_ptr BloomFilterBuilder::Make( + BloomFilterBuildStrategy strategy) { + switch (strategy) { + case BloomFilterBuildStrategy::SINGLE_THREADED: { + std::unique_ptr impl{new BloomFilterBuilder_SingleThreaded()}; + return impl; + } + case BloomFilterBuildStrategy::PARALLEL: { + std::unique_ptr impl{new BloomFilterBuilder_Parallel()}; + return impl; + } + } + return nullptr; +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h new file mode 100644 index 00000000000..b89343373ae --- /dev/null +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -0,0 +1,322 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#if defined(ARROW_HAVE_AVX2) +#include +#endif + +#include +#include +#include +#include "arrow/compute/exec/partition_util.h" +#include "arrow/compute/exec/util.h" +#include "arrow/memory_pool.h" +#include "arrow/result.h" +#include "arrow/status.h" + +namespace arrow { +namespace compute { + +// A set of pre-generated bit masks from a 64-bit word. +// +// It is used to map selected bits of hash to a bit mask that will be used in +// a Bloom filter. +// +// These bit masks need to look random and need to have a similar fractions of +// bits set in order for a Bloom filter to have a low false positives rate. +// +struct ARROW_EXPORT BloomFilterMasks { + // Generate all masks as a single bit vector. Each bit offset in this bit + // vector corresponds to a single mask. + // In each consecutive kBitsPerMask bits, there must be between + // kMinBitsSet and kMaxBitsSet bits set. + // + BloomFilterMasks(); + + inline uint64_t mask(int bit_offset) { +#if ARROW_LITTLE_ENDIAN + return (util::SafeLoadAs(masks_ + bit_offset / 8) >> (bit_offset % 8)) & + kFullMask; +#else + return (BYTESWAP(util::SafeLoadAs(masks_ + bit_offset / 8)) >> + (bit_offset % 8)) & + kFullMask; +#endif + } + + // Masks are 57 bits long because then they can be accessed at an + // arbitrary bit offset using a single unaligned 64-bit load instruction. + // + static constexpr int kBitsPerMask = 57; + static constexpr uint64_t kFullMask = (1ULL << kBitsPerMask) - 1; + + // Minimum and maximum number of bits set in each mask. + // This constraint is enforced when generating the bit masks. + // Values should be close to each other and chosen as to minimize a Bloom + // filter false positives rate. + // + static constexpr int kMinBitsSet = 4; + static constexpr int kMaxBitsSet = 5; + + // Number of generated masks. + // Having more masks to choose will improve false positives rate of Bloom + // filter but will also use more memory, which may lead to more CPU cache + // misses. + // The chosen value results in using only a few cache-lines for mask lookups, + // while providing a good variety of available bit masks. + // + static constexpr int kLogNumMasks = 10; + static constexpr int kNumMasks = 1 << kLogNumMasks; + + // Data of masks. Masks are stored in a single bit vector. Nth mask is + // kBitsPerMask bits starting at bit offset N. + // + static constexpr int kTotalBytes = (kNumMasks + 64) / 8; + uint8_t masks_[kTotalBytes]; +}; + +// A variant of a blocked Bloom filter implementation. +// A Bloom filter is a data structure that provides approximate membership test +// functionality based only on the hash of the key. Membership test may return +// false positives but not false negatives. Approximation of the result allows +// in general case (for arbitrary data types of keys) to save on both memory and +// lookup cost compared to the accurate membership test. +// The accurate test may sometimes still be cheaper for a specific data types +// and inputs, e.g. integers from a small range. +// +// This blocked Bloom filter is optimized for use in hash joins, to achieve a +// good balance between the size of the filter, the cost of its building and +// querying and the rate of false positives. +// +class ARROW_EXPORT BlockedBloomFilter { + friend class BloomFilterBuilder_SingleThreaded; + friend class BloomFilterBuilder_Parallel; + + public: + BlockedBloomFilter() : log_num_blocks_(0), num_blocks_(0), blocks_(NULLPTR) {} + + inline bool Find(uint64_t hash) const { + uint64_t m = mask(hash); + uint64_t b = blocks_[block_id(hash)]; + return (b & m) == m; + } + + // Uses SIMD if available for smaller Bloom filters. + // Uses memory prefetching for larger Bloom filters. + // + void Find(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes, + uint8_t* result_bit_vector, bool enable_prefetch = true) const; + void Find(int64_t hardware_flags, int64_t num_rows, const uint64_t* hashes, + uint8_t* result_bit_vector, bool enable_prefetch = true) const; + + int log_num_blocks() const { return log_num_blocks_; } + + int NumHashBitsUsed() const; + + bool IsSameAs(const BlockedBloomFilter* other) const; + + int64_t NumBitsSet() const; + + // Folding of a block Bloom filter after the initial version + // has been built. + // + // One of the parameters for creation of Bloom filter is the number + // of bits allocated for it. The more bits allocated, the lower the + // probability of false positives. A good heuristic is to aim for + // half of the bits set in the constructed Bloom filter. This should + // result in a good trade off between size (and following cost of + // memory accesses) and false positives rate. + // + // There might have been many duplicate keys in the input provided + // to Bloom filter builder. In that case the resulting bit vector + // would be more sparse then originally intended. It is possible to + // easily correct that and cut in half the size of Bloom filter + // after it has already been constructed. The process to do that is + // approximately equal to OR-ing bits from upper and lower half (the + // way we address these bits when inserting or querying a hash makes + // such folding in half possible). + // + // We will keep folding as long as the fraction of bits set is less + // than 1/4. The resulting bit vector density should be in the [1/4, + // 1/2) range. + // + void Fold(); + + private: + Status CreateEmpty(int64_t num_rows_to_insert, MemoryPool* pool); + + inline void Insert(uint64_t hash) { + uint64_t m = mask(hash); + uint64_t& b = blocks_[block_id(hash)]; + b |= m; + } + + void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes); + void Insert(int64_t hardware_flags, int64_t num_rows, const uint64_t* hashes); + + inline uint64_t mask(uint64_t hash) const { + // The lowest bits of hash are used to pick mask index. + // + int mask_id = static_cast(hash & (BloomFilterMasks::kNumMasks - 1)); + uint64_t result = masks_.mask(mask_id); + + // The next set of hash bits is used to pick the amount of bit + // rotation of the mask. + // + int rotation = (hash >> BloomFilterMasks::kLogNumMasks) & 63; + result = ROTL64(result, rotation); + + return result; + } + + inline int64_t block_id(uint64_t hash) const { + // The next set of hash bits following the bits used to select a + // mask is used to pick block id (index of 64-bit word in a bit + // vector). + // + return (hash >> (BloomFilterMasks::kLogNumMasks + 6)) & (num_blocks_ - 1); + } + + template + inline void InsertImp(int64_t num_rows, const T* hashes); + + template + inline void FindImp(int64_t num_rows, const T* hashes, uint8_t* result_bit_vector, + bool enable_prefetch) const; + + void SingleFold(int num_folds); + +#if defined(ARROW_HAVE_AVX2) + inline __m256i mask_avx2(__m256i hash) const; + inline __m256i block_id_avx2(__m256i hash) const; + int64_t Insert_avx2(int64_t num_rows, const uint32_t* hashes); + int64_t Insert_avx2(int64_t num_rows, const uint64_t* hashes); + template + int64_t InsertImp_avx2(int64_t num_rows, const T* hashes); + int64_t Find_avx2(int64_t num_rows, const uint32_t* hashes, + uint8_t* result_bit_vector) const; + int64_t Find_avx2(int64_t num_rows, const uint64_t* hashes, + uint8_t* result_bit_vector) const; + template + int64_t FindImp_avx2(int64_t num_rows, const T* hashes, + uint8_t* result_bit_vector) const; +#endif + + bool UsePrefetch() const { + return num_blocks_ * sizeof(uint64_t) > kPrefetchLimitBytes; + } + + static constexpr int64_t kPrefetchLimitBytes = 256 * 1024; + + static BloomFilterMasks masks_; + + // Total number of bits used by block Bloom filter must be a power + // of 2. + // + int log_num_blocks_; + int64_t num_blocks_; + + // Buffer allocated to store an array of power of 2 64-bit blocks. + // + std::shared_ptr buf_; + // Pointer to mutable data owned by Buffer + // + uint64_t* blocks_; +}; + +// We have two separate implementations of building a Bloom filter, multi-threaded and +// single-threaded. +// +// Single threaded version is useful in two ways: +// a) It allows to verify parallel implementation in tests (the single threaded one is +// simpler and can be used as the source of truth). +// b) It is preferred for small and medium size Bloom filters, because it skips extra +// synchronization related steps from parallel variant (partitioning and taking locks). +// +enum class ARROW_EXPORT BloomFilterBuildStrategy { + SINGLE_THREADED = 0, + PARALLEL = 1, +}; + +class ARROW_EXPORT BloomFilterBuilder { + public: + virtual ~BloomFilterBuilder() = default; + virtual Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, + int64_t num_rows, int64_t num_batches, + BlockedBloomFilter* build_target) = 0; + virtual int64_t num_tasks() const { return 0; } + virtual Status PushNextBatch(size_t thread_index, int num_rows, + const uint32_t* hashes) = 0; + virtual Status PushNextBatch(size_t thread_index, int num_rows, + const uint64_t* hashes) = 0; + virtual void CleanUp() {} + static std::unique_ptr Make(BloomFilterBuildStrategy strategy); +}; + +class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { + public: + Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, + int64_t num_rows, int64_t num_batches, + BlockedBloomFilter* build_target) override; + + Status PushNextBatch(size_t /*thread_index*/, int num_rows, + const uint32_t* hashes) override; + + Status PushNextBatch(size_t /*thread_index*/, int num_rows, + const uint64_t* hashes) override; + + private: + template + void PushNextBatchImp(int num_rows, const T* hashes); + + int64_t hardware_flags_; + BlockedBloomFilter* build_target_; +}; + +class BloomFilterBuilder_Parallel : public BloomFilterBuilder { + public: + Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, + int64_t num_rows, int64_t num_batches, + BlockedBloomFilter* build_target) override; + + Status PushNextBatch(size_t thread_id, int num_rows, const uint32_t* hashes) override; + + Status PushNextBatch(size_t thread_id, int num_rows, const uint64_t* hashes) override; + + void CleanUp() override; + + private: + template + void PushNextBatchImp(size_t thread_id, int num_rows, const T* hashes); + + int64_t hardware_flags_; + BlockedBloomFilter* build_target_; + int log_num_prtns_; + struct ThreadLocalState { + std::vector partitioned_hashes_32; + std::vector partitioned_hashes_64; + std::vector partition_ranges; + std::vector unprocessed_partition_ids; + }; + std::vector thread_local_states_; + PartitionLocks prtn_locks_; +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc new file mode 100644 index 00000000000..40069618023 --- /dev/null +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -0,0 +1,138 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include "arrow/compute/exec/bloom_filter.h" +#include "arrow/util/bit_util.h" + +namespace arrow { +namespace compute { + +#if defined(ARROW_HAVE_AVX2) + +inline __m256i BlockedBloomFilter::mask_avx2(__m256i hash) const { + // AVX2 translation of mask() method + // + __m256i mask_id = + _mm256_and_si256(hash, _mm256_set1_epi64x(BloomFilterMasks::kNumMasks - 1)); + + auto masks = reinterpret_cast(masks_.masks_); + __m256i mask_byte_index = _mm256_srli_epi64(mask_id, 3); + __m256i result = _mm256_i64gather_epi64(masks, mask_byte_index, 1); + __m256i mask_bit_in_byte_index = _mm256_and_si256(mask_id, _mm256_set1_epi64x(7)); + result = _mm256_srlv_epi64(result, mask_bit_in_byte_index); + result = _mm256_and_si256(result, _mm256_set1_epi64x(BloomFilterMasks::kFullMask)); + + __m256i rotation = _mm256_and_si256( + _mm256_srli_epi64(hash, BloomFilterMasks::kLogNumMasks), _mm256_set1_epi64x(63)); + + result = _mm256_or_si256( + _mm256_sllv_epi64(result, rotation), + _mm256_srlv_epi64(result, _mm256_sub_epi64(_mm256_set1_epi64x(64), rotation))); + + return result; +} + +inline __m256i BlockedBloomFilter::block_id_avx2(__m256i hash) const { + // AVX2 translation of block_id() method + // + __m256i result = _mm256_srli_epi64(hash, BloomFilterMasks::kLogNumMasks + 6); + result = _mm256_and_si256(result, _mm256_set1_epi64x(num_blocks_ - 1)); + return result; +} + +template +int64_t BlockedBloomFilter::FindImp_avx2(int64_t num_rows, const T* hashes, + uint8_t* result_bit_vector) const { + constexpr int unroll = 8; + + auto blocks = reinterpret_cast(blocks_); + + for (int64_t i = 0; i < num_rows / unroll; ++i) { + __m256i hash_A, hash_B; + if (sizeof(T) == sizeof(uint32_t)) { + hash_A = _mm256_cvtepu32_epi64( + _mm_loadu_si128(reinterpret_cast(hashes) + 2 * i + 0)); + hash_B = _mm256_cvtepu32_epi64( + _mm_loadu_si128(reinterpret_cast(hashes) + 2 * i + 1)); + } else { + hash_A = _mm256_loadu_si256(reinterpret_cast(hashes) + 2 * i + 0); + hash_B = _mm256_loadu_si256(reinterpret_cast(hashes) + 2 * i + 1); + } + __m256i mask_A = mask_avx2(hash_A); + __m256i mask_B = mask_avx2(hash_B); + __m256i block_id_A = block_id_avx2(hash_A); + __m256i block_id_B = block_id_avx2(hash_B); + __m256i block_A = _mm256_i64gather_epi64(blocks, block_id_A, sizeof(uint64_t)); + __m256i block_B = _mm256_i64gather_epi64(blocks, block_id_B, sizeof(uint64_t)); + uint64_t result_bytes = _mm256_movemask_epi8( + _mm256_cmpeq_epi64(_mm256_and_si256(block_A, mask_A), mask_A)); + result_bytes |= static_cast(_mm256_movemask_epi8( + _mm256_cmpeq_epi64(_mm256_and_si256(block_B, mask_B), mask_B))) + << 32; + result_bit_vector[i] = + static_cast(_mm256_movemask_epi8(_mm256_set1_epi64x(result_bytes))); + } + + return num_rows - (num_rows % unroll); +} + +int64_t BlockedBloomFilter::Find_avx2(int64_t num_rows, const uint32_t* hashes, + uint8_t* result_bit_vector) const { + return FindImp_avx2(num_rows, hashes, result_bit_vector); +} + +int64_t BlockedBloomFilter::Find_avx2(int64_t num_rows, const uint64_t* hashes, + uint8_t* result_bit_vector) const { + return FindImp_avx2(num_rows, hashes, result_bit_vector); +} + +template +int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { + constexpr int unroll = 4; + + for (int64_t i = 0; i < num_rows / unroll; ++i) { + __m256i hash; + if (sizeof(T) == sizeof(uint32_t)) { + hash = _mm256_cvtepu32_epi64( + _mm_loadu_si128(reinterpret_cast(hashes) + i)); + } else { + hash = _mm256_loadu_si256(reinterpret_cast(hashes) + i); + } + __m256i mask = mask_avx2(hash); + __m256i block_id = block_id_avx2(hash); + blocks_[_mm256_extract_epi64(block_id, 0)] |= _mm256_extract_epi64(mask, 0); + blocks_[_mm256_extract_epi64(block_id, 1)] |= _mm256_extract_epi64(mask, 1); + blocks_[_mm256_extract_epi64(block_id, 2)] |= _mm256_extract_epi64(mask, 2); + blocks_[_mm256_extract_epi64(block_id, 3)] |= _mm256_extract_epi64(mask, 3); + } + + return num_rows - (num_rows % unroll); +} + +int64_t BlockedBloomFilter::Insert_avx2(int64_t num_rows, const uint32_t* hashes) { + return InsertImp_avx2(num_rows, hashes); +} + +int64_t BlockedBloomFilter::Insert_avx2(int64_t num_rows, const uint64_t* hashes) { + return InsertImp_avx2(num_rows, hashes); +} + +#endif + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc new file mode 100644 index 00000000000..1e328fbf82d --- /dev/null +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -0,0 +1,447 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include +#include +#include +#include +#include +#include "arrow/compute/exec/bloom_filter.h" +#include "arrow/compute/exec/key_hash.h" +#include "arrow/compute/exec/test_util.h" +#include "arrow/compute/exec/util.h" +#include "arrow/util/bitmap_ops.h" +#include "arrow/util/cpu_info.h" + +namespace arrow { +namespace compute { + +Status BuildBloomFilter(BloomFilterBuildStrategy strategy, int64_t hardware_flags, + MemoryPool* pool, int64_t num_rows, + std::function get_hash32_impl, + std::function get_hash64_impl, + BlockedBloomFilter* target) { + constexpr int batch_size_max = 32 * 1024; + int64_t num_batches = bit_util::CeilDiv(num_rows, batch_size_max); + + auto builder = BloomFilterBuilder::Make(strategy); + + std::vector thread_local_hashes32; + std::vector thread_local_hashes64; + thread_local_hashes32.resize(batch_size_max); + thread_local_hashes64.resize(batch_size_max); + + RETURN_NOT_OK(builder->Begin(/*num_threads=*/1, hardware_flags, pool, num_rows, + bit_util::CeilDiv(num_rows, batch_size_max), target)); + + for (int64_t i = 0; i < num_batches; ++i) { + size_t thread_index = 0; + int batch_size = static_cast( + std::min(num_rows - i * batch_size_max, static_cast(batch_size_max))); + if (target->NumHashBitsUsed() > 32) { + uint64_t* hashes = thread_local_hashes64.data(); + get_hash64_impl(i * batch_size_max, batch_size, hashes); + RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); + } else { + uint32_t* hashes = thread_local_hashes32.data(); + get_hash32_impl(i * batch_size_max, batch_size, hashes); + RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); + } + } + + builder->CleanUp(); + + return Status::OK(); +} + +// In order to simulate what would happen if there were many duplicate keys as the build +// input of Bloom filter, we use the same sequence of generated hashes multiple times. +// +// This helper function treats input hashes as a ring buffer, that can be viewed as an +// infinite sequence of hashes (for every integer we can pick a hash from the buffer +// taking integer index module buffer size). +// Then it outputs a requested window of hashes from that infinite sequence. +// +template +void TestBloomSmallHashHelper(int64_t num_input_hashes, const T* input_hashes, + int64_t first_row, int64_t num_rows, T* output_hashes) { + int64_t first_row_clamped = first_row % num_input_hashes; + int64_t num_rows_processed = 0; + while (num_rows_processed < num_rows) { + int64_t num_rows_next = + std::min(num_rows - num_rows_processed, num_input_hashes - first_row_clamped); + memcpy(output_hashes + num_rows_processed, input_hashes + first_row_clamped, + num_rows_next * sizeof(T)); + first_row_clamped = 0; + num_rows_processed += num_rows_next; + } +} + +// FPR (false positives rate) - fraction of false positives relative to the sum +// of false positives and true negatives. +// +// Output FPR and build and probe cost. +// +void TestBloomSmall(BloomFilterBuildStrategy strategy, int64_t num_build, + int num_build_copies, bool use_simd, bool enable_prefetch) { + int64_t hardware_flags = use_simd ? ::arrow::internal::CpuInfo::AVX2 : 0; + + // Generate input keys + // + int64_t num_probe = 4 * num_build; + Random64Bit rnd(/*seed=*/0); + std::vector unique_keys; + std::unordered_set unique_keys_set; + for (int64_t i = 0; i < num_build + num_probe; ++i) { + uint64_t value; + for (;;) { + value = rnd.next(); + if (unique_keys_set.find(value) == unique_keys_set.end()) { + break; + } + } + unique_keys.push_back(value); + unique_keys_set.insert(value); + } + unique_keys_set.clear(); + + // Generate input hashes + // + std::vector hashes32; + std::vector hashes64; + hashes32.resize(unique_keys.size()); + hashes64.resize(unique_keys.size()); + int batch_size_max = 1024; + for (size_t i = 0; i < unique_keys.size(); i += batch_size_max) { + int batch_size = static_cast( + std::min(unique_keys.size() - i, static_cast(batch_size_max))); + constexpr int key_length = sizeof(uint64_t); + Hashing32::HashFixed(hardware_flags, /*combine_hashes=*/false, batch_size, key_length, + reinterpret_cast(unique_keys.data() + i), + hashes32.data() + i, nullptr); + Hashing64::HashFixed( + /*combine_hashes=*/false, batch_size, key_length, + reinterpret_cast(unique_keys.data() + i), hashes64.data() + i); + } + + MemoryPool* pool = default_memory_pool(); + + // Build the filter + // + BlockedBloomFilter reference; + BlockedBloomFilter bloom; + + ASSERT_OK(BuildBloomFilter( + BloomFilterBuildStrategy::SINGLE_THREADED, hardware_flags, pool, num_build, + [hashes32](int64_t first_row, int num_rows, uint32_t* output_hashes) { + memcpy(output_hashes, hashes32.data() + first_row, num_rows * sizeof(uint32_t)); + }, + [hashes64](int64_t first_row, int num_rows, uint64_t* output_hashes) { + memcpy(output_hashes, hashes64.data() + first_row, num_rows * sizeof(uint64_t)); + }, + &reference)); + + ASSERT_OK(BuildBloomFilter( + strategy, hardware_flags, pool, num_build * num_build_copies, + [hashes32, num_build](int64_t first_row, int num_rows, uint32_t* output_hashes) { + TestBloomSmallHashHelper(num_build, hashes32.data(), first_row, + num_rows, output_hashes); + }, + [hashes64, num_build](int64_t first_row, int num_rows, uint64_t* output_hashes) { + TestBloomSmallHashHelper(num_build, hashes64.data(), first_row, + num_rows, output_hashes); + }, + &bloom)); + + int log_before = bloom.log_num_blocks(); + + if (num_build_copies > 1) { + reference.Fold(); + bloom.Fold(); + } else { + if (strategy != BloomFilterBuildStrategy::SINGLE_THREADED) { + ASSERT_TRUE(reference.IsSameAs(&bloom)); + } + } + + int log_after = bloom.log_num_blocks(); + + float fraction_of_bits_set = static_cast(bloom.NumBitsSet()) / + static_cast(64LL << bloom.log_num_blocks()); + + ARROW_SCOPED_TRACE("log_before = ", log_before, " log_after = ", log_after, + " percent_bits_set = ", 100.0f * fraction_of_bits_set); + + // Verify no false negatives and false positive rate is + // within reason + int64_t false_positives = 0; + for (int64_t i = 0; i < num_build + num_probe; ++i) { + bool found; + if (bloom.NumHashBitsUsed() > 32) { + found = bloom.Find(hashes64[i]); + } else { + found = bloom.Find(hashes32[i]); + } + // Every build key should be found + if (i < num_build) { + ASSERT_TRUE(found); + } else if (found) { + false_positives++; + } + } + + double fpr = static_cast(false_positives) / num_probe; + // Ideally this should be less than 0.05 but we check 0.1 here to avoid false failures + // due to rounding issues or minor inconsistencies in the theory + ASSERT_LT(fpr, 0.1) << "False positive rate for bloom filter was higher than expected"; +} + +template +void TestBloomLargeHashHelper(int64_t hardware_flags, int64_t block, + const std::vector& first_in_block, + int64_t first_row, int num_rows, T* output_hashes) { + // Largest 63-bit prime + constexpr uint64_t prime = 0x7FFFFFFFFFFFFFE7ULL; + + constexpr int mini_batch_size = 1024; + uint64_t keys[mini_batch_size]; + int64_t ikey = first_row / block * block; + uint64_t key = first_in_block[first_row / block]; + while (ikey < first_row) { + key += prime; + ++ikey; + } + for (int ibase = 0; ibase < num_rows;) { + int next_batch_size = std::min(num_rows - ibase, mini_batch_size); + for (int i = 0; i < next_batch_size; ++i) { + keys[i] = key; + key += prime; + } + + constexpr int key_length = sizeof(uint64_t); + if (sizeof(T) == sizeof(uint32_t)) { + Hashing32::HashFixed(hardware_flags, false, next_batch_size, key_length, + reinterpret_cast(keys), + reinterpret_cast(output_hashes) + ibase, nullptr); + } else { + Hashing64::HashFixed(false, next_batch_size, key_length, + reinterpret_cast(keys), + reinterpret_cast(output_hashes) + ibase); + } + + ibase += next_batch_size; + } +} + +// Test with larger size Bloom filters (use large prime with arithmetic +// sequence modulo 2^64). +// +void TestBloomLarge(BloomFilterBuildStrategy strategy, int64_t num_build, bool use_simd, + bool enable_prefetch) { + int64_t hardware_flags = use_simd ? ::arrow::internal::CpuInfo::AVX2 : 0; + + // Largest 63-bit prime + constexpr uint64_t prime = 0x7FFFFFFFFFFFFFE7ULL; + + // Generate input keys + // + int64_t num_probe = 4 * num_build; + const int64_t block = 1024; + std::vector first_in_block; + first_in_block.resize(bit_util::CeilDiv(num_build + num_probe, block)); + uint64_t current = prime; + for (int64_t i = 0; i < num_build + num_probe; ++i) { + if (i % block == 0) { + first_in_block[i / block] = current; + } + current += prime; + } + + MemoryPool* pool = default_memory_pool(); + + // Build the filter + // + BlockedBloomFilter reference; + BlockedBloomFilter bloom; + + for (int ibuild = 0; ibuild < 2; ++ibuild) { + if (ibuild == 0 && strategy == BloomFilterBuildStrategy::SINGLE_THREADED) { + continue; + } + ASSERT_OK(BuildBloomFilter( + ibuild == 0 ? BloomFilterBuildStrategy::SINGLE_THREADED : strategy, + hardware_flags, pool, num_build, + [hardware_flags, &first_in_block](int64_t first_row, int num_rows, + uint32_t* output_hashes) { + const int64_t block = 1024; + TestBloomLargeHashHelper(hardware_flags, block, first_in_block, first_row, + num_rows, output_hashes); + }, + [hardware_flags, &first_in_block](int64_t first_row, int num_rows, + uint64_t* output_hashes) { + const int64_t block = 1024; + TestBloomLargeHashHelper(hardware_flags, block, first_in_block, first_row, + num_rows, output_hashes); + }, + ibuild == 0 ? &reference : &bloom)); + } + + if (strategy != BloomFilterBuildStrategy::SINGLE_THREADED) { + ASSERT_TRUE(reference.IsSameAs(&bloom)); + } + + std::vector hashes32; + std::vector hashes64; + std::vector result_bit_vector; + hashes32.resize(block); + hashes64.resize(block); + result_bit_vector.resize(bit_util::BytesForBits(block)); + + // Verify no false negatives and measure false positives. + // Measure FPR and performance. + // + int64_t num_negatives_build = 0LL; + int64_t num_negatives_probe = 0LL; + + for (int64_t i = 0; i < num_build + num_probe;) { + int64_t first_row = i < num_build ? i : num_build + ((i - num_build) % num_probe); + int64_t last_row = i < num_build ? num_build : num_build + num_probe; + int64_t next_batch_size = std::min(last_row - first_row, block); + if (bloom.NumHashBitsUsed() > 32) { + TestBloomLargeHashHelper(hardware_flags, block, first_in_block, first_row, + static_cast(next_batch_size), hashes64.data()); + bloom.Find(hardware_flags, next_batch_size, hashes64.data(), + result_bit_vector.data(), enable_prefetch); + } else { + TestBloomLargeHashHelper(hardware_flags, block, first_in_block, first_row, + static_cast(next_batch_size), hashes32.data()); + bloom.Find(hardware_flags, next_batch_size, hashes32.data(), + result_bit_vector.data(), enable_prefetch); + } + uint64_t num_negatives = 0ULL; + for (int iword = 0; iword < next_batch_size / 64; ++iword) { + uint64_t word = reinterpret_cast(result_bit_vector.data())[iword]; + num_negatives += ARROW_POPCOUNT64(~word); + } + if (next_batch_size % 64 > 0) { + uint64_t word = reinterpret_cast( + result_bit_vector.data())[next_batch_size / 64]; + uint64_t mask = (1ULL << (next_batch_size % 64)) - 1; + word |= ~mask; + num_negatives += ARROW_POPCOUNT64(~word); + } + if (i < num_build) { + num_negatives_build += num_negatives; + } else { + num_negatives_probe += num_negatives; + } + i += next_batch_size; + } + + ASSERT_EQ(num_negatives_build, 0); + int64_t probe_positives = num_probe - num_negatives_probe; + double fpr = probe_positives / static_cast(num_probe); + // Ideally this should be less than 0.05 but we check 0.1 here to avoid false failures + // due to rounding issues or minor inconsistencies in the theory + ASSERT_LT(fpr, 0.1) << "False positive rate for bloom filter was higher than expected"; +} + +TEST(BloomFilter, Basic) { + std::vector num_build; + constexpr int log_min = 8; + constexpr int log_max = 16; + constexpr int log_large = 22; + for (int log_num_build = log_min; log_num_build < log_max; ++log_num_build) { + constexpr int num_intermediate_points = 2; + for (int i = 0; i < num_intermediate_points; ++i) { + int64_t num_left = 1LL << log_num_build; + int64_t num_right = 1LL << (log_num_build + 1); + num_build.push_back((num_left * (num_intermediate_points - i) + num_right * i) / + num_intermediate_points); + } + } + num_build.push_back(1LL << log_max); + num_build.push_back(1LL << log_large); + + constexpr int num_param_sets = 3; + struct { + bool use_avx2; + bool enable_prefetch; + bool insert_multiple_copies; + } params[num_param_sets]; + for (int i = 0; i < num_param_sets; ++i) { + params[i].use_avx2 = (i == 1); + params[i].enable_prefetch = (i == 2); + params[i].insert_multiple_copies = (i == 3); + } + + std::vector strategy; + strategy.push_back(BloomFilterBuildStrategy::SINGLE_THREADED); + strategy.push_back(BloomFilterBuildStrategy::PARALLEL); + + static constexpr int64_t min_rows_for_large = 2 * 1024 * 1024; + + for (size_t istrategy = 0; istrategy < strategy.size(); ++istrategy) { + for (int iparam_set = 0; iparam_set < num_param_sets; ++iparam_set) { + ARROW_SCOPED_TRACE("%s ", params[iparam_set].use_avx2 ? "AVX2" + : params[iparam_set].enable_prefetch ? "PREFETCH" + : params[iparam_set].insert_multiple_copies ? "FOLDING" + : "REGULAR"); + for (size_t inum_build = 0; inum_build < num_build.size(); ++inum_build) { + ARROW_SCOPED_TRACE("num_build ", static_cast(num_build[inum_build])); + if (num_build[inum_build] >= min_rows_for_large) { + TestBloomLarge(strategy[istrategy], num_build[inum_build], + params[iparam_set].use_avx2, params[iparam_set].enable_prefetch); + + } else { + TestBloomSmall(strategy[istrategy], num_build[inum_build], + params[iparam_set].insert_multiple_copies ? 8 : 1, + params[iparam_set].use_avx2, params[iparam_set].enable_prefetch); + } + } + } + } +} + +TEST(BloomFilter, Scaling) { + std::vector num_build; + num_build.push_back(1000000); + num_build.push_back(4000000); + + std::vector strategy; + strategy.push_back(BloomFilterBuildStrategy::PARALLEL); + + for (bool use_avx2 : {false, true}) { + for (size_t istrategy = 0; istrategy < strategy.size(); ++istrategy) { + for (size_t inum_build = 0; inum_build < num_build.size(); ++inum_build) { + ARROW_SCOPED_TRACE("num_build = ", static_cast(num_build[inum_build])); + ARROW_SCOPED_TRACE("strategy = ", + strategy[istrategy] == BloomFilterBuildStrategy::PARALLEL + ? "PARALLEL" + : "SINGLE_THREADED"); + ARROW_SCOPED_TRACE("avx2 = ", use_avx2 ? "AVX2" : "SCALAR"); + TestBloomLarge(strategy[istrategy], num_build[inum_build], use_avx2, + /*enable_prefetch=*/false); + } + } + } +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index a0c75af3fa9..96469a78ab2 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -32,7 +32,6 @@ #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" #include "arrow/util/make_unique.h" -#include "arrow/util/pcg_random.h" #include "arrow/util/thread_pool.h" using testing::UnorderedElementsAreArray; @@ -244,20 +243,6 @@ TEST_P(HashJoinTest, TestSemiJoinsEmpty) { RunEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } -class Random64Bit { - public: - explicit Random64Bit(random::SeedType seed) : rng_(seed) {} - uint64_t next() { return dist_(rng_); } - template - inline T from_range(const T& min_val, const T& max_val) { - return static_cast(min_val + (next() % (max_val - min_val + 1))); - } - - private: - random::pcg32_fast rng_; - std::uniform_int_distribution dist_; -}; - struct RandomDataTypeConstraints { int64_t data_type_enabled_mask; // Null related diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index da9ca6023bd..bc4cae74ddc 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -22,313 +22,856 @@ #include #include -#include "arrow/compute/exec/util.h" +#include "arrow/util/bit_util.h" #include "arrow/util/ubsan.h" namespace arrow { namespace compute { -inline uint32_t Hashing::avalanche_helper(uint32_t acc) { - acc ^= (acc >> 15); - acc *= PRIME32_2; - acc ^= (acc >> 13); - acc *= PRIME32_3; - acc ^= (acc >> 16); +inline uint32_t Hashing32::Round(uint32_t acc, uint32_t input) { + acc += input * PRIME32_2; + acc = ROTL(acc, 13); + acc *= PRIME32_1; return acc; } -void Hashing::avalanche(int64_t hardware_flags, uint32_t num_keys, uint32_t* hashes) { - uint32_t processed = 0; +inline uint32_t Hashing32::CombineAccumulators(uint32_t acc1, uint32_t acc2, + uint32_t acc3, uint32_t acc4) { + return ROTL(acc1, 1) + ROTL(acc2, 7) + ROTL(acc3, 12) + ROTL(acc4, 18); +} + +inline void Hashing32::ProcessFullStripes(uint64_t num_stripes, const uint8_t* key, + uint32_t* out_acc1, uint32_t* out_acc2, + uint32_t* out_acc3, uint32_t* out_acc4) { + uint32_t acc1, acc2, acc3, acc4; + acc1 = static_cast( + (static_cast(PRIME32_1) + static_cast(PRIME32_2)) & 0xffffffff); + acc2 = PRIME32_2; + acc3 = 0; + acc4 = static_cast(-static_cast(PRIME32_1)); + + for (int64_t istripe = 0; istripe < static_cast(num_stripes) - 1; ++istripe) { + const uint8_t* stripe = key + istripe * 4 * sizeof(uint32_t); + uint32_t stripe1 = util::SafeLoadAs(stripe); + uint32_t stripe2 = util::SafeLoadAs(stripe + sizeof(uint32_t)); + uint32_t stripe3 = util::SafeLoadAs(stripe + 2 * sizeof(uint32_t)); + uint32_t stripe4 = util::SafeLoadAs(stripe + 3 * sizeof(uint32_t)); + acc1 = Round(acc1, stripe1); + acc2 = Round(acc2, stripe2); + acc3 = Round(acc3, stripe3); + acc4 = Round(acc4, stripe4); + } + + *out_acc1 = acc1; + *out_acc2 = acc2; + *out_acc3 = acc3; + *out_acc4 = acc4; +} + +inline void Hashing32::ProcessLastStripe(uint32_t mask1, uint32_t mask2, uint32_t mask3, + uint32_t mask4, const uint8_t* last_stripe, + uint32_t* acc1, uint32_t* acc2, uint32_t* acc3, + uint32_t* acc4) { + uint32_t stripe1 = util::SafeLoadAs(last_stripe); + uint32_t stripe2 = util::SafeLoadAs(last_stripe + sizeof(uint32_t)); + uint32_t stripe3 = util::SafeLoadAs(last_stripe + 2 * sizeof(uint32_t)); + uint32_t stripe4 = util::SafeLoadAs(last_stripe + 3 * sizeof(uint32_t)); + stripe1 &= mask1; + stripe2 &= mask2; + stripe3 &= mask3; + stripe4 &= mask4; + *acc1 = Round(*acc1, stripe1); + *acc2 = Round(*acc2, stripe2); + *acc3 = Round(*acc3, stripe3); + *acc4 = Round(*acc4, stripe4); +} + +inline void Hashing32::StripeMask(int i, uint32_t* mask1, uint32_t* mask2, + uint32_t* mask3, uint32_t* mask4) { + // Return a 16 byte mask (encoded as 4x 32-bit integers), where the first i + // bytes are 0xff and the remaining ones are 0x00 + // + + ARROW_DCHECK(i >= 0 && i <= kStripeSize); + + static const uint32_t bytes[] = {~0U, ~0U, ~0U, ~0U, 0U, 0U, 0U, 0U}; + int offset = kStripeSize - i; + const uint8_t* mask_base = reinterpret_cast(bytes) + offset; + *mask1 = util::SafeLoadAs(mask_base); + *mask2 = util::SafeLoadAs(mask_base + sizeof(uint32_t)); + *mask3 = util::SafeLoadAs(mask_base + 2 * sizeof(uint32_t)); + *mask4 = util::SafeLoadAs(mask_base + 3 * sizeof(uint32_t)); +} + +template +void Hashing32::HashFixedLenImp(uint32_t num_rows, uint64_t length, const uint8_t* keys, + uint32_t* hashes) { + // Calculate the number of rows that skip the last 16 bytes + // + uint32_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && (num_rows - num_rows_safe) * length < kStripeSize) { + --num_rows_safe; + } + + // Compute masks for the last 16 byte stripe + // + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize); + uint32_t mask1, mask2, mask3, mask4; + StripeMask(((length - 1) & (kStripeSize - 1)) + 1, &mask1, &mask2, &mask3, &mask4); + + for (uint32_t i = 0; i < num_rows_safe; ++i) { + const uint8_t* key = keys + static_cast(i) * length; + uint32_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + ProcessLastStripe(mask1, mask2, mask3, mask4, key + (num_stripes - 1) * kStripeSize, + &acc1, &acc2, &acc3, &acc4); + uint32_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } + + uint32_t last_stripe_copy[4]; + for (uint32_t i = num_rows_safe; i < num_rows; ++i) { + const uint8_t* key = keys + static_cast(i) * length; + uint32_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + memcpy(last_stripe_copy, key + (num_stripes - 1) * kStripeSize, + length - (num_stripes - 1) * kStripeSize); + ProcessLastStripe(mask1, mask2, mask3, mask4, + reinterpret_cast(last_stripe_copy), &acc1, &acc2, + &acc3, &acc4); + uint32_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } +} + +template +void Hashing32::HashVarLenImp(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes) { + // Calculate the number of rows that skip the last 16 bytes + // + uint32_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && offsets[num_rows] - offsets[num_rows_safe] < kStripeSize) { + --num_rows_safe; + } + + for (uint32_t i = 0; i < num_rows_safe; ++i) { + uint64_t length = offsets[i + 1] - offsets[i]; + + // Compute masks for the last 16 byte stripe. + // For an empty string set number of stripes to 1 but mask to all zeroes. + // + int is_non_empty = length == 0 ? 0 : 1; + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize) + (1 - is_non_empty); + uint32_t mask1, mask2, mask3, mask4; + StripeMask(((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty, &mask1, + &mask2, &mask3, &mask4); + + const uint8_t* key = concatenated_keys + offsets[i]; + uint32_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + if (num_stripes > 0) { + ProcessLastStripe(mask1, mask2, mask3, mask4, key + (num_stripes - 1) * kStripeSize, + &acc1, &acc2, &acc3, &acc4); + } + uint32_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } + + uint32_t last_stripe_copy[4]; + for (uint32_t i = num_rows_safe; i < num_rows; ++i) { + uint64_t length = offsets[i + 1] - offsets[i]; + + // Compute masks for the last 16 byte stripe. + // For an empty string set number of stripes to 1 but mask to all zeroes. + // + int is_non_empty = length == 0 ? 0 : 1; + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize) + (1 - is_non_empty); + uint32_t mask1, mask2, mask3, mask4; + StripeMask(((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty, &mask1, + &mask2, &mask3, &mask4); + + const uint8_t* key = concatenated_keys + offsets[i]; + uint32_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + if (length > 0) { + memcpy(last_stripe_copy, key + (num_stripes - 1) * kStripeSize, + length - (num_stripes - 1) * kStripeSize); + } + if (num_stripes > 0) { + ProcessLastStripe(mask1, mask2, mask3, mask4, + reinterpret_cast(last_stripe_copy), &acc1, &acc2, + &acc3, &acc4); + } + uint32_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } +} + +void Hashing32::HashVarLen(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, + const uint32_t* offsets, const uint8_t* concatenated_keys, + uint32_t* hashes, uint32_t* hashes_temp_for_combine) { + uint32_t num_processed = 0; #if defined(ARROW_HAVE_AVX2) if (hardware_flags & arrow::internal::CpuInfo::AVX2) { - int tail = num_keys % 8; - avalanche_avx2(num_keys - tail, hashes); - processed = num_keys - tail; + num_processed = HashVarLen_avx2(combine_hashes, num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); } #endif - for (uint32_t i = processed; i < num_keys; ++i) { - hashes[i] = avalanche_helper(hashes[i]); + if (combine_hashes) { + HashVarLenImp(num_rows - num_processed, offsets + num_processed, + concatenated_keys, hashes + num_processed); + } else { + HashVarLenImp(num_rows - num_processed, offsets + num_processed, + concatenated_keys, hashes + num_processed); } } -inline uint32_t Hashing::combine_accumulators(const uint32_t acc1, const uint32_t acc2, - const uint32_t acc3, const uint32_t acc4) { - return ROTL(acc1, 1) + ROTL(acc2, 7) + ROTL(acc3, 12) + ROTL(acc4, 18); +void Hashing32::HashVarLen(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, + const uint64_t* offsets, const uint8_t* concatenated_keys, + uint32_t* hashes, uint32_t* hashes_temp_for_combine) { + uint32_t num_processed = 0; +#if defined(ARROW_HAVE_AVX2) + if (hardware_flags & arrow::internal::CpuInfo::AVX2) { + num_processed = HashVarLen_avx2(combine_hashes, num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); + } +#endif + if (combine_hashes) { + HashVarLenImp(num_rows - num_processed, offsets + num_processed, + concatenated_keys, hashes + num_processed); + } else { + HashVarLenImp(num_rows - num_processed, offsets + num_processed, + concatenated_keys, hashes + num_processed); + } } -template -inline void Hashing::helper_8B(uint32_t key_length, uint32_t num_keys, const T* keys, - uint32_t* hashes) { - ARROW_DCHECK(key_length <= 8); - constexpr uint64_t multiplier = 14029467366897019727ULL; +template +void Hashing32::HashBitImp(int64_t bit_offset, uint32_t num_keys, const uint8_t* keys, + uint32_t* hashes) { + for (uint32_t i = 0; i < num_keys; ++i) { + uint32_t bit = bit_util::GetBit(keys, bit_offset + i) ? 1 : 0; + uint32_t hash = PRIME32_1 * (1 - bit) + PRIME32_2 * bit; + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], hash); + } else { + hashes[i] = hash; + } + } +} + +void Hashing32::HashBit(bool combine_hashes, int64_t bit_offset, uint32_t num_keys, + const uint8_t* keys, uint32_t* hashes) { + if (combine_hashes) { + HashBitImp(bit_offset, num_keys, keys, hashes); + } else { + HashBitImp(bit_offset, num_keys, keys, hashes); + } +} + +template +void Hashing32::HashIntImp(uint32_t num_keys, const T* keys, uint32_t* hashes) { + constexpr uint64_t multiplier = 11400714785074694791ULL; for (uint32_t ikey = 0; ikey < num_keys; ++ikey) { uint64_t x = static_cast(keys[ikey]); - hashes[ikey] = static_cast(BYTESWAP(x * multiplier)); + uint32_t hash = static_cast(BYTESWAP(x * multiplier)); + + if (T_COMBINE_HASHES) { + hashes[ikey] = CombineHashesImp(hashes[ikey], hash); + } else { + hashes[ikey] = hash; + } } } -inline void Hashing::helper_stripe(uint32_t offset, uint64_t mask_hi, const uint8_t* keys, - uint32_t& acc1, uint32_t& acc2, uint32_t& acc3, - uint32_t& acc4) { - uint64_t v1 = util::SafeLoadAs(keys + offset); - // We do not need to mask v1, because we will not process a stripe - // unless at least 9 bytes of it are part of the key. - uint64_t v2 = util::SafeLoadAs(keys + offset + 8); - v2 &= mask_hi; - uint32_t x1 = static_cast(v1); - uint32_t x2 = static_cast(v1 >> 32); - uint32_t x3 = static_cast(v2); - uint32_t x4 = static_cast(v2 >> 32); - acc1 += x1 * PRIME32_2; - acc1 = ROTL(acc1, 13) * PRIME32_1; - acc2 += x2 * PRIME32_2; - acc2 = ROTL(acc2, 13) * PRIME32_1; - acc3 += x3 * PRIME32_2; - acc3 = ROTL(acc3, 13) * PRIME32_1; - acc4 += x4 * PRIME32_2; - acc4 = ROTL(acc4, 13) * PRIME32_1; +void Hashing32::HashInt(bool combine_hashes, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint32_t* hashes) { + switch (length_key) { + case sizeof(uint8_t): + if (combine_hashes) { + HashIntImp(num_keys, keys, hashes); + } else { + HashIntImp(num_keys, keys, hashes); + } + break; + case sizeof(uint16_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + case sizeof(uint32_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + case sizeof(uint64_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + default: + ARROW_DCHECK(false); + break; + } } -void Hashing::helper_stripes(int64_t hardware_flags, uint32_t num_keys, - uint32_t key_length, const uint8_t* keys, uint32_t* hash) { - uint32_t processed = 0; +void Hashing32::HashFixed(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, + uint64_t length, const uint8_t* keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { + if (ARROW_POPCOUNT64(length) == 1 && length <= sizeof(uint64_t)) { + HashInt(combine_hashes, num_rows, length, keys, hashes); + return; + } + + uint32_t num_processed = 0; #if defined(ARROW_HAVE_AVX2) if (hardware_flags & arrow::internal::CpuInfo::AVX2) { - int tail = num_keys % 2; - helper_stripes_avx2(num_keys - tail, key_length, keys, hash); - processed = num_keys - tail; + num_processed = HashFixedLen_avx2(combine_hashes, num_rows, length, keys, hashes, + hashes_temp_for_combine); } #endif + if (combine_hashes) { + HashFixedLenImp(num_rows - num_processed, length, keys + length * num_processed, + hashes + num_processed); + } else { + HashFixedLenImp(num_rows - num_processed, length, + keys + length * num_processed, hashes + num_processed); + } +} - // If length modulo stripe length is less than or equal 8, round down to the nearest 16B - // boundary (8B ending will be processed in a separate function), otherwise round up. - const uint32_t num_stripes = (key_length + 7) / 16; - uint64_t mask_hi = - ~0ULL >> - (8 * ((num_stripes * 16 > key_length) ? num_stripes * 16 - key_length : 0)); +void Hashing32::HashMultiColumn(const std::vector& cols, + KeyEncoder::KeyEncoderContext* ctx, uint32_t* hashes) { + uint32_t num_rows = static_cast(cols[0].length()); - for (uint32_t i = processed; i < num_keys; ++i) { - uint32_t acc1, acc2, acc3, acc4; - acc1 = static_cast( - (static_cast(PRIME32_1) + static_cast(PRIME32_2)) & - 0xffffffff); - acc2 = PRIME32_2; - acc3 = 0; - acc4 = static_cast(-static_cast(PRIME32_1)); - uint32_t offset = i * key_length; - for (uint32_t stripe = 0; stripe < num_stripes - 1; ++stripe) { - helper_stripe(offset, ~0ULL, keys, acc1, acc2, acc3, acc4); - offset += 16; + constexpr uint32_t max_batch_size = util::MiniBatch::kMiniBatchLength; + + auto hash_temp_buf = util::TempVectorHolder(ctx->stack, max_batch_size); + uint32_t* hash_temp = hash_temp_buf.mutable_data(); + + auto null_indices_buf = util::TempVectorHolder(ctx->stack, max_batch_size); + uint16_t* null_indices = null_indices_buf.mutable_data(); + int num_null_indices; + + auto null_hash_temp_buf = util::TempVectorHolder(ctx->stack, max_batch_size); + uint32_t* null_hash_temp = null_hash_temp_buf.mutable_data(); + + for (uint32_t first_row = 0; first_row < num_rows;) { + uint32_t batch_size_next = std::min(num_rows - first_row, max_batch_size); + + for (size_t icol = 0; icol < cols.size(); ++icol) { + if (cols[icol].metadata().is_null_type) { + if (icol == 0) { + for (uint32_t i = 0; i < batch_size_next; ++i) { + hashes[first_row + i] = 0; + } + } else { + for (uint32_t i = 0; i < batch_size_next; ++i) { + hashes[first_row + i] = CombineHashesImp(hashes[first_row + i], 0); + } + } + continue; + } + + // Get indices of null values within current minibatch + if (cols[icol].data(0)) { + util::bit_util::bits_to_indexes( + 0, ctx->hardware_flags, batch_size_next, cols[icol].data(0) + first_row / 8, + &num_null_indices, null_indices, first_row % 8 + cols[icol].bit_offset(0)); + // Make a backup copy of hash for nulls if needed + if (icol > 0) { + for (int i = 0; i < num_null_indices; ++i) { + null_hash_temp[i] = hashes[first_row + null_indices[i]]; + } + } + } + + if (cols[icol].metadata().is_fixed_length) { + uint32_t col_width = cols[icol].metadata().fixed_length; + if (col_width == 0) { + HashBit(icol > 0, cols[icol].bit_offset(1), batch_size_next, + cols[icol].data(1) + first_row / 8, hashes + first_row); + } else { + HashFixed(ctx->hardware_flags, icol > 0, batch_size_next, col_width, + cols[icol].data(1) + first_row * col_width, hashes + first_row, + hash_temp); + } + } else { + // TODO: add support for 64-bit offsets + HashVarLen(ctx->hardware_flags, icol > 0, batch_size_next, + cols[icol].offsets() + first_row, cols[icol].data(2), + hashes + first_row, hash_temp); + } + + // Zero hash for nulls + if (cols[icol].data(0)) { + if (icol == 0) { + for (int i = 0; i < num_null_indices; ++i) { + hashes[first_row + null_indices[i]] = 0; + } + } else { + for (int i = 0; i < num_null_indices; ++i) { + hashes[first_row + null_indices[i]] = CombineHashesImp(null_hash_temp[i], 0); + } + } + } } - helper_stripe(offset, mask_hi, keys, acc1, acc2, acc3, acc4); - hash[i] = combine_accumulators(acc1, acc2, acc3, acc4); + + first_row += batch_size_next; } } -// Process tail data in a length of 8 bytes (uint64_t) -// The caller needs to ensure that the `keys` is not less than 8 bytes (uint64_t) -inline uint32_t Hashing::helper_tail(uint32_t offset, uint64_t mask, const uint8_t* keys, - uint32_t acc) { - return helper_tail(offset, mask, keys, acc, sizeof(uint64_t)); +inline uint64_t Hashing64::Avalanche(uint64_t acc) { + acc ^= (acc >> 33); + acc *= PRIME64_2; + acc ^= (acc >> 29); + acc *= PRIME64_3; + acc ^= (acc >> 32); + return acc; } -// Process tail data with a specific `keys_length` -inline uint32_t Hashing::helper_tail(uint32_t offset, uint64_t mask, const uint8_t* keys, - uint32_t acc, uint32_t key_length) { - uint64_t v = 0; - std::memcpy(&v, keys + offset, key_length); - v &= mask; - uint32_t x1 = static_cast(v); - uint32_t x2 = static_cast(v >> 32); - acc += x1 * PRIME32_3; - acc = ROTL(acc, 17) * PRIME32_4; - acc += x2 * PRIME32_3; - acc = ROTL(acc, 17) * PRIME32_4; +inline uint64_t Hashing64::Round(uint64_t acc, uint64_t input) { + acc += input * PRIME64_2; + acc = ROTL64(acc, 31); + acc *= PRIME64_1; return acc; } -void Hashing::helper_tails(int64_t hardware_flags, uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash) { - uint32_t processed = 0; -#if defined(ARROW_HAVE_AVX2) - if (hardware_flags & arrow::internal::CpuInfo::AVX2) { - int tail = num_keys % 8; - helper_tails_avx2(num_keys - tail, key_length, keys, hash); - processed = num_keys - tail; +inline uint64_t Hashing64::CombineAccumulators(uint64_t acc1, uint64_t acc2, + uint64_t acc3, uint64_t acc4) { + uint64_t acc = ROTL64(acc1, 1) + ROTL64(acc2, 7) + ROTL64(acc3, 12) + ROTL64(acc4, 18); + + acc ^= Round(0, acc1); + acc *= PRIME64_1; + acc += PRIME64_4; + + acc ^= Round(0, acc2); + acc *= PRIME64_1; + acc += PRIME64_4; + + acc ^= Round(0, acc3); + acc *= PRIME64_1; + acc += PRIME64_4; + + acc ^= Round(0, acc4); + acc *= PRIME64_1; + acc += PRIME64_4; + + return acc; +} + +inline void Hashing64::ProcessFullStripes(uint64_t num_stripes, const uint8_t* key, + uint64_t* out_acc1, uint64_t* out_acc2, + uint64_t* out_acc3, uint64_t* out_acc4) { + uint64_t acc1 = PRIME64_1 + (PRIME64_2 & ~(1ULL << 63)); + uint64_t acc2 = PRIME64_2; + uint64_t acc3 = 0; + uint64_t acc4 = static_cast(-static_cast(PRIME64_1)); + + for (int64_t istripe = 0; istripe < static_cast(num_stripes) - 1; ++istripe) { + const uint8_t* stripe = key + istripe * kStripeSize; + uint64_t stripe1 = util::SafeLoadAs(stripe); + uint64_t stripe2 = util::SafeLoadAs(stripe + sizeof(uint64_t)); + uint64_t stripe3 = util::SafeLoadAs(stripe + 2 * sizeof(uint64_t)); + uint64_t stripe4 = util::SafeLoadAs(stripe + 3 * sizeof(uint64_t)); + acc1 = Round(acc1, stripe1); + acc2 = Round(acc2, stripe2); + acc3 = Round(acc3, stripe3); + acc4 = Round(acc4, stripe4); } -#endif - uint64_t mask = ~0ULL >> (8 * (((key_length % 8) == 0) ? 0 : 8 - (key_length % 8))); - uint32_t offset = key_length / 16 * 16; - offset += processed * key_length; - for (uint32_t i = processed; i < num_keys; ++i) { - hash[i] = helper_tail(offset, mask, keys, hash[i]); - offset += key_length; + + *out_acc1 = acc1; + *out_acc2 = acc2; + *out_acc3 = acc3; + *out_acc4 = acc4; +} + +inline void Hashing64::ProcessLastStripe(uint64_t mask1, uint64_t mask2, uint64_t mask3, + uint64_t mask4, const uint8_t* last_stripe, + uint64_t* acc1, uint64_t* acc2, uint64_t* acc3, + uint64_t* acc4) { + uint64_t stripe1 = util::SafeLoadAs(last_stripe); + uint64_t stripe2 = util::SafeLoadAs(last_stripe + sizeof(uint64_t)); + uint64_t stripe3 = util::SafeLoadAs(last_stripe + 2 * sizeof(uint64_t)); + uint64_t stripe4 = util::SafeLoadAs(last_stripe + 3 * sizeof(uint64_t)); + stripe1 &= mask1; + stripe2 &= mask2; + stripe3 &= mask3; + stripe4 &= mask4; + *acc1 = Round(*acc1, stripe1); + *acc2 = Round(*acc2, stripe2); + *acc3 = Round(*acc3, stripe3); + *acc4 = Round(*acc4, stripe4); +} + +inline void Hashing64::StripeMask(int i, uint64_t* mask1, uint64_t* mask2, + uint64_t* mask3, uint64_t* mask4) { + // Return a 32 byte mask (encoded as 4x 64-bit integers), where the first i + // bytes are 0xff and the remaining ones are 0x00 + // + + ARROW_DCHECK(i >= 0 && i <= kStripeSize); + + static const uint64_t bytes[] = {~0ULL, ~0ULL, ~0ULL, ~0ULL, 0ULL, 0ULL, 0ULL, 0ULL}; + int offset = kStripeSize - i; + const uint8_t* mask_base = reinterpret_cast(bytes) + offset; + *mask1 = util::SafeLoadAs(mask_base); + *mask2 = util::SafeLoadAs(mask_base + sizeof(uint64_t)); + *mask3 = util::SafeLoadAs(mask_base + 2 * sizeof(uint64_t)); + *mask4 = util::SafeLoadAs(mask_base + 3 * sizeof(uint64_t)); +} + +template +void Hashing64::HashFixedLenImp(uint32_t num_rows, uint64_t length, const uint8_t* keys, + uint64_t* hashes) { + // Calculate the number of rows that skip the last 32 bytes + // + uint32_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && (num_rows - num_rows_safe) * length < kStripeSize) { + --num_rows_safe; + } + + // Compute masks for the last 32 byte stripe + // + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize); + uint64_t mask1, mask2, mask3, mask4; + StripeMask(((length - 1) & (kStripeSize - 1)) + 1, &mask1, &mask2, &mask3, &mask4); + + for (uint32_t i = 0; i < num_rows_safe; ++i) { + const uint8_t* key = keys + static_cast(i) * length; + uint64_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + ProcessLastStripe(mask1, mask2, mask3, mask4, key + (num_stripes - 1) * kStripeSize, + &acc1, &acc2, &acc3, &acc4); + uint64_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } + + uint64_t last_stripe_copy[4]; + for (uint32_t i = num_rows_safe; i < num_rows; ++i) { + const uint8_t* key = keys + static_cast(i) * length; + uint64_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + memcpy(last_stripe_copy, key + (num_stripes - 1) * kStripeSize, + length - (num_stripes - 1) * kStripeSize); + ProcessLastStripe(mask1, mask2, mask3, mask4, + reinterpret_cast(last_stripe_copy), &acc1, &acc2, + &acc3, &acc4); + uint64_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } } } -void Hashing::hash_fixed(int64_t hardware_flags, uint32_t num_keys, uint32_t length_key, - const uint8_t* keys, uint32_t* hashes) { - ARROW_DCHECK(length_key > 0); - - if (length_key <= 8 && ARROW_POPCOUNT64(length_key) == 1) { - switch (length_key) { - case 1: - helper_8B(length_key, num_keys, keys, hashes); - break; - case 2: - helper_8B(length_key, num_keys, reinterpret_cast(keys), hashes); - break; - case 4: - helper_8B(length_key, num_keys, reinterpret_cast(keys), hashes); - break; - case 8: - helper_8B(length_key, num_keys, reinterpret_cast(keys), hashes); - break; - default: - ARROW_DCHECK(false); +template +void Hashing64::HashVarLenImp(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint64_t* hashes) { + // Calculate the number of rows that skip the last 32 bytes + // + uint32_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && offsets[num_rows] - offsets[num_rows_safe] < kStripeSize) { + --num_rows_safe; + } + + for (uint32_t i = 0; i < num_rows_safe; ++i) { + uint64_t length = offsets[i + 1] - offsets[i]; + + // Compute masks for the last 32 byte stripe. + // For an empty string set number of stripes to 1 but mask to all zeroes. + // + int is_non_empty = length == 0 ? 0 : 1; + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize) + (1 - is_non_empty); + uint64_t mask1, mask2, mask3, mask4; + StripeMask(((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty, &mask1, + &mask2, &mask3, &mask4); + + const uint8_t* key = concatenated_keys + offsets[i]; + uint64_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + if (num_stripes > 0) { + ProcessLastStripe(mask1, mask2, mask3, mask4, key + (num_stripes - 1) * kStripeSize, + &acc1, &acc2, &acc3, &acc4); + } + uint64_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; + } + } + + uint64_t last_stripe_copy[4]; + for (uint32_t i = num_rows_safe; i < num_rows; ++i) { + uint64_t length = offsets[i + 1] - offsets[i]; + + // Compute masks for the last 32 byte stripe + // + int is_non_empty = length == 0 ? 0 : 1; + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize) + (1 - is_non_empty); + uint64_t mask1, mask2, mask3, mask4; + StripeMask(((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty, &mask1, + &mask2, &mask3, &mask4); + + const uint8_t* key = concatenated_keys + offsets[i]; + uint64_t acc1, acc2, acc3, acc4; + ProcessFullStripes(num_stripes, key, &acc1, &acc2, &acc3, &acc4); + if (length > 0) { + memcpy(last_stripe_copy, key + (num_stripes - 1) * kStripeSize, + length - (num_stripes - 1) * kStripeSize); + } + if (num_stripes > 0) { + ProcessLastStripe(mask1, mask2, mask3, mask4, + reinterpret_cast(last_stripe_copy), &acc1, &acc2, + &acc3, &acc4); + } + uint64_t acc = CombineAccumulators(acc1, acc2, acc3, acc4); + acc = Avalanche(acc); + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], acc); + } else { + hashes[i] = acc; } - return; } - helper_stripes(hardware_flags, num_keys, length_key, keys, hashes); - if ((length_key % 16) > 0 && (length_key % 16) <= 8) { - helper_tails(hardware_flags, num_keys, length_key, keys, hashes); +} + +void Hashing64::HashVarLen(bool combine_hashes, uint32_t num_rows, + const uint32_t* offsets, const uint8_t* concatenated_keys, + uint64_t* hashes) { + if (combine_hashes) { + HashVarLenImp(num_rows, offsets, concatenated_keys, hashes); + } else { + HashVarLenImp(num_rows, offsets, concatenated_keys, hashes); } - avalanche(hardware_flags, num_keys, hashes); } -void Hashing::hash_varlen(int64_t hardware_flags, uint32_t num_rows, - const uint32_t* offsets, const uint8_t* concatenated_keys, - uint32_t* temp_buffer, // Needs to hold 4 x 32-bit per row - uint32_t* hashes) { -#if defined(ARROW_HAVE_AVX2) - if (hardware_flags & arrow::internal::CpuInfo::AVX2) { - hash_varlen_avx2(num_rows, offsets, concatenated_keys, temp_buffer, hashes); - return; +void Hashing64::HashVarLen(bool combine_hashes, uint32_t num_rows, + const uint64_t* offsets, const uint8_t* concatenated_keys, + uint64_t* hashes) { + if (combine_hashes) { + HashVarLenImp(num_rows, offsets, concatenated_keys, hashes); + } else { + HashVarLenImp(num_rows, offsets, concatenated_keys, hashes); } -#endif - static const uint64_t masks[9] = {0, - 0xffULL, - 0xffffULL, - 0xffffffULL, - 0xffffffffULL, - 0xffffffffffULL, - 0xffffffffffffULL, - 0xffffffffffffffULL, - ~0ULL}; - - for (uint32_t i = 0; i < num_rows; ++i) { - uint32_t offset = offsets[i]; - uint32_t key_length = offsets[i + 1] - offsets[i]; - const uint32_t num_stripes = key_length / 16; +} - uint32_t acc1, acc2, acc3, acc4; - acc1 = static_cast( - (static_cast(PRIME32_1) + static_cast(PRIME32_2)) & - 0xffffffff); - acc2 = PRIME32_2; - acc3 = 0; - acc4 = static_cast(-static_cast(PRIME32_1)); - - for (uint32_t stripe = 0; stripe < num_stripes; ++stripe) { - helper_stripe(offset, ~0ULL, concatenated_keys, acc1, acc2, acc3, acc4); - offset += 16; +template +void Hashing64::HashBitImp(int64_t bit_offset, uint32_t num_keys, const uint8_t* keys, + uint64_t* hashes) { + for (uint32_t i = 0; i < num_keys; ++i) { + uint64_t bit = bit_util::GetBit(keys, bit_offset + i) ? 1ULL : 0ULL; + uint64_t hash = PRIME64_1 * (1 - bit) + PRIME64_2 * bit; + + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], hash); + } else { + hashes[i] = hash; } - uint32_t key_length_remaining = key_length - num_stripes * 16; - if (key_length_remaining > 8) { - helper_stripe(offset, masks[key_length_remaining - 8], concatenated_keys, acc1, - acc2, acc3, acc4); - hashes[i] = combine_accumulators(acc1, acc2, acc3, acc4); - } else if (key_length > 0) { - uint32_t acc_combined = combine_accumulators(acc1, acc2, acc3, acc4); - hashes[i] = helper_tail(offset, masks[key_length_remaining], concatenated_keys, - acc_combined, key_length_remaining); + } +} + +void Hashing64::HashBit(bool combine_hashes, int64_t bit_offset, uint32_t num_keys, + const uint8_t* keys, uint64_t* hashes) { + if (combine_hashes) { + HashBitImp(bit_offset, num_keys, keys, hashes); + } else { + HashBitImp(bit_offset, num_keys, keys, hashes); + } +} + +template +void Hashing64::HashIntImp(uint32_t num_keys, const T* keys, uint64_t* hashes) { + constexpr uint64_t multiplier = 11400714785074694791ULL; + for (uint32_t ikey = 0; ikey < num_keys; ++ikey) { + uint64_t x = static_cast(keys[ikey]); + uint64_t hash = static_cast(BYTESWAP(x * multiplier)); + + if (T_COMBINE_HASHES) { + hashes[ikey] = CombineHashesImp(hashes[ikey], hash); } else { - hashes[i] = combine_accumulators(acc1, acc2, acc3, acc4); + hashes[ikey] = hash; } } - avalanche(hardware_flags, num_rows, hashes); } -// From: -// https://www.boost.org/doc/libs/1_37_0/doc/html/hash/reference.html#boost.hash_combine -// template -// inline void hash_combine(std::size_t& seed, const T& v) -//{ -// std::hash hasher; -// seed ^= hasher(v) + 0x9e3779b9 + (seed<<6) + (seed>>2); -//} -void Hashing::HashCombine(KeyEncoder::KeyEncoderContext* ctx, uint32_t num_rows, - uint32_t* accumulated_hash, const uint32_t* next_column_hash) { - uint32_t num_processed = 0; -#if defined(ARROW_HAVE_AVX2) - if (ctx->has_avx2()) { - num_processed = HashCombine_avx2(num_rows, accumulated_hash, next_column_hash); +void Hashing64::HashInt(bool combine_hashes, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint64_t* hashes) { + switch (length_key) { + case sizeof(uint8_t): + if (combine_hashes) { + HashIntImp(num_keys, keys, hashes); + } else { + HashIntImp(num_keys, keys, hashes); + } + break; + case sizeof(uint16_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + case sizeof(uint32_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + case sizeof(uint64_t): + if (combine_hashes) { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } else { + HashIntImp(num_keys, reinterpret_cast(keys), + hashes); + } + break; + default: + ARROW_DCHECK(false); + break; } -#endif - for (uint32_t i = num_processed; i < num_rows; ++i) { - uint32_t acc = accumulated_hash[i]; - uint32_t next = next_column_hash[i]; - next += 0x9e3779b9 + (acc << 6) + (acc >> 2); - acc ^= next; - accumulated_hash[i] = acc; +} + +void Hashing64::HashFixed(bool combine_hashes, uint32_t num_rows, uint64_t length, + const uint8_t* keys, uint64_t* hashes) { + if (ARROW_POPCOUNT64(length) == 1 && length <= sizeof(uint64_t)) { + HashInt(combine_hashes, num_rows, length, keys, hashes); + return; + } + + if (combine_hashes) { + HashFixedLenImp(num_rows, length, keys, hashes); + } else { + HashFixedLenImp(num_rows, length, keys, hashes); } } -void Hashing::HashMultiColumn(const std::vector& cols, - KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash) { +void Hashing64::HashMultiColumn(const std::vector& cols, + KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes) { uint32_t num_rows = static_cast(cols[0].length()); - auto hash_temp_buf = util::TempVectorHolder(ctx->stack, num_rows); - auto hash_null_index_buf = util::TempVectorHolder(ctx->stack, num_rows); - auto byte_temp_buf = util::TempVectorHolder(ctx->stack, num_rows); - auto varbin_temp_buf = util::TempVectorHolder(ctx->stack, 4 * num_rows); + constexpr uint32_t max_batch_size = util::MiniBatch::kMiniBatchLength; - bool is_first = true; + auto null_indices_buf = util::TempVectorHolder(ctx->stack, max_batch_size); + uint16_t* null_indices = null_indices_buf.mutable_data(); + int num_null_indices; - for (size_t icol = 0; icol < cols.size(); ++icol) { - // If the col is the first one, the hash value is stored in the out_hash buffer - // Otherwise, the hash value of current column is stored in a temp buf, and then - // combined into the out_hash buffer - uint32_t* dst_hash = is_first ? out_hash : hash_temp_buf.mutable_data(); + auto null_hash_temp_buf = util::TempVectorHolder(ctx->stack, max_batch_size); + uint64_t* null_hash_temp = null_hash_temp_buf.mutable_data(); + + for (uint32_t first_row = 0; first_row < num_rows;) { + uint32_t batch_size_next = std::min(num_rows - first_row, max_batch_size); + + for (size_t icol = 0; icol < cols.size(); ++icol) { + if (cols[icol].metadata().is_null_type) { + if (icol == 0) { + for (uint32_t i = 0; i < batch_size_next; ++i) { + hashes[first_row + i] = 0ULL; + } + } else { + for (uint32_t i = 0; i < batch_size_next; ++i) { + hashes[first_row + i] = CombineHashesImp(hashes[first_row + i], 0ULL); + } + } + continue; + } + + // Get indices of null values within current minibatch + if (cols[icol].data(0)) { + util::bit_util::bits_to_indexes( + 0, ctx->hardware_flags, batch_size_next, cols[icol].data(0) + first_row / 8, + &num_null_indices, null_indices, first_row % 8 + cols[icol].bit_offset(0)); + // Make a backup copy of hash for nulls if needed + if (icol > 0) { + for (int i = 0; i < num_null_indices; ++i) { + null_hash_temp[i] = hashes[first_row + null_indices[i]]; + } + } + } - // Set the hash value as zero for a null type col - if (cols[icol].metadata().is_null_type) { - memset(dst_hash, 0, sizeof(uint32_t) * cols[icol].length()); - } else { if (cols[icol].metadata().is_fixed_length) { - uint32_t col_width = cols[icol].metadata().fixed_length; + uint64_t col_width = cols[icol].metadata().fixed_length; if (col_width == 0) { - util::bit_util::bits_to_bytes(ctx->hardware_flags, num_rows, cols[icol].data(1), - byte_temp_buf.mutable_data(), - cols[icol].bit_offset(1)); + HashBit(icol > 0, cols[icol].bit_offset(1), batch_size_next, + cols[icol].data(1) + first_row / 8, hashes + first_row); + } else { + HashFixed(icol > 0, batch_size_next, col_width, + cols[icol].data(1) + first_row * col_width, hashes + first_row); } - Hashing::hash_fixed( - ctx->hardware_flags, num_rows, col_width == 0 ? 1 : col_width, - col_width == 0 ? byte_temp_buf.mutable_data() : cols[icol].data(1), dst_hash); } else { - Hashing::hash_varlen( - ctx->hardware_flags, num_rows, cols[icol].offsets(), cols[icol].data(2), - varbin_temp_buf.mutable_data(), // Needs to hold 4 x 32-bit per row - dst_hash); + // TODO: add support for 64-bit offsets + HashVarLen(icol > 0, batch_size_next, cols[icol].offsets() + first_row, + cols[icol].data(2), hashes + first_row); } // Zero hash for nulls if (cols[icol].data(0)) { - int num_nulls; - util::bit_util::bits_to_indexes( - 0, ctx->hardware_flags, num_rows, cols[icol].data(0), &num_nulls, - hash_null_index_buf.mutable_data(), cols[icol].bit_offset(0)); - for (int i = 0; i < num_nulls; ++i) { - uint16_t row_id = hash_null_index_buf.mutable_data()[i]; - dst_hash[row_id] = 0; + if (icol == 0) { + for (int i = 0; i < num_null_indices; ++i) { + hashes[first_row + null_indices[i]] = 0; + } + } else { + for (int i = 0; i < num_null_indices; ++i) { + hashes[first_row + null_indices[i]] = CombineHashesImp(null_hash_temp[i], 0); + } } } } - if (!is_first) { - HashCombine(ctx, num_rows, out_hash, hash_temp_buf.mutable_data()); - } - is_first = false; + first_row += batch_size_next; } } diff --git a/cpp/src/arrow/compute/exec/key_hash.h b/cpp/src/arrow/compute/exec/key_hash.h index f5505e95915..88f77be1a4f 100644 --- a/cpp/src/arrow/compute/exec/key_hash.h +++ b/cpp/src/arrow/compute/exec/key_hash.h @@ -29,79 +29,184 @@ namespace arrow { namespace compute { +// Forward declarations only needed for making test functions a friend of the classes in +// this file. +// +enum class BloomFilterBuildStrategy; + // Implementations are based on xxh3 32-bit algorithm description from: // https://github.com/Cyan4973/xxHash/blob/dev/doc/xxhash_spec.md // -class Hashing { - public: - static void hash_fixed(int64_t hardware_flags, uint32_t num_keys, uint32_t length_key, - const uint8_t* keys, uint32_t* hashes); - - static void hash_varlen(int64_t hardware_flags, uint32_t num_rows, - const uint32_t* offsets, const uint8_t* concatenated_keys, - uint32_t* temp_buffer, // Needs to hold 4 x 32-bit per row - uint32_t* hashes); +class ARROW_EXPORT Hashing32 { + friend class TestVectorHash; + template + friend void TestBloomLargeHashHelper(int64_t, int64_t, const std::vector&, + int64_t, int, T*); + friend void TestBloomSmall(BloomFilterBuildStrategy, int64_t, int, bool, bool); + public: static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash); private: - static const uint32_t PRIME32_1 = 0x9E3779B1; // 0b10011110001101110111100110110001 - static const uint32_t PRIME32_2 = 0x85EBCA77; // 0b10000101111010111100101001110111 - static const uint32_t PRIME32_3 = 0xC2B2AE3D; // 0b11000010101100101010111000111101 - static const uint32_t PRIME32_4 = 0x27D4EB2F; // 0b00100111110101001110101100101111 - static const uint32_t PRIME32_5 = 0x165667B1; // 0b00010110010101100110011110110001 - - static void HashCombine(KeyEncoder::KeyEncoderContext* ctx, uint32_t num_rows, - uint32_t* accumulated_hash, const uint32_t* next_column_hash); - -#if defined(ARROW_HAVE_AVX2) - static uint32_t HashCombine_avx2(uint32_t num_rows, uint32_t* accumulated_hash, - const uint32_t* next_column_hash); -#endif - - // Avalanche - static inline uint32_t avalanche_helper(uint32_t acc); -#if defined(ARROW_HAVE_AVX2) - static void avalanche_avx2(uint32_t num_keys, uint32_t* hashes); -#endif - static void avalanche(int64_t hardware_flags, uint32_t num_keys, uint32_t* hashes); + static const uint32_t PRIME32_1 = 0x9E3779B1; + static const uint32_t PRIME32_2 = 0x85EBCA77; + static const uint32_t PRIME32_3 = 0xC2B2AE3D; + static const uint32_t PRIME32_4 = 0x27D4EB2F; + static const uint32_t PRIME32_5 = 0x165667B1; + static const uint32_t kCombineConst = 0x9e3779b9UL; + static const int64_t kStripeSize = 4 * sizeof(uint32_t); + + static void HashFixed(int64_t hardware_flags, bool combine_hashes, uint32_t num_keys, + uint64_t length_key, const uint8_t* keys, uint32_t* hashes, + uint32_t* temp_hashes_for_combine); + + static void HashVarLen(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, + const uint32_t* offsets, const uint8_t* concatenated_keys, + uint32_t* hashes, uint32_t* temp_hashes_for_combine); + + static void HashVarLen(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, + const uint64_t* offsets, const uint8_t* concatenated_keys, + uint32_t* hashes, uint32_t* temp_hashes_for_combine); + + static inline uint32_t Avalanche(uint32_t acc) { + acc ^= (acc >> 15); + acc *= PRIME32_2; + acc ^= (acc >> 13); + acc *= PRIME32_3; + acc ^= (acc >> 16); + return acc; + } + static inline uint32_t Round(uint32_t acc, uint32_t input); + static inline uint32_t CombineAccumulators(uint32_t acc1, uint32_t acc2, uint32_t acc3, + uint32_t acc4); + static inline uint32_t CombineHashesImp(uint32_t previous_hash, uint32_t hash) { + uint32_t next_hash = previous_hash ^ (hash + kCombineConst + (previous_hash << 6) + + (previous_hash >> 2)); + return next_hash; + } + static inline void ProcessFullStripes(uint64_t num_stripes, const uint8_t* key, + uint32_t* out_acc1, uint32_t* out_acc2, + uint32_t* out_acc3, uint32_t* out_acc4); + static inline void ProcessLastStripe(uint32_t mask1, uint32_t mask2, uint32_t mask3, + uint32_t mask4, const uint8_t* last_stripe, + uint32_t* acc1, uint32_t* acc2, uint32_t* acc3, + uint32_t* acc4); + static inline void StripeMask(int i, uint32_t* mask1, uint32_t* mask2, uint32_t* mask3, + uint32_t* mask4); + template + static void HashFixedLenImp(uint32_t num_rows, uint64_t length, const uint8_t* keys, + uint32_t* hashes); + template + static void HashVarLenImp(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes); + template + static void HashBitImp(int64_t bit_offset, uint32_t num_keys, const uint8_t* keys, + uint32_t* hashes); + static void HashBit(bool combine_hashes, int64_t bit_offset, uint32_t num_keys, + const uint8_t* keys, uint32_t* hashes); + template + static void HashIntImp(uint32_t num_keys, const T* keys, uint32_t* hashes); + static void HashInt(bool combine_hashes, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint32_t* hashes); - // Accumulator combine - static inline uint32_t combine_accumulators(const uint32_t acc1, const uint32_t acc2, - const uint32_t acc3, const uint32_t acc4); #if defined(ARROW_HAVE_AVX2) - static inline uint64_t combine_accumulators_avx2(__m256i acc); + static inline __m256i Avalanche_avx2(__m256i hash); + static inline __m256i CombineHashesImp_avx2(__m256i previous_hash, __m256i hash); + template + static void AvalancheAll_avx2(uint32_t num_rows, uint32_t* hashes, + const uint32_t* hashes_temp_for_combine); + static inline __m256i Round_avx2(__m256i acc, __m256i input); + static inline uint64_t CombineAccumulators_avx2(__m256i acc); + static inline __m256i StripeMask_avx2(int i, int j); + template + static inline __m256i ProcessStripes_avx2(int64_t num_stripes_A, int64_t num_stripes_B, + __m256i mask_last_stripe, const uint8_t* keys, + int64_t offset_A, int64_t offset_B); + template + static uint32_t HashFixedLenImp_avx2(uint32_t num_rows, uint64_t length, + const uint8_t* keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine); + static uint32_t HashFixedLen_avx2(bool combine_hashes, uint32_t num_rows, + uint64_t length, const uint8_t* keys, + uint32_t* hashes, uint32_t* hashes_temp_for_combine); + template + static uint32_t HashVarLenImp_avx2(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine); + static uint32_t HashVarLen_avx2(bool combine_hashes, uint32_t num_rows, + const uint32_t* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine); + static uint32_t HashVarLen_avx2(bool combine_hashes, uint32_t num_rows, + const uint64_t* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine); #endif +}; - // Helpers +class ARROW_EXPORT Hashing64 { + friend class TestVectorHash; template - static inline void helper_8B(uint32_t key_length, uint32_t num_keys, const T* keys, - uint32_t* hashes); - static inline void helper_stripe(uint32_t offset, uint64_t mask_hi, const uint8_t* keys, - uint32_t& acc1, uint32_t& acc2, uint32_t& acc3, - uint32_t& acc4); - static inline uint32_t helper_tail(uint32_t offset, uint64_t mask, const uint8_t* keys, - uint32_t acc); - static inline uint32_t helper_tail(uint32_t offset, uint64_t mask, const uint8_t* keys, - uint32_t acc, uint32_t key_length); -#if defined(ARROW_HAVE_AVX2) - static void helper_stripes_avx2(uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash); - static void helper_tails_avx2(uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash); -#endif - static void helper_stripes(int64_t hardware_flags, uint32_t num_keys, - uint32_t key_length, const uint8_t* keys, uint32_t* hash); - static void helper_tails(int64_t hardware_flags, uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash); + friend void TestBloomLargeHashHelper(int64_t, int64_t, const std::vector&, + int64_t, int, T*); + friend void TestBloomSmall(BloomFilterBuildStrategy, int64_t, int, bool, bool); -#if defined(ARROW_HAVE_AVX2) - static void hash_varlen_avx2(uint32_t num_rows, const uint32_t* offsets, - const uint8_t* concatenated_keys, - uint32_t* temp_buffer, // Needs to hold 4 x 32-bit per row - uint32_t* hashes); -#endif + public: + static void HashMultiColumn(const std::vector& cols, + KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes); + + private: + static const uint64_t PRIME64_1 = 0x9E3779B185EBCA87ULL; + static const uint64_t PRIME64_2 = 0xC2B2AE3D27D4EB4FULL; + static const uint64_t PRIME64_3 = 0x165667B19E3779F9ULL; + static const uint64_t PRIME64_4 = 0x85EBCA77C2B2AE63ULL; + static const uint64_t PRIME64_5 = 0x27D4EB2F165667C5ULL; + static const uint32_t kCombineConst = 0x9e3779b9UL; + static const int64_t kStripeSize = 4 * sizeof(uint64_t); + + static void HashFixed(bool combine_hashes, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint64_t* hashes); + + static void HashVarLen(bool combine_hashes, uint32_t num_rows, const uint32_t* offsets, + const uint8_t* concatenated_keys, uint64_t* hashes); + + static void HashVarLen(bool combine_hashes, uint32_t num_rows, const uint64_t* offsets, + const uint8_t* concatenated_keys, uint64_t* hashes); + + static inline uint64_t Avalanche(uint64_t acc); + static inline uint64_t Round(uint64_t acc, uint64_t input); + static inline uint64_t CombineAccumulators(uint64_t acc1, uint64_t acc2, uint64_t acc3, + uint64_t acc4); + static inline uint64_t CombineHashesImp(uint64_t previous_hash, uint64_t hash) { + uint64_t next_hash = previous_hash ^ (hash + kCombineConst + (previous_hash << 6) + + (previous_hash >> 2)); + return next_hash; + } + static inline void ProcessFullStripes(uint64_t num_stripes, const uint8_t* key, + uint64_t* out_acc1, uint64_t* out_acc2, + uint64_t* out_acc3, uint64_t* out_acc4); + static inline void ProcessLastStripe(uint64_t mask1, uint64_t mask2, uint64_t mask3, + uint64_t mask4, const uint8_t* last_stripe, + uint64_t* acc1, uint64_t* acc2, uint64_t* acc3, + uint64_t* acc4); + static inline void StripeMask(int i, uint64_t* mask1, uint64_t* mask2, uint64_t* mask3, + uint64_t* mask4); + template + static void HashFixedLenImp(uint32_t num_rows, uint64_t length, const uint8_t* keys, + uint64_t* hashes); + template + static void HashVarLenImp(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint64_t* hashes); + template + static void HashBitImp(int64_t bit_offset, uint32_t num_keys, const uint8_t* keys, + uint64_t* hashes); + static void HashBit(bool T_COMBINE_HASHES, int64_t bit_offset, uint32_t num_keys, + const uint8_t* keys, uint64_t* hashes); + template + static void HashIntImp(uint32_t num_keys, const T* keys, uint64_t* hashes); + static void HashInt(bool T_COMBINE_HASHES, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint64_t* hashes); }; } // namespace compute diff --git a/cpp/src/arrow/compute/exec/key_hash_avx2.cc b/cpp/src/arrow/compute/exec/key_hash_avx2.cc index 3804afe106d..d36df9fc9f3 100644 --- a/cpp/src/arrow/compute/exec/key_hash_avx2.cc +++ b/cpp/src/arrow/compute/exec/key_hash_avx2.cc @@ -18,31 +18,80 @@ #include #include "arrow/compute/exec/key_hash.h" +#include "arrow/util/bit_util.h" namespace arrow { namespace compute { #if defined(ARROW_HAVE_AVX2) -void Hashing::avalanche_avx2(uint32_t num_keys, uint32_t* hashes) { +inline __m256i Hashing32::Avalanche_avx2(__m256i hash) { + hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 15)); + hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_2)); + hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 13)); + hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_3)); + hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 16)); + return hash; +} + +inline __m256i Hashing32::CombineHashesImp_avx2(__m256i previous_hash, __m256i hash) { + // previous_hash ^= acc + kCombineConst + (previous_hash << 6) + + // (previous_hash >> 2); + // + __m256i x = _mm256_add_epi32(_mm256_slli_epi32(previous_hash, 6), + _mm256_srli_epi32(previous_hash, 2)); + __m256i y = _mm256_add_epi32(hash, _mm256_set1_epi32(kCombineConst)); + __m256i new_hash = _mm256_xor_si256(previous_hash, _mm256_add_epi32(x, y)); + return new_hash; +} + +template +void Hashing32::AvalancheAll_avx2(uint32_t num_rows_to_process, uint32_t* hashes, + const uint32_t* hashes_temp_for_combine) { constexpr int unroll = 8; - ARROW_DCHECK(num_keys % unroll == 0); - for (uint32_t i = 0; i < num_keys / unroll; ++i) { - __m256i hash = _mm256_loadu_si256(reinterpret_cast(hashes) + i); - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 15)); - hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_2)); - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 13)); - hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_3)); - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 16)); - _mm256_storeu_si256((reinterpret_cast<__m256i*>(hashes)) + i, hash); + for (uint32_t i = 0; i < num_rows_to_process / unroll; ++i) { + __m256i acc; + if (T_COMBINE_HASHES) { + acc = _mm256_loadu_si256(reinterpret_cast(hashes_temp_for_combine) + + i); + } else { + acc = _mm256_loadu_si256(reinterpret_cast(hashes) + i); + } + acc = Avalanche_avx2(acc); + if (T_COMBINE_HASHES) { + __m256i previous_hash = + _mm256_loadu_si256(reinterpret_cast(hashes) + i); + acc = CombineHashesImp_avx2(previous_hash, acc); + } + _mm256_storeu_si256(reinterpret_cast<__m256i*>(hashes) + i, acc); } + for (uint32_t i = num_rows_to_process - (num_rows_to_process % unroll); + i < num_rows_to_process; ++i) { + if (T_COMBINE_HASHES) { + hashes[i] = CombineHashesImp(hashes[i], Avalanche(hashes_temp_for_combine[i])); + } else { + hashes[i] = Avalanche(hashes[i]); + } + } +} + +inline __m256i Hashing32::Round_avx2(__m256i acc, __m256i input) { + acc = _mm256_add_epi32(acc, _mm256_mullo_epi32(input, _mm256_set1_epi32(PRIME32_2))); + acc = _mm256_or_si256(_mm256_slli_epi32(acc, 13), _mm256_srli_epi32(acc, 32 - 13)); + acc = _mm256_mullo_epi32(acc, _mm256_set1_epi32(PRIME32_1)); + return acc; } -inline uint64_t Hashing::combine_accumulators_avx2(__m256i acc) { - acc = _mm256_or_si256( - _mm256_sllv_epi32(acc, _mm256_setr_epi32(1, 7, 12, 18, 1, 7, 12, 18)), - _mm256_srlv_epi32(acc, _mm256_setr_epi32(32 - 1, 32 - 7, 32 - 12, 32 - 18, 32 - 1, - 32 - 7, 32 - 12, 32 - 18))); +inline uint64_t Hashing32::CombineAccumulators_avx2(__m256i acc) { + // Each 128-bit lane of input represents a set of 4 accumulators related to + // a single hash (we process here two hashes together). + // + __m256i rotate_const_left = _mm256_setr_epi32(1, 7, 12, 18, 1, 7, 12, 18); + __m256i rotate_const_right = _mm256_setr_epi32(32 - 1, 32 - 7, 32 - 12, 32 - 18, 32 - 1, + 32 - 7, 32 - 12, 32 - 18); + + acc = _mm256_or_si256(_mm256_sllv_epi32(acc, rotate_const_left), + _mm256_srlv_epi32(acc, rotate_const_right)); acc = _mm256_add_epi32(acc, _mm256_shuffle_epi32(acc, 0xee)); // 0b11101110 acc = _mm256_add_epi32(acc, _mm256_srli_epi64(acc, 32)); acc = _mm256_permutevar8x32_epi32(acc, _mm256_setr_epi32(0, 4, 0, 0, 0, 0, 0, 0)); @@ -50,216 +99,220 @@ inline uint64_t Hashing::combine_accumulators_avx2(__m256i acc) { return result; } -void Hashing::helper_stripes_avx2(uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash) { +inline __m256i Hashing32::StripeMask_avx2(int i, int j) { + // Return two 16 byte masks, where the first i/j bytes are 0xff and the + // remaining ones are 0x00 + // + ARROW_DCHECK(i >= 0 && i <= kStripeSize && j >= 0 && j <= kStripeSize); + return _mm256_cmpgt_epi8( + _mm256_blend_epi32(_mm256_set1_epi8(i), _mm256_set1_epi8(j), 0xf0), + _mm256_setr_epi64x(0x0706050403020100ULL, 0x0f0e0d0c0b0a0908ULL, + 0x0706050403020100ULL, 0x0f0e0d0c0b0a0908ULL)); +} + +template +inline __m256i Hashing32::ProcessStripes_avx2(int64_t num_stripes_A, + int64_t num_stripes_B, + __m256i mask_last_stripe, + const uint8_t* keys, int64_t offset_A, + int64_t offset_B) { + ARROW_DCHECK(num_stripes_A > 0 && num_stripes_B > 0); + + __m256i acc = _mm256_setr_epi32( + static_cast((static_cast(PRIME32_1) + PRIME32_2) & 0xffffffff), + PRIME32_2, 0, static_cast(-static_cast(PRIME32_1)), + static_cast((static_cast(PRIME32_1) + PRIME32_2) & 0xffffffff), + PRIME32_2, 0, static_cast(-static_cast(PRIME32_1))); + + // Constant for permutexvar8x32 instruction that conditionally swaps two + // 128-bit lanes if and only if num_stripes_B > num_stripes_A. + // + __m256i swap_permute = _mm256_setzero_si256(); + int64_t offset_shorter, offset_longer; + int64_t num_stripes_shorter, num_stripes_longer; + + if (!two_equal_lengths) { + int64_t swap_mask = num_stripes_B > num_stripes_A ? ~0LL : 0LL; + swap_permute = _mm256_xor_si256(_mm256_setr_epi32(0, 1, 2, 3, 4, 5, 6, 7), + _mm256_set1_epi32(swap_mask & 4)); + offset_shorter = (offset_A & swap_mask) | (offset_B & ~swap_mask); + offset_longer = (offset_A & ~swap_mask) | (offset_B & swap_mask); + num_stripes_shorter = (num_stripes_A & swap_mask) | (num_stripes_B & ~swap_mask); + num_stripes_longer = (num_stripes_A & ~swap_mask) | (num_stripes_B & swap_mask); + } else { + ARROW_DCHECK(num_stripes_A == num_stripes_B); + offset_longer = offset_A; + offset_shorter = offset_B; + num_stripes_longer = num_stripes_A; + num_stripes_shorter = num_stripes_A; + } + + int64_t istripe = 0; + for (; istripe + 1 < num_stripes_shorter; ++istripe) { + __m256i stripe = _mm256_inserti128_si256( + _mm256_castsi128_si256(_mm_loadu_si128( + reinterpret_cast(keys + offset_longer) + istripe)), + _mm_loadu_si128(reinterpret_cast(keys + offset_shorter) + + istripe), + 1); + acc = Round_avx2(acc, stripe); + } + __m256i stripe = _mm256_inserti128_si256( + _mm256_castsi128_si256(_mm_loadu_si128( + reinterpret_cast(keys + offset_longer) + istripe)), + _mm_loadu_si128(reinterpret_cast(keys + offset_shorter) + istripe), + 1); + if (!two_equal_lengths) { + __m256i acc_copy = acc; + for (; istripe + 1 < num_stripes_longer; ++istripe) { + acc = Round_avx2(acc, stripe); + stripe = _mm256_inserti128_si256( + stripe, + _mm_loadu_si128(reinterpret_cast(keys + offset_longer) + + istripe + 1), + 0); + } + acc = _mm256_blend_epi32(acc, acc_copy, 0xf0); + mask_last_stripe = _mm256_permutevar8x32_epi32(mask_last_stripe, swap_permute); + } + stripe = _mm256_and_si256(stripe, mask_last_stripe); + acc = Round_avx2(acc, stripe); + if (!two_equal_lengths) { + acc = _mm256_permutevar8x32_epi32(acc, swap_permute); + } + return acc; +} + +template +uint32_t Hashing32::HashFixedLenImp_avx2(uint32_t num_rows, uint64_t length, + const uint8_t* keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { constexpr int unroll = 2; - ARROW_DCHECK(num_keys % unroll == 0); - - constexpr uint64_t kByteSequence0To7 = 0x0706050403020100ULL; - constexpr uint64_t kByteSequence8To15 = 0x0f0e0d0c0b0a0908ULL; - - const __m256i mask_last_stripe = - (key_length % 16) <= 8 - ? _mm256_set1_epi8(static_cast(0xffU)) - : _mm256_cmpgt_epi8(_mm256_set1_epi8(key_length % 16), - _mm256_setr_epi64x(kByteSequence0To7, kByteSequence8To15, - kByteSequence0To7, kByteSequence8To15)); - - // If length modulo stripe length is less than or equal 8, round down to the nearest 16B - // boundary (8B ending will be processed in a separate function), otherwise round up. - const uint32_t num_stripes = (key_length + 7) / 16; - for (uint32_t i = 0; i < num_keys / unroll; ++i) { - __m256i acc = _mm256_setr_epi32( - static_cast((static_cast(PRIME32_1) + PRIME32_2) & - 0xffffffff), - PRIME32_2, 0, static_cast(-static_cast(PRIME32_1)), - static_cast((static_cast(PRIME32_1) + PRIME32_2) & - 0xffffffff), - PRIME32_2, 0, static_cast(-static_cast(PRIME32_1))); - auto key0 = reinterpret_cast(keys + key_length * 2 * i); - auto key1 = reinterpret_cast(keys + key_length * 2 * i + key_length); - for (uint32_t stripe = 0; stripe < num_stripes - 1; ++stripe) { - auto key_stripe = - _mm256_inserti128_si256(_mm256_castsi128_si256(_mm_loadu_si128(key0 + stripe)), - _mm_loadu_si128(key1 + stripe), 1); - acc = _mm256_add_epi32( - acc, _mm256_mullo_epi32(key_stripe, _mm256_set1_epi32(PRIME32_2))); - acc = _mm256_or_si256(_mm256_slli_epi32(acc, 13), _mm256_srli_epi32(acc, 32 - 13)); - acc = _mm256_mullo_epi32(acc, _mm256_set1_epi32(PRIME32_1)); + + // Do not process rows that could read past the end of the buffer using 16 + // byte loads. Round down number of rows to process to multiple of 2. + // + uint64_t num_rows_to_skip = bit_util::CeilDiv(length, kStripeSize); + uint32_t num_rows_to_process = + (num_rows_to_skip > num_rows) + ? 0 + : (num_rows - static_cast(num_rows_to_skip)); + num_rows_to_process -= (num_rows_to_process % unroll); + + uint64_t num_stripes = bit_util::CeilDiv(length, kStripeSize); + int num_tail_bytes = ((length - 1) & (kStripeSize - 1)) + 1; + __m256i mask_last_stripe = StripeMask_avx2(num_tail_bytes, num_tail_bytes); + + for (uint32_t i = 0; i < num_rows_to_process / unroll; ++i) { + __m256i acc = ProcessStripes_avx2( + num_stripes, num_stripes, mask_last_stripe, keys, + static_cast(i) * unroll * length, + static_cast(i) * unroll * length + length); + + if (T_COMBINE_HASHES) { + reinterpret_cast(hashes_temp_for_combine)[i] = + CombineAccumulators_avx2(acc); + } else { + reinterpret_cast(hashes)[i] = CombineAccumulators_avx2(acc); } - auto key_stripe = _mm256_inserti128_si256( - _mm256_castsi128_si256(_mm_loadu_si128(key0 + num_stripes - 1)), - _mm_loadu_si128(key1 + num_stripes - 1), 1); - key_stripe = _mm256_and_si256(key_stripe, mask_last_stripe); - acc = _mm256_add_epi32(acc, - _mm256_mullo_epi32(key_stripe, _mm256_set1_epi32(PRIME32_2))); - acc = _mm256_or_si256(_mm256_slli_epi32(acc, 13), _mm256_srli_epi32(acc, 32 - 13)); - acc = _mm256_mullo_epi32(acc, _mm256_set1_epi32(PRIME32_1)); - uint64_t result = combine_accumulators_avx2(acc); - reinterpret_cast(hash)[i] = result; } + + AvalancheAll_avx2(num_rows_to_process, hashes, + hashes_temp_for_combine); + + return num_rows_to_process; } -void Hashing::helper_tails_avx2(uint32_t num_keys, uint32_t key_length, - const uint8_t* keys, uint32_t* hash) { - constexpr int unroll = 8; - ARROW_DCHECK(num_keys % unroll == 0); - auto keys_i64 = reinterpret_cast(keys); - - // Process between 1 and 8 last bytes of each key, starting from 16B boundary. - // The caller needs to make sure that there are no more than 8 bytes to process after - // that 16B boundary. - uint32_t first_offset = key_length - (key_length % 16); - __m256i mask = _mm256_set1_epi64x((~0ULL) >> (8 * (8 - (key_length % 16)))); - __m256i offset = - _mm256_setr_epi32(0, key_length, key_length * 2, key_length * 3, key_length * 4, - key_length * 5, key_length * 6, key_length * 7); - offset = _mm256_add_epi32(offset, _mm256_set1_epi32(first_offset)); - __m256i offset_incr = _mm256_set1_epi32(key_length * 8); - - for (uint32_t i = 0; i < num_keys / unroll; ++i) { - auto v1 = _mm256_i32gather_epi64(keys_i64, _mm256_castsi256_si128(offset), 1); - auto v2 = _mm256_i32gather_epi64(keys_i64, _mm256_extracti128_si256(offset, 1), 1); - v1 = _mm256_and_si256(v1, mask); - v2 = _mm256_and_si256(v2, mask); - v1 = _mm256_permutevar8x32_epi32(v1, _mm256_setr_epi32(0, 2, 4, 6, 1, 3, 5, 7)); - v2 = _mm256_permutevar8x32_epi32(v2, _mm256_setr_epi32(0, 2, 4, 6, 1, 3, 5, 7)); - auto x1 = _mm256_permute2x128_si256(v1, v2, 0x20); - auto x2 = _mm256_permute2x128_si256(v1, v2, 0x31); - __m256i acc = _mm256_loadu_si256((reinterpret_cast(hash)) + i); - - acc = _mm256_add_epi32(acc, _mm256_mullo_epi32(x1, _mm256_set1_epi32(PRIME32_3))); - acc = _mm256_or_si256(_mm256_slli_epi32(acc, 17), _mm256_srli_epi32(acc, 32 - 17)); - acc = _mm256_mullo_epi32(acc, _mm256_set1_epi32(PRIME32_4)); - - acc = _mm256_add_epi32(acc, _mm256_mullo_epi32(x2, _mm256_set1_epi32(PRIME32_3))); - acc = _mm256_or_si256(_mm256_slli_epi32(acc, 17), _mm256_srli_epi32(acc, 32 - 17)); - acc = _mm256_mullo_epi32(acc, _mm256_set1_epi32(PRIME32_4)); - - _mm256_storeu_si256((reinterpret_cast<__m256i*>(hash)) + i, acc); - - offset = _mm256_add_epi32(offset, offset_incr); +uint32_t Hashing32::HashFixedLen_avx2(bool combine_hashes, uint32_t num_rows, + uint64_t length, const uint8_t* keys, + uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { + if (combine_hashes) { + return HashFixedLenImp_avx2(num_rows, length, keys, hashes, + hashes_temp_for_combine); + } else { + return HashFixedLenImp_avx2(num_rows, length, keys, hashes, + hashes_temp_for_combine); } } -void Hashing::hash_varlen_avx2(uint32_t num_rows, const uint32_t* offsets, - const uint8_t* concatenated_keys, - uint32_t* temp_buffer, // Needs to hold 4 x 32-bit per row - uint32_t* hashes) { - constexpr uint64_t kByteSequence0To7 = 0x0706050403020100ULL; - constexpr uint64_t kByteSequence8To15 = 0x0f0e0d0c0b0a0908ULL; +template +uint32_t Hashing32::HashVarLenImp_avx2(uint32_t num_rows, const T* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { + constexpr int unroll = 2; - const __m128i sequence = _mm_set_epi64x(kByteSequence8To15, kByteSequence0To7); - const __m128i acc_init = _mm_setr_epi32( - static_cast((static_cast(PRIME32_1) + PRIME32_2) & 0xffffffff), - PRIME32_2, 0, static_cast(-static_cast(PRIME32_1))); + // Do not process rows that could read past the end of the buffer using 16 + // byte loads. Round down number of rows to process to multiple of 2. + // + uint32_t num_rows_to_process = num_rows; + while (num_rows_to_process > 0 && + offsets[num_rows_to_process] + kStripeSize > offsets[num_rows]) { + --num_rows_to_process; + } + num_rows_to_process -= (num_rows_to_process % unroll); - // Variable length keys are always processed as a sequence of 16B stripes, - // with the last stripe, if extending past the end of the key, having extra bytes set to - // 0 on the fly. - for (uint32_t ikey = 0; ikey < num_rows; ++ikey) { - uint32_t begin = offsets[ikey]; - uint32_t end = offsets[ikey + 1]; - uint32_t length = end - begin; - const uint8_t* base = concatenated_keys + begin; - - __m128i acc = acc_init; - - if (length) { - uint32_t i; - for (i = 0; i < (length - 1) / 16; ++i) { - __m128i key_stripe = _mm_loadu_si128(reinterpret_cast(base) + i); - acc = _mm_add_epi32(acc, _mm_mullo_epi32(key_stripe, _mm_set1_epi32(PRIME32_2))); - acc = _mm_or_si128(_mm_slli_epi32(acc, 13), _mm_srli_epi32(acc, 32 - 13)); - acc = _mm_mullo_epi32(acc, _mm_set1_epi32(PRIME32_1)); - } - __m128i key_stripe = _mm_loadu_si128(reinterpret_cast(base) + i); - __m128i mask = _mm_cmpgt_epi8(_mm_set1_epi8(((length - 1) % 16) + 1), sequence); - key_stripe = _mm_and_si128(key_stripe, mask); - acc = _mm_add_epi32(acc, _mm_mullo_epi32(key_stripe, _mm_set1_epi32(PRIME32_2))); - acc = _mm_or_si128(_mm_slli_epi32(acc, 13), _mm_srli_epi32(acc, 32 - 13)); - acc = _mm_mullo_epi32(acc, _mm_set1_epi32(PRIME32_1)); - } + for (uint32_t i = 0; i < num_rows_to_process / unroll; ++i) { + T offset_A = offsets[unroll * i + 0]; + T offset_B = offsets[unroll * i + 1]; + T offset_end = offsets[unroll * i + 2]; - _mm_storeu_si128(reinterpret_cast<__m128i*>(temp_buffer) + ikey, acc); - } + T length = offset_B - offset_A; + int is_non_empty = length == 0 ? 0 : 1; + int64_t num_stripes_A = + static_cast(bit_util::CeilDiv(length, kStripeSize)) + (1 - is_non_empty); + int num_tail_bytes_A = ((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty; - // Combine accumulators and perform avalanche - constexpr int unroll = 8; - for (uint32_t i = 0; i < num_rows / unroll; ++i) { - __m256i accA = - _mm256_loadu_si256(reinterpret_cast(temp_buffer) + 4 * i + 0); - __m256i accB = - _mm256_loadu_si256(reinterpret_cast(temp_buffer) + 4 * i + 1); - __m256i accC = - _mm256_loadu_si256(reinterpret_cast(temp_buffer) + 4 * i + 2); - __m256i accD = - _mm256_loadu_si256(reinterpret_cast(temp_buffer) + 4 * i + 3); - // Transpose 2x 4x4 32-bit matrices - __m256i r0 = _mm256_unpacklo_epi32(accA, accB); - __m256i r1 = _mm256_unpackhi_epi32(accA, accB); - __m256i r2 = _mm256_unpacklo_epi32(accC, accD); - __m256i r3 = _mm256_unpackhi_epi32(accC, accD); - accA = _mm256_unpacklo_epi64(r0, r2); - accB = _mm256_unpackhi_epi64(r0, r2); - accC = _mm256_unpacklo_epi64(r1, r3); - accD = _mm256_unpackhi_epi64(r1, r3); - // _rotl(accA, 1) - // _rotl(accB, 7) - // _rotl(accC, 12) - // _rotl(accD, 18) - accA = _mm256_or_si256(_mm256_slli_epi32(accA, 1), _mm256_srli_epi32(accA, 32 - 1)); - accB = _mm256_or_si256(_mm256_slli_epi32(accB, 7), _mm256_srli_epi32(accB, 32 - 7)); - accC = _mm256_or_si256(_mm256_slli_epi32(accC, 12), _mm256_srli_epi32(accC, 32 - 12)); - accD = _mm256_or_si256(_mm256_slli_epi32(accD, 18), _mm256_srli_epi32(accD, 32 - 18)); - accA = _mm256_add_epi32(_mm256_add_epi32(accA, accB), _mm256_add_epi32(accC, accD)); - // avalanche - __m256i hash = accA; - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 15)); - hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_2)); - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 13)); - hash = _mm256_mullo_epi32(hash, _mm256_set1_epi32(PRIME32_3)); - hash = _mm256_xor_si256(hash, _mm256_srli_epi32(hash, 16)); - // Store. - // At this point, because of way 2x 4x4 transposition was done, output hashes are in - // order: 0, 2, 4, 6, 1, 3, 5, 7. Bring back the original order. - _mm256_storeu_si256( - reinterpret_cast<__m256i*>(hashes) + i, - _mm256_permutevar8x32_epi32(hash, _mm256_setr_epi32(0, 4, 1, 5, 2, 6, 3, 7))); + length = offset_end - offset_B; + is_non_empty = length == 0 ? 0 : 1; + int64_t num_stripes_B = + static_cast(bit_util::CeilDiv(length, kStripeSize)) + (1 - is_non_empty); + int num_tail_bytes_B = ((length - is_non_empty) & (kStripeSize - 1)) + is_non_empty; + + __m256i mask_last_stripe = StripeMask_avx2(num_tail_bytes_A, num_tail_bytes_B); + + __m256i acc = ProcessStripes_avx2( + num_stripes_A, num_stripes_B, mask_last_stripe, concatenated_keys, + static_cast(offset_A), static_cast(offset_B)); + + if (T_COMBINE_HASHES) { + reinterpret_cast(hashes_temp_for_combine)[i] = + CombineAccumulators_avx2(acc); + } else { + reinterpret_cast(hashes)[i] = CombineAccumulators_avx2(acc); + } } - // Process the tail of up to 7 hashes - for (uint32_t i = num_rows - num_rows % unroll; i < num_rows; ++i) { - uint32_t* temp_buffer_base = temp_buffer + i * 4; - uint32_t acc = ROTL(temp_buffer_base[0], 1) + ROTL(temp_buffer_base[1], 7) + - ROTL(temp_buffer_base[2], 12) + ROTL(temp_buffer_base[3], 18); - - // avalanche - acc ^= (acc >> 15); - acc *= PRIME32_2; - acc ^= (acc >> 13); - acc *= PRIME32_3; - acc ^= (acc >> 16); - - hashes[i] = acc; + + AvalancheAll_avx2(num_rows_to_process, hashes, + hashes_temp_for_combine); + + return num_rows_to_process; +} + +uint32_t Hashing32::HashVarLen_avx2(bool combine_hashes, uint32_t num_rows, + const uint32_t* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { + if (combine_hashes) { + return HashVarLenImp_avx2(num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); + } else { + return HashVarLenImp_avx2(num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); } } -uint32_t Hashing::HashCombine_avx2(uint32_t num_rows, uint32_t* accumulated_hash, - const uint32_t* next_column_hash) { - constexpr uint32_t unroll = 8; - for (uint32_t i = 0; i < num_rows / unroll; ++i) { - __m256i acc = - _mm256_loadu_si256(reinterpret_cast(accumulated_hash) + i); - __m256i next = - _mm256_loadu_si256(reinterpret_cast(next_column_hash) + i); - next = _mm256_add_epi32(next, _mm256_set1_epi32(0x9e3779b9)); - next = _mm256_add_epi32(next, _mm256_slli_epi32(acc, 6)); - next = _mm256_add_epi32(next, _mm256_srli_epi32(acc, 2)); - acc = _mm256_xor_si256(acc, next); - _mm256_storeu_si256(reinterpret_cast<__m256i*>(accumulated_hash) + i, acc); +uint32_t Hashing32::HashVarLen_avx2(bool combine_hashes, uint32_t num_rows, + const uint64_t* offsets, + const uint8_t* concatenated_keys, uint32_t* hashes, + uint32_t* hashes_temp_for_combine) { + if (combine_hashes) { + return HashVarLenImp_avx2(num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); + } else { + return HashVarLenImp_avx2(num_rows, offsets, concatenated_keys, + hashes, hashes_temp_for_combine); } - uint32_t num_processed = num_rows / unroll * unroll; - return num_processed; } #endif diff --git a/cpp/src/arrow/compute/exec/key_hash_test.cc b/cpp/src/arrow/compute/exec/key_hash_test.cc new file mode 100644 index 00000000000..fd08f444cb7 --- /dev/null +++ b/cpp/src/arrow/compute/exec/key_hash_test.cc @@ -0,0 +1,239 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include +#include +#include +#include "arrow/array/builder_binary.h" +#include "arrow/compute/exec/key_hash.h" +#include "arrow/compute/exec/test_util.h" +#include "arrow/compute/exec/util.h" +#include "arrow/util/cpu_info.h" +#include "arrow/util/pcg_random.h" + +namespace arrow { + +using internal::checked_pointer_cast; + +namespace compute { + +class TestVectorHash { + private: + static Result> GenerateUniqueRandomBinary( + random::pcg32_fast* random, int num, int min_length, int max_length) { + BinaryBuilder builder; + std::unordered_set unique_key_strings; + std::vector temp_buffer; + temp_buffer.resize(max_length); + std::uniform_int_distribution length_gen(min_length, max_length); + std::uniform_int_distribution byte_gen(0, + std::numeric_limits::max()); + + int num_inserted = 0; + while (num_inserted < num) { + int length = length_gen(*random); + std::generate(temp_buffer.begin(), temp_buffer.end(), + [&] { return static_cast(byte_gen(*random)); }); + std::string buffer_as_str(reinterpret_cast(temp_buffer.data()), + static_cast(length)); + if (unique_key_strings.insert(buffer_as_str).second) { + num_inserted++; + ARROW_RETURN_NOT_OK(builder.Append(temp_buffer.data(), length)); + } + } + ARROW_ASSIGN_OR_RAISE(auto uniques, builder.Finish()); + return checked_pointer_cast(uniques); + } + + static Result, std::shared_ptr>> + SampleUniqueBinary(random::pcg32_fast* random, int num, const BinaryArray& uniques) { + BinaryBuilder builder; + std::vector row_ids; + row_ids.resize(num); + + std::uniform_int_distribution row_id_gen(0, + static_cast(uniques.length()) - 1); + for (int i = 0; i < num; ++i) { + int row_id = row_id_gen(*random); + row_ids[i] = row_id; + ARROW_RETURN_NOT_OK(builder.Append(uniques.GetView(row_id))); + } + ARROW_ASSIGN_OR_RAISE(std::shared_ptr sampled, builder.Finish()); + return std::pair, std::shared_ptr>{ + std::move(row_ids), checked_pointer_cast(sampled)}; + } + + public: + static void RunSingle(random::pcg32_fast* random, bool use_32bit_hash, + bool use_varlen_input, int min_length, int max_length) { + constexpr int min_num_unique = 100; + constexpr int max_num_unique = 1000; + constexpr int min_num_rows = 4000; + constexpr int max_num_rows = 64000; + + std::uniform_int_distribution num_unique_gen(min_num_unique, max_num_unique); + std::uniform_int_distribution num_rows_gen(min_num_rows, max_num_rows); + + int num_unique = num_unique_gen(*random); + int num_rows = num_rows_gen(*random); + + SCOPED_TRACE("num_bits = " + std::to_string(use_32bit_hash ? 32 : 64) + + " varlen = " + std::string(use_varlen_input ? "yes" : "no") + + " num_unique " + std::to_string(num_unique) + " num_rows " + + std::to_string(num_rows) + " min_length " + std::to_string(min_length) + + " max_length " + std::to_string(max_length)); + + // The hash can only support 2^(length_in_bits-1) unique values + if (max_length == 1) { + num_unique &= 0x7f; + } + + std::uniform_int_distribution fixed_length_gen(min_length, max_length); + int fixed_length = use_varlen_input ? 0 : std::max(2, fixed_length_gen(*random)); + if (!use_varlen_input) { + min_length = max_length = fixed_length; + } + + ASSERT_OK_AND_ASSIGN( + std::shared_ptr uniques, + GenerateUniqueRandomBinary(random, num_unique, min_length, max_length)); + ASSERT_OK_AND_ASSIGN(auto sampled, SampleUniqueBinary(random, num_rows, *uniques)); + const std::vector& row_ids = sampled.first; + const std::shared_ptr& keys_array = sampled.second; + const uint8_t* keys = keys_array->raw_data(); + const uint32_t* key_offsets = + reinterpret_cast(keys_array->raw_value_offsets()); + + constexpr int min_rows_for_timing = 1 << 23; + int num_repeats = static_cast(bit_util::CeilDiv(min_rows_for_timing, num_rows)); +#ifndef NDEBUG + num_repeats = 1; +#endif + + std::vector hashes_scalar32; + std::vector hashes_scalar64; + hashes_scalar32.resize(num_rows); + hashes_scalar64.resize(num_rows); + std::vector hashes_simd32; + std::vector hashes_simd64; + hashes_simd32.resize(num_rows); + hashes_simd64.resize(num_rows); + + int64_t hardware_flags_scalar = 0LL; + int64_t hardware_flags_simd = ::arrow::internal::CpuInfo::AVX2; + + constexpr int mini_batch_size = 1024; + std::vector temp_buffer; + temp_buffer.resize(mini_batch_size * 4); + + for (bool use_simd : {false, true}) { + for (int i = 0; i < num_repeats; ++i) { + if (use_32bit_hash) { + if (!use_varlen_input) { + Hashing32::HashFixed(use_simd ? hardware_flags_simd : hardware_flags_scalar, + /*combine_hashes=*/false, num_rows, fixed_length, keys, + use_simd ? hashes_simd32.data() : hashes_scalar32.data(), + temp_buffer.data()); + } else { + for (int first_row = 0; first_row < num_rows;) { + int batch_size_next = std::min(num_rows - first_row, mini_batch_size); + + Hashing32::HashVarLen( + use_simd ? hardware_flags_simd : hardware_flags_scalar, + /*combine_hashes=*/false, batch_size_next, key_offsets + first_row, + keys, + (use_simd ? hashes_simd32.data() : hashes_scalar32.data()) + first_row, + temp_buffer.data()); + + first_row += batch_size_next; + } + } + } else { + if (!use_varlen_input) { + Hashing64::HashFixed( + /*combine_hashes=*/false, num_rows, fixed_length, keys, + use_simd ? hashes_simd64.data() : hashes_scalar64.data()); + } else { + Hashing64::HashVarLen( + /*combine_hashes=*/false, num_rows, key_offsets, keys, + use_simd ? hashes_simd64.data() : hashes_scalar64.data()); + } + } + } + } + + if (use_32bit_hash) { + for (int i = 0; i < num_rows; ++i) { + hashes_scalar64[i] = hashes_scalar32[i]; + hashes_simd64[i] = hashes_simd32[i]; + } + } + + // Verify that both scalar and AVX2 implementations give the same hashes + // + for (int i = 0; i < num_rows; ++i) { + ASSERT_EQ(hashes_scalar64[i], hashes_simd64[i]) + << "scalar and simd approaches yielded different hashes"; + } + + // Verify that the same key appearing multiple times generates the same hash + // each time. Measure the number of unique hashes and compare to the number + // of unique keys. + // + std::map unique_key_to_hash; + std::set unique_hashes; + for (int i = 0; i < num_rows; ++i) { + std::map::iterator iter = unique_key_to_hash.find(row_ids[i]); + if (iter == unique_key_to_hash.end()) { + unique_key_to_hash.insert(std::make_pair(row_ids[i], hashes_scalar64[i])); + } else { + ASSERT_EQ(iter->second, hashes_scalar64[i]); + } + if (unique_hashes.find(hashes_scalar64[i]) == unique_hashes.end()) { + unique_hashes.insert(hashes_scalar64[i]); + } + } + float percent_hash_collisions = + 100.0f * static_cast(num_unique - unique_hashes.size()) / + static_cast(num_unique); + SCOPED_TRACE("percent_hash_collisions " + std::to_string(percent_hash_collisions)); + ASSERT_LT(percent_hash_collisions, 5.0f) << "hash collision rate was too high"; + } +}; + +TEST(VectorHash, Basic) { + random::pcg32_fast gen(/*seed=*/0); + + int numtest = 100; + + constexpr int min_length = 0; + constexpr int max_length = 50; + + for (bool use_32bit_hash : {true, false}) { + for (bool use_varlen_input : {false, true}) { + for (int itest = 0; itest < numtest; ++itest) { + TestVectorHash::RunSingle(&gen, use_32bit_hash, use_varlen_input, min_length, + max_length); + } + } + } +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/compute/exec/partition_util.cc new file mode 100644 index 00000000000..ed5e37edca3 --- /dev/null +++ b/cpp/src/arrow/compute/exec/partition_util.cc @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/partition_util.h" +#include + +namespace arrow { +namespace compute { + +PartitionLocks::PartitionLocks() + : num_prtns_(0), + locks_(nullptr), + rand_seed_{0, 0, 0, 0, 0, 0, 0, 0}, + rand_engine_(rand_seed_) {} + +PartitionLocks::~PartitionLocks() { CleanUp(); } + +void PartitionLocks::Init(int num_prtns) { + num_prtns_ = num_prtns; + locks_.reset(new PartitionLock[num_prtns]); + for (int i = 0; i < num_prtns; ++i) { + locks_[i].lock.store(false); + } +} + +void PartitionLocks::CleanUp() { + locks_.reset(); + num_prtns_ = 0; +} + +std::atomic* PartitionLocks::lock_ptr(int prtn_id) { + ARROW_DCHECK(locks_); + ARROW_DCHECK(prtn_id >= 0 && prtn_id < num_prtns_); + return &(locks_[prtn_id].lock); +} + +int PartitionLocks::random_int(int num_values) { + return rand_distribution_(rand_engine_) % num_values; +} + +bool PartitionLocks::AcquirePartitionLock(int num_prtns_to_try, + const int* prtn_ids_to_try, bool limit_retries, + int max_retries, int* locked_prtn_id, + int* locked_prtn_id_pos) { + int trial = 0; + while (!limit_retries || trial <= max_retries) { + int prtn_id_pos = random_int(num_prtns_to_try); + int prtn_id = prtn_ids_to_try[prtn_id_pos]; + + std::atomic* lock = lock_ptr(prtn_id); + + bool expected = false; + if (lock->compare_exchange_weak(expected, true)) { + *locked_prtn_id = prtn_id; + *locked_prtn_id_pos = prtn_id_pos; + return true; + } + + ++trial; + } + + *locked_prtn_id = -1; + *locked_prtn_id_pos = -1; + return false; +} + +void PartitionLocks::ReleasePartitionLock(int prtn_id) { + ARROW_DCHECK(prtn_id >= 0 && prtn_id < num_prtns_); + std::atomic* lock = lock_ptr(prtn_id); + lock->store(false); +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h new file mode 100644 index 00000000000..6efda4aeeb0 --- /dev/null +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -0,0 +1,136 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include "arrow/buffer.h" +#include "arrow/compute/exec/util.h" + +namespace arrow { +namespace compute { + +class PartitionSort { + public: + /// \brief Bucket sort rows on partition ids in O(num_rows) time. + /// + /// Include in the output exclusive cummulative sum of bucket sizes. + /// This corresponds to ranges in the sorted array containing all row ids for + /// each of the partitions. + /// + /// prtn_ranges must be initailized and have at least prtn_ranges + 1 elements + /// when this method returns prtn_ranges[i] will contains the total number of + /// elements in partitions 0 through i. prtn_ranges[0] will be 0. + /// + /// prtn_id_impl must be a function that takes in a row id (int) and returns + /// a partition id (int). The returned partition id must be between 0 and + /// num_prtns (exclusive). + /// + /// output_pos_impl is a function that takes in a row id (int) and a position (int) + /// in the bucket sorted output. The function should insert the row in the + /// output. + /// + /// For example: + /// + /// in_arr: [5, 7, 2, 3, 5, 4] + /// num_prtns: 3 + /// prtn_id_impl: [&in_arr] (int row_id) { return in_arr[row_id] / 3; } + /// output_pos_impl: [&out_arr] (int row_id, int pos) { out_arr[pos] = row_id; } + /// + /// After Execution + /// out_arr: [2, 5, 3, 5, 4, 7] + /// prtn_ranges: [0, 1, 5, 6] + template + static void Eval(int num_rows, int num_prtns, uint16_t* prtn_ranges, + INPUT_PRTN_ID_FN prtn_id_impl, OUTPUT_POS_FN output_pos_impl) { + ARROW_DCHECK(num_rows > 0 && num_rows <= (1 << 15)); + ARROW_DCHECK(num_prtns >= 1 && num_prtns <= (1 << 15)); + + memset(prtn_ranges, 0, (num_prtns + 1) * sizeof(uint16_t)); + + for (int i = 0; i < num_rows; ++i) { + int prtn_id = static_cast(prtn_id_impl(i)); + ++prtn_ranges[prtn_id + 1]; + } + + uint16_t sum = 0; + for (int i = 0; i < num_prtns; ++i) { + uint16_t sum_next = sum + prtn_ranges[i + 1]; + prtn_ranges[i + 1] = sum; + sum = sum_next; + } + + for (int i = 0; i < num_rows; ++i) { + int prtn_id = static_cast(prtn_id_impl(i)); + int pos = prtn_ranges[prtn_id + 1]++; + output_pos_impl(i, pos); + } + } +}; + +/// \brief A control for synchronizing threads on a partitionable workload +class PartitionLocks { + public: + PartitionLocks(); + ~PartitionLocks(); + /// \brief Initializes the control, must be called before use + /// + /// \param num_prtns Number of partitions to synchronize + void Init(int num_prtns); + /// \brief Cleans up the control, it should not be used after this call + void CleanUp(); + /// \brief Acquire a partition to work on one + /// + /// \param num_prtns Length of prtns_to_try, must be <= num_prtns used in Init + /// \param prtns_to_try An array of partitions that still have remaining work + /// \param limit_retries If false, this method will spinwait forever until success + /// \param max_retries Max times to attempt checking out work before returning false + /// \param[out] locked_prtn_id The id of the partition locked + /// \param[out] locked_prtn_id_pos The index of the partition locked in prtns_to_try + /// \return True if a partition was locked, false if max_retries was attempted + /// without successfully acquiring a lock + /// + /// This method is thread safe + bool AcquirePartitionLock(int num_prtns, const int* prtns_to_try, bool limit_retries, + int max_retries, int* locked_prtn_id, + int* locked_prtn_id_pos); + /// \brief Release a partition so that other threads can work on it + void ReleasePartitionLock(int prtn_id); + + private: + std::atomic* lock_ptr(int prtn_id); + int random_int(int num_values); + + struct PartitionLock { + static constexpr int kCacheLineBytes = 64; + std::atomic lock; + uint8_t padding[kCacheLineBytes]; + }; + int num_prtns_; + std::unique_ptr locks_; + + std::seed_seq rand_seed_; + std::mt19937 rand_engine_; + std::uniform_int_distribution rand_distribution_; +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/compute/exec/test_util.h index 4a46ca95839..9347d1343f1 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/compute/exec/test_util.h @@ -21,6 +21,7 @@ #include #include +#include #include #include @@ -28,6 +29,7 @@ #include "arrow/compute/exec/exec_plan.h" #include "arrow/testing/visibility.h" #include "arrow/util/async_generator.h" +#include "arrow/util/pcg_random.h" #include "arrow/util/string_view.h" namespace arrow { @@ -112,5 +114,19 @@ bool operator==(const Declaration&, const Declaration&); ARROW_TESTING_EXPORT void PrintTo(const Declaration& decl, std::ostream* os); +class Random64Bit { + public: + explicit Random64Bit(int32_t seed) : rng_(seed) {} + uint64_t next() { return dist_(rng_); } + template + inline T from_range(const T& min_val, const T& max_val) { + return static_cast(min_val + (next() % (max_val - min_val + 1))); + } + + private: + random::pcg32_fast rng_; + std::uniform_int_distribution dist_; +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 3efb3115830..b0e423c8580 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -37,11 +37,20 @@ #if defined(__clang__) || defined(__GNUC__) #define BYTESWAP(x) __builtin_bswap64(x) -#define ROTL(x, n) (((x) << (n)) | ((x) >> (32 - (n)))) +#define ROTL(x, n) (((x) << (n)) | ((x) >> ((-n) & 31))) +#define ROTL64(x, n) (((x) << (n)) | ((x) >> ((-n) & 63))) +#define PREFETCH(ptr) __builtin_prefetch((ptr), 0 /* rw==read */, 3 /* locality */) #elif defined(_MSC_VER) #include #define BYTESWAP(x) _byteswap_uint64(x) #define ROTL(x, n) _rotl((x), (n)) +#define ROTL64(x, n) _rotl64((x), (n)) +#if defined(_M_X64) || defined(_M_I86) +#include // https://msdn.microsoft.com/fr-fr/library/84szxsww(v=vs.90).aspx +#define PREFETCH(ptr) _mm_prefetch((const char*)(ptr), _MM_HINT_T0) +#else +#define PREFETCH(ptr) (void)(ptr) /* disabled */ +#endif #endif namespace arrow { @@ -60,6 +69,17 @@ inline void CheckAlignment(const void* ptr) { // using int64_for_gather_t = const long long int; // NOLINT runtime-int +// All MiniBatch... classes use TempVectorStack for vector allocations and can +// only work with vectors up to 1024 elements. +// +// They should only be allocated on the stack to guarantee the right sequence +// of allocation and deallocation of vectors from TempVectorStack. +// +class MiniBatch { + public: + static constexpr int kMiniBatchLength = 1024; +}; + /// Storage used to allocate temporary vectors of a batch size. /// Temporary vectors should resemble allocating temporary variables on the stack /// but in the context of vectorized processing where we need to store a vector of diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 0abff01f2f4..ab8e6cd77d1 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -370,8 +370,8 @@ struct GrouperFastImpl : Grouper { encoder_.PrepareEncodeSelected(start_row, batch_size_next, cols_); // Compute hash - Hashing::HashMultiColumn(encoder_.GetBatchColumns(), &encode_ctx_, - minibatch_hashes_.data()); + Hashing32::HashMultiColumn(encoder_.GetBatchColumns(), &encode_ctx_, + minibatch_hashes_.data()); // Map auto match_bitvector =